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 ar...@apache.org on 2015/07/14 22:25:17 UTC

[01/50] hadoop git commit: HADOOP-5732. Add SFTPFileSystem. Contributed by Ramtin Boustani and Inigo Goiri

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 9e63be7ef -> 6da5a33bb


HADOOP-5732. Add SFTPFileSystem. Contributed by Ramtin Boustani and Inigo Goiri


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

Branch: refs/heads/HDFS-7240
Commit: 559425dcb9302861c3c28b759492a68f8d597092
Parents: e0febce
Author: Chris Douglas <cd...@apache.org>
Authored: Tue Jul 7 10:07:07 2015 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Tue Jul 7 10:08:59 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 hadoop-common-project/hadoop-common/pom.xml     |   5 +
 .../hadoop/fs/sftp/SFTPConnectionPool.java      | 303 +++++++++
 .../apache/hadoop/fs/sftp/SFTPFileSystem.java   | 671 +++++++++++++++++++
 .../apache/hadoop/fs/sftp/SFTPInputStream.java  | 130 ++++
 .../org/apache/hadoop/fs/sftp/package-info.java |  19 +
 .../hadoop/fs/sftp/TestSFTPFileSystem.java      | 308 +++++++++
 hadoop-project/pom.xml                          |   5 +
 8 files changed, 1444 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/559425dc/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 ee96eee..194e2e3 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -526,6 +526,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-10971. Add -C flag to make `hadoop fs -ls` print filenames only.
     (Kengo Seki via aajisaka)
 
+    HADOOP-5732. Add SFTP FileSystem. (Ramtin Boustani and Inigo Goiri via
+    cdouglas)
+
   IMPROVEMENTS
 
     HADOOP-6842. "hadoop fs -text" does not give a useful text representation

http://git-wip-us.apache.org/repos/asf/hadoop/blob/559425dc/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 c3a306b..6b1388a 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -238,6 +238,11 @@
       <artifactId>jsr305</artifactId>
       <scope>compile</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.sshd</groupId>
+      <artifactId>sshd-core</artifactId>
+      <scope>test</scope>
+    </dependency>
 
     <dependency>
       <groupId>org.apache.htrace</groupId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/559425dc/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPConnectionPool.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPConnectionPool.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPConnectionPool.java
new file mode 100644
index 0000000..c7fae7b
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPConnectionPool.java
@@ -0,0 +1,303 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.sftp;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.StringUtils;
+
+import com.jcraft.jsch.ChannelSftp;
+import com.jcraft.jsch.JSch;
+import com.jcraft.jsch.JSchException;
+import com.jcraft.jsch.Session;
+
+/** Concurrent/Multiple Connections. */
+class SFTPConnectionPool {
+
+  public static final Log LOG = LogFactory.getLog(SFTPFileSystem.class);
+  // Maximum number of allowed live connections. This doesn't mean we cannot
+  // have more live connections. It means that when we have more
+  // live connections than this threshold, any unused connection will be
+  // closed.
+  private int maxConnection;
+  private int liveConnectionCount = 0;
+  private HashMap<ConnectionInfo, HashSet<ChannelSftp>> idleConnections =
+      new HashMap<ConnectionInfo, HashSet<ChannelSftp>>();
+  private HashMap<ChannelSftp, ConnectionInfo> con2infoMap =
+      new HashMap<ChannelSftp, ConnectionInfo>();
+
+  SFTPConnectionPool(int maxConnection) {
+    this.maxConnection = maxConnection;
+  }
+
+  synchronized ChannelSftp getFromPool(ConnectionInfo info) throws IOException {
+    Set<ChannelSftp> cons = idleConnections.get(info);
+    ChannelSftp channel;
+
+    if (cons != null && cons.size() > 0) {
+      Iterator<ChannelSftp> it = cons.iterator();
+      if (it.hasNext()) {
+        channel = it.next();
+        idleConnections.remove(info);
+        return channel;
+      } else {
+        throw new IOException("Connection pool error.");
+      }
+    }
+    return null;
+  }
+
+  /** Add the channel into pool.
+   * @param channel
+   */
+  synchronized void returnToPool(ChannelSftp channel) {
+    ConnectionInfo info = con2infoMap.get(channel);
+    HashSet<ChannelSftp> cons = idleConnections.get(info);
+    if (cons == null) {
+      cons = new HashSet<ChannelSftp>();
+      idleConnections.put(info, cons);
+    }
+    cons.add(channel);
+
+  }
+
+  /** Shutdown the connection pool and close all open connections. */
+  synchronized void shutdown() {
+    if (this.con2infoMap == null){
+      return; // already shutdown in case it is called
+    }
+    LOG.info("Inside shutdown, con2infoMap size=" + con2infoMap.size());
+
+    this.maxConnection = 0;
+    Set<ChannelSftp> cons = con2infoMap.keySet();
+    if (cons != null && cons.size() > 0) {
+      // make a copy since we need to modify the underlying Map
+      Set<ChannelSftp> copy = new HashSet<ChannelSftp>(cons);
+      // Initiate disconnect from all outstanding connections
+      for (ChannelSftp con : copy) {
+        try {
+          disconnect(con);
+        } catch (IOException ioe) {
+          ConnectionInfo info = con2infoMap.get(con);
+          LOG.error(
+              "Error encountered while closing connection to " + info.getHost(),
+              ioe);
+        }
+      }
+    }
+    // make sure no further connections can be returned.
+    this.idleConnections = null;
+    this.con2infoMap = null;
+  }
+
+  public synchronized int getMaxConnection() {
+    return maxConnection;
+  }
+
+  public synchronized void setMaxConnection(int maxConn) {
+    this.maxConnection = maxConn;
+  }
+
+  public ChannelSftp connect(String host, int port, String user,
+      String password, String keyFile) throws IOException {
+    // get connection from pool
+    ConnectionInfo info = new ConnectionInfo(host, port, user);
+    ChannelSftp channel = getFromPool(info);
+
+    if (channel != null) {
+      if (channel.isConnected()) {
+        return channel;
+      } else {
+        channel = null;
+        synchronized (this) {
+          --liveConnectionCount;
+          con2infoMap.remove(channel);
+        }
+      }
+    }
+
+    // create a new connection and add to pool
+    JSch jsch = new JSch();
+    Session session = null;
+    try {
+      if (user == null || user.length() == 0) {
+        user = System.getProperty("user.name");
+      }
+
+      if (password == null) {
+        password = "";
+      }
+
+      if (keyFile != null && keyFile.length() > 0) {
+        jsch.addIdentity(keyFile);
+      }
+
+      if (port <= 0) {
+        session = jsch.getSession(user, host);
+      } else {
+        session = jsch.getSession(user, host, port);
+      }
+
+      session.setPassword(password);
+
+      java.util.Properties config = new java.util.Properties();
+      config.put("StrictHostKeyChecking", "no");
+      session.setConfig(config);
+
+      session.connect();
+      channel = (ChannelSftp) session.openChannel("sftp");
+      channel.connect();
+
+      synchronized (this) {
+        con2infoMap.put(channel, info);
+        liveConnectionCount++;
+      }
+
+      return channel;
+
+    } catch (JSchException e) {
+      throw new IOException(StringUtils.stringifyException(e));
+    }
+  }
+
+  void disconnect(ChannelSftp channel) throws IOException {
+    if (channel != null) {
+      // close connection if too many active connections
+      boolean closeConnection = false;
+      synchronized (this) {
+        if (liveConnectionCount > maxConnection) {
+          --liveConnectionCount;
+          con2infoMap.remove(channel);
+          closeConnection = true;
+        }
+      }
+      if (closeConnection) {
+        if (channel.isConnected()) {
+          try {
+            Session session = channel.getSession();
+            channel.disconnect();
+            session.disconnect();
+          } catch (JSchException e) {
+            throw new IOException(StringUtils.stringifyException(e));
+          }
+        }
+
+      } else {
+        returnToPool(channel);
+      }
+    }
+  }
+
+  public int getIdleCount() {
+    return this.idleConnections.size();
+  }
+
+  public int getLiveConnCount() {
+    return this.liveConnectionCount;
+  }
+
+  public int getConnPoolSize() {
+    return this.con2infoMap.size();
+  }
+
+  /**
+   * Class to capture the minimal set of information that distinguish
+   * between different connections.
+   */
+  static class ConnectionInfo {
+    private String host = "";
+    private int port;
+    private String user = "";
+
+    ConnectionInfo(String hst, int prt, String usr) {
+      this.host = hst;
+      this.port = prt;
+      this.user = usr;
+    }
+
+    public String getHost() {
+      return host;
+    }
+
+    public void setHost(String hst) {
+      this.host = hst;
+    }
+
+    public int getPort() {
+      return port;
+    }
+
+    public void setPort(int prt) {
+      this.port = prt;
+    }
+
+    public String getUser() {
+      return user;
+    }
+
+    public void setUser(String usr) {
+      this.user = usr;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) {
+        return true;
+      }
+
+      if (obj instanceof ConnectionInfo) {
+        ConnectionInfo con = (ConnectionInfo) obj;
+
+        boolean ret = true;
+        if (this.host == null || !this.host.equalsIgnoreCase(con.host)) {
+          ret = false;
+        }
+        if (this.port >= 0 && this.port != con.port) {
+          ret = false;
+        }
+        if (this.user == null || !this.user.equalsIgnoreCase(con.user)) {
+          ret = false;
+        }
+        return ret;
+      } else {
+        return false;
+      }
+
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 0;
+      if (host != null) {
+        hashCode += host.hashCode();
+      }
+      hashCode += port;
+      if (user != null) {
+        hashCode += user.hashCode();
+      }
+      return hashCode;
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/559425dc/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java
new file mode 100644
index 0000000..8b6267a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java
@@ -0,0 +1,671 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.sftp;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.net.URLDecoder;
+import java.util.ArrayList;
+import java.util.Vector;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Progressable;
+
+import com.jcraft.jsch.ChannelSftp;
+import com.jcraft.jsch.ChannelSftp.LsEntry;
+import com.jcraft.jsch.SftpATTRS;
+import com.jcraft.jsch.SftpException;
+
+/** SFTP FileSystem. */
+public class SFTPFileSystem extends FileSystem {
+
+  public static final Log LOG = LogFactory.getLog(SFTPFileSystem.class);
+
+  private SFTPConnectionPool connectionPool;
+  private URI uri;
+
+  private static final int DEFAULT_SFTP_PORT = 22;
+  private static final int DEFAULT_MAX_CONNECTION = 5;
+  public static final int DEFAULT_BUFFER_SIZE = 1024 * 1024;
+  public static final int DEFAULT_BLOCK_SIZE = 4 * 1024;
+  public static final String FS_SFTP_USER_PREFIX = "fs.sftp.user.";
+  public static final String FS_SFTP_PASSWORD_PREFIX = "fs.sftp.password.";
+  public static final String FS_SFTP_HOST = "fs.sftp.host";
+  public static final String FS_SFTP_HOST_PORT = "fs.sftp.host.port";
+  public static final String FS_SFTP_KEYFILE = "fs.sftp.keyfile";
+  public static final String FS_SFTP_CONNECTION_MAX = "fs.sftp.connection.max";
+  public static final String E_SAME_DIRECTORY_ONLY =
+      "only same directory renames are supported";
+  public static final String E_HOST_NULL = "Invalid host specified";
+  public static final String E_USER_NULL =
+      "No user specified for sftp connection. Expand URI or credential file.";
+  public static final String E_PATH_DIR = "Path %s is a directory.";
+  public static final String E_FILE_STATUS = "Failed to get file status";
+  public static final String E_FILE_NOTFOUND = "File %s does not exist.";
+  public static final String E_FILE_EXIST = "File already exists: %s";
+  public static final String E_CREATE_DIR =
+      "create(): Mkdirs failed to create: %s";
+  public static final String E_DIR_CREATE_FROMFILE =
+      "Can't make directory for path %s since it is a file.";
+  public static final String E_MAKE_DIR_FORPATH =
+      "Can't make directory for path \"%s\" under \"%s\".";
+  public static final String E_DIR_NOTEMPTY = "Directory: %s is not empty.";
+  public static final String E_FILE_CHECK_FAILED = "File check failed";
+  public static final String E_NOT_SUPPORTED = "Not supported";
+  public static final String E_SPATH_NOTEXIST = "Source path %s does not exist";
+  public static final String E_DPATH_EXIST =
+      "Destination path %s already exist, cannot rename!";
+  public static final String E_FAILED_GETHOME = "Failed to get home directory";
+  public static final String E_FAILED_DISCONNECT = "Failed to disconnect";
+
+  /**
+   * Set configuration from UI.
+   *
+   * @param uri
+   * @param conf
+   * @throws IOException
+   */
+  private void setConfigurationFromURI(URI uriInfo, Configuration conf)
+      throws IOException {
+
+    // get host information from URI
+    String host = uriInfo.getHost();
+    host = (host == null) ? conf.get(FS_SFTP_HOST, null) : host;
+    if (host == null) {
+      throw new IOException(E_HOST_NULL);
+    }
+    conf.set(FS_SFTP_HOST, host);
+
+    int port = uriInfo.getPort();
+    port = (port == -1)
+      ? conf.getInt(FS_SFTP_HOST_PORT, DEFAULT_SFTP_PORT)
+      : port;
+    conf.setInt(FS_SFTP_HOST_PORT, port);
+
+    // get user/password information from URI
+    String userAndPwdFromUri = uriInfo.getUserInfo();
+    if (userAndPwdFromUri != null) {
+      String[] userPasswdInfo = userAndPwdFromUri.split(":");
+      String user = userPasswdInfo[0];
+      user = URLDecoder.decode(user, "UTF-8");
+      conf.set(FS_SFTP_USER_PREFIX + host, user);
+      if (userPasswdInfo.length > 1) {
+        conf.set(FS_SFTP_PASSWORD_PREFIX + host + "." +
+            user, userPasswdInfo[1]);
+      }
+    }
+
+    String user = conf.get(FS_SFTP_USER_PREFIX + host);
+    if (user == null || user.equals("")) {
+      throw new IllegalStateException(E_USER_NULL);
+    }
+
+    int connectionMax =
+        conf.getInt(FS_SFTP_CONNECTION_MAX, DEFAULT_MAX_CONNECTION);
+    connectionPool = new SFTPConnectionPool(connectionMax);
+  }
+
+  /**
+   * Connecting by using configuration parameters.
+   *
+   * @return An FTPClient instance
+   * @throws IOException
+   */
+  private ChannelSftp connect() throws IOException {
+    Configuration conf = getConf();
+
+    String host = conf.get(FS_SFTP_HOST, null);
+    int port = conf.getInt(FS_SFTP_HOST_PORT, DEFAULT_SFTP_PORT);
+    String user = conf.get(FS_SFTP_USER_PREFIX + host, null);
+    String pwd = conf.get(FS_SFTP_PASSWORD_PREFIX + host + "." + user, null);
+    String keyFile = conf.get(FS_SFTP_KEYFILE, null);
+
+    ChannelSftp channel =
+        connectionPool.connect(host, port, user, pwd, keyFile);
+
+    return channel;
+  }
+
+  /**
+   * Logout and disconnect the given channel.
+   *
+   * @param client
+   * @throws IOException
+   */
+  private void disconnect(ChannelSftp channel) throws IOException {
+    connectionPool.disconnect(channel);
+  }
+
+  /**
+   * Resolve against given working directory.
+   *
+   * @param workDir
+   * @param path
+   * @return absolute path
+   */
+  private Path makeAbsolute(Path workDir, Path path) {
+    if (path.isAbsolute()) {
+      return path;
+    }
+    return new Path(workDir, path);
+  }
+
+  /**
+   * Convenience method, so that we don't open a new connection when using this
+   * method from within another method. Otherwise every API invocation incurs
+   * the overhead of opening/closing a TCP connection.
+   * @throws IOException
+   */
+  private boolean exists(ChannelSftp channel, Path file) throws IOException {
+    try {
+      getFileStatus(channel, file);
+      return true;
+    } catch (FileNotFoundException fnfe) {
+      return false;
+    } catch (IOException ioe) {
+      throw new IOException(E_FILE_STATUS, ioe);
+    }
+  }
+
+  /**
+   * Convenience method, so that we don't open a new connection when using this
+   * method from within another method. Otherwise every API invocation incurs
+   * the overhead of opening/closing a TCP connection.
+   */
+  @SuppressWarnings("unchecked")
+  private FileStatus getFileStatus(ChannelSftp client, Path file)
+      throws IOException {
+    FileStatus fileStat = null;
+    Path workDir;
+    try {
+      workDir = new Path(client.pwd());
+    } catch (SftpException e) {
+      throw new IOException(e);
+    }
+    Path absolute = makeAbsolute(workDir, file);
+    Path parentPath = absolute.getParent();
+    if (parentPath == null) { // root directory
+      long length = -1; // Length of root directory on server not known
+      boolean isDir = true;
+      int blockReplication = 1;
+      long blockSize = DEFAULT_BLOCK_SIZE; // Block Size not known.
+      long modTime = -1; // Modification time of root directory not known.
+      Path root = new Path("/");
+      return new FileStatus(length, isDir, blockReplication, blockSize,
+          modTime,
+          root.makeQualified(this.getUri(), this.getWorkingDirectory()));
+    }
+    String pathName = parentPath.toUri().getPath();
+    Vector<LsEntry> sftpFiles;
+    try {
+      sftpFiles = (Vector<LsEntry>) client.ls(pathName);
+    } catch (SftpException e) {
+      throw new FileNotFoundException(String.format(E_FILE_NOTFOUND, file));
+    }
+    if (sftpFiles != null) {
+      for (LsEntry sftpFile : sftpFiles) {
+        if (sftpFile.getFilename().equals(file.getName())) {
+          // file found in directory
+          fileStat = getFileStatus(client, sftpFile, parentPath);
+          break;
+        }
+      }
+      if (fileStat == null) {
+        throw new FileNotFoundException(String.format(E_FILE_NOTFOUND, file));
+      }
+    } else {
+      throw new FileNotFoundException(String.format(E_FILE_NOTFOUND, file));
+    }
+    return fileStat;
+  }
+
+  /**
+   * Convert the file information in LsEntry to a {@link FileStatus} object. *
+   *
+   * @param sftpFile
+   * @param parentPath
+   * @return file status
+   * @throws IOException
+   */
+  private FileStatus getFileStatus(ChannelSftp channel, LsEntry sftpFile,
+      Path parentPath) throws IOException {
+
+    SftpATTRS attr = sftpFile.getAttrs();
+    long length = attr.getSize();
+    boolean isDir = attr.isDir();
+    boolean isLink = attr.isLink();
+    if (isLink) {
+      String link = parentPath.toUri().getPath() + "/" + sftpFile.getFilename();
+      try {
+        link = channel.realpath(link);
+
+        Path linkParent = new Path("/", link);
+
+        FileStatus fstat = getFileStatus(channel, linkParent);
+        isDir = fstat.isDirectory();
+        length = fstat.getLen();
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+    }
+    int blockReplication = 1;
+    // Using default block size since there is no way in SFTP channel to know of
+    // block sizes on server. The assumption could be less than ideal.
+    long blockSize = DEFAULT_BLOCK_SIZE;
+    long modTime = attr.getMTime() * 1000; // convert to milliseconds
+    long accessTime = 0;
+    FsPermission permission = getPermissions(sftpFile);
+    // not be able to get the real user group name, just use the user and group
+    // id
+    String user = Integer.toString(attr.getUId());
+    String group = Integer.toString(attr.getGId());
+    Path filePath = new Path(parentPath, sftpFile.getFilename());
+
+    return new FileStatus(length, isDir, blockReplication, blockSize, modTime,
+        accessTime, permission, user, group, filePath.makeQualified(
+            this.getUri(), this.getWorkingDirectory()));
+  }
+
+  /**
+   * Return file permission.
+   *
+   * @param sftpFile
+   * @return file permission
+   */
+  private FsPermission getPermissions(LsEntry sftpFile) {
+    return new FsPermission((short) sftpFile.getAttrs().getPermissions());
+  }
+
+  /**
+   * Convenience method, so that we don't open a new connection when using this
+   * method from within another method. Otherwise every API invocation incurs
+   * the overhead of opening/closing a TCP connection.
+   */
+  private boolean mkdirs(ChannelSftp client, Path file, FsPermission permission)
+      throws IOException {
+    boolean created = true;
+    Path workDir;
+    try {
+      workDir = new Path(client.pwd());
+    } catch (SftpException e) {
+      throw new IOException(e);
+    }
+    Path absolute = makeAbsolute(workDir, file);
+    String pathName = absolute.getName();
+    if (!exists(client, absolute)) {
+      Path parent = absolute.getParent();
+      created =
+          (parent == null || mkdirs(client, parent, FsPermission.getDefault()));
+      if (created) {
+        String parentDir = parent.toUri().getPath();
+        boolean succeeded = true;
+        try {
+          client.cd(parentDir);
+          client.mkdir(pathName);
+        } catch (SftpException e) {
+          throw new IOException(String.format(E_MAKE_DIR_FORPATH, pathName,
+              parentDir));
+        }
+        created = created & succeeded;
+      }
+    } else if (isFile(client, absolute)) {
+      throw new IOException(String.format(E_DIR_CREATE_FROMFILE, absolute));
+    }
+    return created;
+  }
+
+  /**
+   * Convenience method, so that we don't open a new connection when using this
+   * method from within another method. Otherwise every API invocation incurs
+   * the overhead of opening/closing a TCP connection.
+   * @throws IOException
+   */
+  private boolean isFile(ChannelSftp channel, Path file) throws IOException {
+    try {
+      return !getFileStatus(channel, file).isDirectory();
+    } catch (FileNotFoundException e) {
+      return false; // file does not exist
+    } catch (IOException ioe) {
+      throw new IOException(E_FILE_CHECK_FAILED, ioe);
+    }
+  }
+
+  /**
+   * Convenience method, so that we don't open a new connection when using this
+   * method from within another method. Otherwise every API invocation incurs
+   * the overhead of opening/closing a TCP connection.
+   */
+  private boolean delete(ChannelSftp channel, Path file, boolean recursive)
+      throws IOException {
+    Path workDir;
+    try {
+      workDir = new Path(channel.pwd());
+    } catch (SftpException e) {
+      throw new IOException(e);
+    }
+    Path absolute = makeAbsolute(workDir, file);
+    String pathName = absolute.toUri().getPath();
+    FileStatus fileStat = null;
+    try {
+      fileStat = getFileStatus(channel, absolute);
+    } catch (FileNotFoundException e) {
+      // file not found, no need to delete, return true
+      return false;
+    }
+    if (!fileStat.isDirectory()) {
+      boolean status = true;
+      try {
+        channel.rm(pathName);
+      } catch (SftpException e) {
+        status = false;
+      }
+      return status;
+    } else {
+      boolean status = true;
+      FileStatus[] dirEntries = listStatus(channel, absolute);
+      if (dirEntries != null && dirEntries.length > 0) {
+        if (!recursive) {
+          throw new IOException(String.format(E_DIR_NOTEMPTY, file));
+        }
+        for (int i = 0; i < dirEntries.length; ++i) {
+          delete(channel, new Path(absolute, dirEntries[i].getPath()),
+              recursive);
+        }
+      }
+      try {
+        channel.rmdir(pathName);
+      } catch (SftpException e) {
+        status = false;
+      }
+      return status;
+    }
+  }
+
+  /**
+   * Convenience method, so that we don't open a new connection when using this
+   * method from within another method. Otherwise every API invocation incurs
+   * the overhead of opening/closing a TCP connection.
+   */
+  @SuppressWarnings("unchecked")
+  private FileStatus[] listStatus(ChannelSftp client, Path file)
+      throws IOException {
+    Path workDir;
+    try {
+      workDir = new Path(client.pwd());
+    } catch (SftpException e) {
+      throw new IOException(e);
+    }
+    Path absolute = makeAbsolute(workDir, file);
+    FileStatus fileStat = getFileStatus(client, absolute);
+    if (!fileStat.isDirectory()) {
+      return new FileStatus[] {fileStat};
+    }
+    Vector<LsEntry> sftpFiles;
+    try {
+      sftpFiles = (Vector<LsEntry>) client.ls(absolute.toUri().getPath());
+    } catch (SftpException e) {
+      throw new IOException(e);
+    }
+    ArrayList<FileStatus> fileStats = new ArrayList<FileStatus>();
+    for (int i = 0; i < sftpFiles.size(); i++) {
+      LsEntry entry = sftpFiles.get(i);
+      String fname = entry.getFilename();
+      // skip current and parent directory, ie. "." and ".."
+      if (!".".equalsIgnoreCase(fname) && !"..".equalsIgnoreCase(fname)) {
+        fileStats.add(getFileStatus(client, entry, absolute));
+      }
+    }
+    return fileStats.toArray(new FileStatus[fileStats.size()]);
+  }
+
+  /**
+   * Convenience method, so that we don't open a new connection when using this
+   * method from within another method. Otherwise every API invocation incurs
+   * the overhead of opening/closing a TCP connection.
+   *
+   * @param channel
+   * @param src
+   * @param dst
+   * @return rename successful?
+   * @throws IOException
+   */
+  private boolean rename(ChannelSftp channel, Path src, Path dst)
+      throws IOException {
+    Path workDir;
+    try {
+      workDir = new Path(channel.pwd());
+    } catch (SftpException e) {
+      throw new IOException(e);
+    }
+    Path absoluteSrc = makeAbsolute(workDir, src);
+    Path absoluteDst = makeAbsolute(workDir, dst);
+
+    if (!exists(channel, absoluteSrc)) {
+      throw new IOException(String.format(E_SPATH_NOTEXIST, src));
+    }
+    if (exists(channel, absoluteDst)) {
+      throw new IOException(String.format(E_DPATH_EXIST, dst));
+    }
+    boolean renamed = true;
+    try {
+      channel.cd("/");
+      channel.rename(src.toUri().getPath(), dst.toUri().getPath());
+    } catch (SftpException e) {
+      renamed = false;
+    }
+    return renamed;
+  }
+
+  @Override
+  public void initialize(URI uriInfo, Configuration conf) throws IOException {
+    super.initialize(uriInfo, conf);
+
+    setConfigurationFromURI(uriInfo, conf);
+    setConf(conf);
+    this.uri = uriInfo;
+  }
+
+  @Override
+  public URI getUri() {
+    return uri;
+  }
+
+  @Override
+  public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+    ChannelSftp channel = connect();
+    Path workDir;
+    try {
+      workDir = new Path(channel.pwd());
+    } catch (SftpException e) {
+      throw new IOException(e);
+    }
+    Path absolute = makeAbsolute(workDir, f);
+    FileStatus fileStat = getFileStatus(channel, absolute);
+    if (fileStat.isDirectory()) {
+      disconnect(channel);
+      throw new IOException(String.format(E_PATH_DIR, f));
+    }
+    InputStream is;
+    try {
+      // the path could be a symbolic link, so get the real path
+      absolute = new Path("/", channel.realpath(absolute.toUri().getPath()));
+
+      is = channel.get(absolute.toUri().getPath());
+    } catch (SftpException e) {
+      throw new IOException(e);
+    }
+
+    FSDataInputStream fis =
+        new FSDataInputStream(new SFTPInputStream(is, channel, statistics));
+    return fis;
+  }
+
+  /**
+   * A stream obtained via this call must be closed before using other APIs of
+   * this class or else the invocation will block.
+   */
+  @Override
+  public FSDataOutputStream create(Path f, FsPermission permission,
+      boolean overwrite, int bufferSize, short replication, long blockSize,
+      Progressable progress) throws IOException {
+    final ChannelSftp client = connect();
+    Path workDir;
+    try {
+      workDir = new Path(client.pwd());
+    } catch (SftpException e) {
+      throw new IOException(e);
+    }
+    Path absolute = makeAbsolute(workDir, f);
+    if (exists(client, f)) {
+      if (overwrite) {
+        delete(client, f, false);
+      } else {
+        disconnect(client);
+        throw new IOException(String.format(E_FILE_EXIST, f));
+      }
+    }
+    Path parent = absolute.getParent();
+    if (parent == null || !mkdirs(client, parent, FsPermission.getDefault())) {
+      parent = (parent == null) ? new Path("/") : parent;
+      disconnect(client);
+      throw new IOException(String.format(E_CREATE_DIR, parent));
+    }
+    OutputStream os;
+    try {
+      client.cd(parent.toUri().getPath());
+      os = client.put(f.getName());
+    } catch (SftpException e) {
+      throw new IOException(e);
+    }
+    FSDataOutputStream fos = new FSDataOutputStream(os, statistics) {
+      @Override
+      public void close() throws IOException {
+        super.close();
+        disconnect(client);
+      }
+    };
+
+    return fos;
+  }
+
+  @Override
+  public FSDataOutputStream append(Path f, int bufferSize,
+      Progressable progress)
+      throws IOException {
+    throw new IOException(E_NOT_SUPPORTED);
+  }
+
+  /*
+   * The parent of source and destination can be different. It is suppose to
+   * work like 'move'
+   */
+  @Override
+  public boolean rename(Path src, Path dst) throws IOException {
+    ChannelSftp channel = connect();
+    try {
+      boolean success = rename(channel, src, dst);
+      return success;
+    } finally {
+      disconnect(channel);
+    }
+  }
+
+  @Override
+  public boolean delete(Path f, boolean recursive) throws IOException {
+    ChannelSftp channel = connect();
+    try {
+      boolean success = delete(channel, f, recursive);
+      return success;
+    } finally {
+      disconnect(channel);
+    }
+  }
+
+  @Override
+  public FileStatus[] listStatus(Path f) throws IOException {
+    ChannelSftp client = connect();
+    try {
+      FileStatus[] stats = listStatus(client, f);
+      return stats;
+    } finally {
+      disconnect(client);
+    }
+  }
+
+  @Override
+  public void setWorkingDirectory(Path newDir) {
+    // we do not maintain the working directory state
+  }
+
+  @Override
+  public Path getWorkingDirectory() {
+    // Return home directory always since we do not maintain state.
+    return getHomeDirectory();
+  }
+
+  @Override
+  public Path getHomeDirectory() {
+    ChannelSftp channel = null;
+    try {
+      channel = connect();
+      Path homeDir = new Path(channel.pwd());
+      return homeDir;
+    } catch (Exception ioe) {
+      return null;
+    } finally {
+      try {
+        disconnect(channel);
+      } catch (IOException ioe) {
+        return null;
+      }
+    }
+  }
+
+  @Override
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+    ChannelSftp client = connect();
+    try {
+      boolean success = mkdirs(client, f, permission);
+      return success;
+    } finally {
+      disconnect(client);
+    }
+  }
+
+  @Override
+  public FileStatus getFileStatus(Path f) throws IOException {
+    ChannelSftp channel = connect();
+    try {
+      FileStatus status = getFileStatus(channel, f);
+      return status;
+    } finally {
+      disconnect(channel);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/559425dc/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPInputStream.java
new file mode 100644
index 0000000..ece2c1c
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPInputStream.java
@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.sftp;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.util.StringUtils;
+
+import com.jcraft.jsch.ChannelSftp;
+import com.jcraft.jsch.JSchException;
+import com.jcraft.jsch.Session;
+
+/** SFTP FileSystem input stream. */
+class SFTPInputStream extends FSInputStream {
+
+  public static final String E_SEEK_NOTSUPPORTED = "Seek not supported";
+  public static final String E_CLIENT_NULL =
+      "SFTP client null or not connected";
+  public static final String E_NULL_INPUTSTREAM = "Null InputStream";
+  public static final String E_STREAM_CLOSED = "Stream closed";
+  public static final String E_CLIENT_NOTCONNECTED = "Client not connected";
+
+  private InputStream wrappedStream;
+  private ChannelSftp channel;
+  private FileSystem.Statistics stats;
+  private boolean closed;
+  private long pos;
+
+  SFTPInputStream(InputStream stream, ChannelSftp channel,
+      FileSystem.Statistics stats) {
+
+    if (stream == null) {
+      throw new IllegalArgumentException(E_NULL_INPUTSTREAM);
+    }
+    if (channel == null || !channel.isConnected()) {
+      throw new IllegalArgumentException(E_CLIENT_NULL);
+    }
+    this.wrappedStream = stream;
+    this.channel = channel;
+    this.stats = stats;
+
+    this.pos = 0;
+    this.closed = false;
+  }
+
+  @Override
+  public void seek(long position) throws IOException {
+    throw new IOException(E_SEEK_NOTSUPPORTED);
+  }
+
+  @Override
+  public boolean seekToNewSource(long targetPos) throws IOException {
+    throw new IOException(E_SEEK_NOTSUPPORTED);
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return pos;
+  }
+
+  @Override
+  public synchronized int read() throws IOException {
+    if (closed) {
+      throw new IOException(E_STREAM_CLOSED);
+    }
+
+    int byteRead = wrappedStream.read();
+    if (byteRead >= 0) {
+      pos++;
+    }
+    if (stats != null & byteRead >= 0) {
+      stats.incrementBytesRead(1);
+    }
+    return byteRead;
+  }
+
+  public synchronized int read(byte[] buf, int off, int len)
+      throws IOException {
+    if (closed) {
+      throw new IOException(E_STREAM_CLOSED);
+    }
+
+    int result = wrappedStream.read(buf, off, len);
+    if (result > 0) {
+      pos += result;
+    }
+    if (stats != null & result > 0) {
+      stats.incrementBytesRead(result);
+    }
+
+    return result;
+  }
+
+  public synchronized void close() throws IOException {
+    if (closed) {
+      return;
+    }
+    super.close();
+    closed = true;
+    if (!channel.isConnected()) {
+      throw new IOException(E_CLIENT_NOTCONNECTED);
+    }
+
+    try {
+      Session session = channel.getSession();
+      channel.disconnect();
+      session.disconnect();
+    } catch (JSchException e) {
+      throw new IOException(StringUtils.stringifyException(e));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/559425dc/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/package-info.java
new file mode 100644
index 0000000..1427e48
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+/** SFTP FileSystem package. */
+package org.apache.hadoop.fs.sftp;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/559425dc/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/sftp/TestSFTPFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/sftp/TestSFTPFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/sftp/TestSFTPFileSystem.java
new file mode 100644
index 0000000..06d9bf0
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/sftp/TestSFTPFileSystem.java
@@ -0,0 +1,308 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.fs.sftp;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.Shell;
+
+import org.apache.sshd.SshServer;
+import org.apache.sshd.common.NamedFactory;
+import org.apache.sshd.server.Command;
+import org.apache.sshd.server.PasswordAuthenticator;
+import org.apache.sshd.server.UserAuth;
+import org.apache.sshd.server.auth.UserAuthPassword;
+import org.apache.sshd.server.keyprovider.SimpleGeneratorHostKeyProvider;
+import org.apache.sshd.server.session.ServerSession;
+import org.apache.sshd.server.sftp.SftpSubsystem;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+import static org.junit.Assert.*;
+import static org.junit.Assume.assumeTrue;
+
+public class TestSFTPFileSystem {
+
+  private static final String TEST_SFTP_DIR = "testsftp";
+  private static final String TEST_ROOT_DIR =
+    System.getProperty("test.build.data", "build/test/data");
+
+  @Rule public TestName name = new TestName();
+
+  private static final String connection = "sftp://user:password@localhost";
+  private static Path localDir = null;
+  private static FileSystem localFs = null;
+  private static FileSystem sftpFs = null;
+  private static SshServer sshd = null;
+  private static int port;
+
+  private static void startSshdServer() throws IOException {
+    sshd = SshServer.setUpDefaultServer();
+    // ask OS to assign a port
+    sshd.setPort(0);
+    sshd.setKeyPairProvider(new SimpleGeneratorHostKeyProvider());
+
+    List<NamedFactory<UserAuth>> userAuthFactories =
+        new ArrayList<NamedFactory<UserAuth>>();
+    userAuthFactories.add(new UserAuthPassword.Factory());
+
+    sshd.setUserAuthFactories(userAuthFactories);
+
+    sshd.setPasswordAuthenticator(new PasswordAuthenticator() {
+      @Override
+      public boolean authenticate(String username, String password,
+          ServerSession session) {
+        if (username.equals("user") && password.equals("password")) {
+          return true;
+        }
+        return false;
+      }
+    });
+
+    sshd.setSubsystemFactories(
+        Arrays.<NamedFactory<Command>>asList(new SftpSubsystem.Factory()));
+
+    sshd.start();
+    port = sshd.getPort();
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    // skip all tests if running on Windows
+    assumeTrue(!Shell.WINDOWS);
+
+    startSshdServer();
+
+    Configuration conf = new Configuration();
+    conf.setClass("fs.sftp.impl", SFTPFileSystem.class, FileSystem.class);
+    conf.setInt("fs.sftp.host.port", port);
+    conf.setBoolean("fs.sftp.impl.disable.cache", true);
+
+    localFs = FileSystem.getLocal(conf);
+    localDir = localFs.makeQualified(new Path(TEST_ROOT_DIR, TEST_SFTP_DIR));
+    if (localFs.exists(localDir)) {
+      localFs.delete(localDir, true);
+    }
+    localFs.mkdirs(localDir);
+
+    sftpFs = FileSystem.get(URI.create(connection), conf);
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    if (localFs != null) {
+      try {
+        localFs.delete(localDir, true);
+        localFs.close();
+      } catch (IOException e) {
+        // ignore
+      }
+    }
+    if (sftpFs != null) {
+      try {
+        sftpFs.close();
+      } catch (IOException e) {
+        // ignore
+      }
+    }
+    if (sshd != null) {
+      try {
+        sshd.stop(true);
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+  }
+
+  private static final Path touch(FileSystem fs, String filename)
+      throws IOException {
+    return touch(fs, filename, null);
+  }
+
+  private static final Path touch(FileSystem fs, String filename, byte[] data)
+      throws IOException {
+    Path lPath = new Path(localDir.toUri().getPath(), filename);
+    FSDataOutputStream out = null;
+    try {
+      out = fs.create(lPath);
+      if (data != null) {
+        out.write(data);
+      }
+    } finally {
+      if (out != null) {
+        out.close();
+      }
+    }
+    return lPath;
+  }
+
+  /**
+   * Creates a file and deletes it.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testCreateFile() throws Exception {
+    Path file = touch(sftpFs, name.getMethodName().toLowerCase());
+    assertTrue(localFs.exists(file));
+    assertTrue(sftpFs.delete(file, false));
+    assertFalse(localFs.exists(file));
+  }
+
+  /**
+   * Checks if a new created file exists.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testFileExists() throws Exception {
+    Path file = touch(localFs, name.getMethodName().toLowerCase());
+    assertTrue(sftpFs.exists(file));
+    assertTrue(localFs.exists(file));
+    assertTrue(sftpFs.delete(file, false));
+    assertFalse(sftpFs.exists(file));
+    assertFalse(localFs.exists(file));
+  }
+
+  /**
+   * Test writing to a file and reading its value.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testReadFile() throws Exception {
+    byte[] data = "yaks".getBytes();
+    Path file = touch(localFs, name.getMethodName().toLowerCase(), data);
+    FSDataInputStream is = null;
+    try {
+      is = sftpFs.open(file);
+      byte[] b = new byte[data.length];
+      is.read(b);
+      assertArrayEquals(data, b);
+    } finally {
+      if (is != null) {
+        is.close();
+      }
+    }
+    assertTrue(sftpFs.delete(file, false));
+  }
+
+  /**
+   * Test getting the status of a file.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testStatFile() throws Exception {
+    byte[] data = "yaks".getBytes();
+    Path file = touch(localFs, name.getMethodName().toLowerCase(), data);
+
+    FileStatus lstat = localFs.getFileStatus(file);
+    FileStatus sstat = sftpFs.getFileStatus(file);
+    assertNotNull(sstat);
+
+    assertEquals(lstat.getPath().toUri().getPath(),
+                 sstat.getPath().toUri().getPath());
+    assertEquals(data.length, sstat.getLen());
+    assertEquals(lstat.getLen(), sstat.getLen());
+    assertTrue(sftpFs.delete(file, false));
+  }
+
+  /**
+   * Test deleting a non empty directory.
+   *
+   * @throws Exception
+   */
+  @Test(expected=java.io.IOException.class)
+  public void testDeleteNonEmptyDir() throws Exception {
+    Path file = touch(localFs, name.getMethodName().toLowerCase());
+    sftpFs.delete(localDir, false);
+  }
+
+  /**
+   * Test deleting a file that does not exist.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testDeleteNonExistFile() throws Exception {
+    Path file = new Path(localDir, name.getMethodName().toLowerCase());
+    assertFalse(sftpFs.delete(file, false));
+  }
+
+  /**
+   * Test renaming a file.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testRenameFile() throws Exception {
+    byte[] data = "dingos".getBytes();
+    Path file1 = touch(localFs, name.getMethodName().toLowerCase() + "1");
+    Path file2 = new Path(localDir, name.getMethodName().toLowerCase() + "2");
+
+    assertTrue(sftpFs.rename(file1, file2));
+
+    assertTrue(sftpFs.exists(file2));
+    assertFalse(sftpFs.exists(file1));
+
+    assertTrue(localFs.exists(file2));
+    assertFalse(localFs.exists(file1));
+
+    assertTrue(sftpFs.delete(file2, false));
+  }
+
+  /**
+   * Test renaming a file that does not exist.
+   *
+   * @throws Exception
+   */
+  @Test(expected=java.io.IOException.class)
+  public void testRenameNonExistFile() throws Exception {
+    Path file1 = new Path(localDir, name.getMethodName().toLowerCase() + "1");
+    Path file2 = new Path(localDir, name.getMethodName().toLowerCase() + "2");
+    sftpFs.rename(file1, file2);
+  }
+
+  /**
+   * Test renaming a file onto an existing file.
+   *
+   * @throws Exception
+   */
+  @Test(expected=java.io.IOException.class)
+  public void testRenamingFileOntoExistingFile() throws Exception {
+    Path file1 = touch(localFs, name.getMethodName().toLowerCase() + "1");
+    Path file2 = touch(localFs, name.getMethodName().toLowerCase() + "2");
+    sftpFs.rename(file1, file2);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/559425dc/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 16d2058..e010de1 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -666,6 +666,11 @@
         <version>2.0.0-M5</version>
       </dependency>
       <dependency>
+        <groupId>org.apache.sshd</groupId>
+        <artifactId>sshd-core</artifactId>
+        <version>0.14.0</version>
+      </dependency>
+      <dependency>
         <groupId>org.apache.ftpserver</groupId>
         <artifactId>ftplet-api</artifactId>
         <version>1.0.0</version>


[32/50] hadoop git commit: HDFS-8751. Remove setBlocks API from INodeFile and misc code cleanup. Contributed by Zhe Zhang

Posted by ar...@apache.org.
HDFS-8751. Remove setBlocks API from INodeFile and misc code cleanup. Contributed by Zhe Zhang


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

Branch: refs/heads/HDFS-7240
Commit: 47f4c54106ebb234a7d3dc71320aa584ecba161a
Parents: 0824426
Author: Jing Zhao <ji...@apache.org>
Authored: Fri Jul 10 14:15:22 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Fri Jul 10 14:15:22 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../hdfs/server/namenode/FSDirConcatOp.java     |  2 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  | 33 +++++++++++++-------
 .../hdfs/server/namenode/TestINodeFile.java     |  9 ++++++
 4 files changed, 34 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/47f4c541/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 13b2621..4fa566d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -713,6 +713,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8726. Move protobuf files that define the client-sever protocols to
     hdfs-client. (wheat9)
 
+    HDFS-8751. Remove setBlocks API from INodeFile and misc code cleanup. (Zhe
+    Zhang via jing9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47f4c541/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
index 3f22f51..bb00130 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
@@ -228,7 +228,7 @@ class FSDirConcatOp {
     int count = 0;
     for (INodeFile nodeToRemove : srcList) {
       if(nodeToRemove != null) {
-        nodeToRemove.setBlocks(null);
+        nodeToRemove.clearBlocks();
         nodeToRemove.getParent().removeChild(nodeToRemove);
         fsd.getINodeMap().remove(nodeToRemove);
         count++;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47f4c541/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 48879d7..ec0c6c2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -369,7 +369,7 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   /** Set the replication factor of this file. */
-  public final void setFileReplication(short replication) {
+  private void setFileReplication(short replication) {
     header = HeaderFormat.REPLICATION.BITS.combine(replication, header);
   }
 
@@ -413,33 +413,35 @@ public class INodeFile extends INodeWithAdditionalFields
     setStoragePolicyID(storagePolicyId);
   }
 
-  @Override
+  @Override // INodeFileAttributes
   public long getHeaderLong() {
     return header;
   }
 
   /** @return the blocks of the file. */
-  @Override
+  @Override // BlockCollection
   public BlockInfo[] getBlocks() {
     return this.blocks;
   }
 
   /** @return blocks of the file corresponding to the snapshot. */
   public BlockInfo[] getBlocks(int snapshot) {
-    if(snapshot == CURRENT_STATE_ID || getDiffs() == null)
+    if(snapshot == CURRENT_STATE_ID || getDiffs() == null) {
       return getBlocks();
+    }
     FileDiff diff = getDiffs().getDiffById(snapshot);
-    BlockInfo[] snapshotBlocks =
-        diff == null ? getBlocks() : diff.getBlocks();
-    if(snapshotBlocks != null)
+    BlockInfo[] snapshotBlocks = diff == null ? getBlocks() : diff.getBlocks();
+    if (snapshotBlocks != null) {
       return snapshotBlocks;
+    }
     // Blocks are not in the current snapshot
     // Find next snapshot with blocks present or return current file blocks
     snapshotBlocks = getDiffs().findLaterSnapshotBlocks(snapshot);
     return (snapshotBlocks == null) ? getBlocks() : snapshotBlocks;
   }
 
-  void updateBlockCollection() {
+  /** Used during concat to update the BlockCollection for each block. */
+  private void updateBlockCollection() {
     if (blocks != null) {
       for(BlockInfo b : blocks) {
         b.setBlockCollection(this);
@@ -486,10 +488,15 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   /** Set the blocks. */
-  public void setBlocks(BlockInfo[] blocks) {
+  private void setBlocks(BlockInfo[] blocks) {
     this.blocks = blocks;
   }
 
+  /** Clear all blocks of the file. */
+  public void clearBlocks() {
+    setBlocks(null);
+  }
+
   @Override
   public void cleanSubtree(ReclaimContext reclaimContext,
       final int snapshot, int priorSnapshotId) {
@@ -543,7 +550,7 @@ public class INodeFile extends INodeWithAdditionalFields
         blk.setBlockCollection(null);
       }
     }
-    setBlocks(null);
+    clearBlocks();
     if (getAclFeature() != null) {
       AclStorage.removeAclFeature(getAclFeature());
     }
@@ -783,16 +790,18 @@ public class INodeFile extends INodeWithAdditionalFields
   public long collectBlocksBeyondMax(final long max,
       final BlocksMapUpdateInfo collectedBlocks) {
     final BlockInfo[] oldBlocks = getBlocks();
-    if (oldBlocks == null)
+    if (oldBlocks == null) {
       return 0;
+    }
     // find the minimum n such that the size of the first n blocks > max
     int n = 0;
     long size = 0;
     for(; n < oldBlocks.length && max > size; n++) {
       size += oldBlocks[n].getNumBytes();
     }
-    if (n >= oldBlocks.length)
+    if (n >= oldBlocks.length) {
       return size;
+    }
 
     // starting from block n, the data is beyond max.
     // resize the array.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47f4c541/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
index ec4de1b..b8db998 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.fail;
 
 import java.io.FileNotFoundException;
@@ -1142,4 +1143,12 @@ public class TestINodeFile {
     f1 = inf.getXAttrFeature();
     assertEquals(f1, null);
   }
+
+  @Test
+  public void testClearBlocks() {
+    INodeFile toBeCleared = createINodeFiles(1, "toBeCleared")[0];
+    assertEquals(1, toBeCleared.getBlocks().length);
+    toBeCleared.clearBlocks();
+    assertNull(toBeCleared.getBlocks());
+  }
 }


[48/50] hadoop git commit: HDFS-8722. Optimize datanode writes for small writes and flushes. Contributed by Kihwal Lee

Posted by ar...@apache.org.
HDFS-8722. Optimize datanode writes for small writes and flushes. 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/59388a80
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/59388a80
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/59388a80

Branch: refs/heads/HDFS-7240
Commit: 59388a801514d6af64ef27fbf246d8054f1dcc74
Parents: b7fb6ec
Author: Kihwal Lee <ki...@apache.org>
Authored: Tue Jul 14 14:04:06 2015 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Tue Jul 14 14:04:06 2015 -0500

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  2 ++
 .../hdfs/server/datanode/BlockReceiver.java     | 34 +++++++++++++-------
 2 files changed, 24 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/59388a80/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 86b1ea1..14f3403 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1053,6 +1053,8 @@ Release 2.7.2 - UNRELEASED
 
   OPTIMIZATIONS
 
+  HDFS-8722. Optimize datanode writes for small writes and flushes (kihwal)
+
   BUG FIXES
 
 Release 2.7.1 - 2015-07-06 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/59388a80/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 2468f43..55c9d57 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
@@ -598,14 +598,19 @@ class BlockReceiver implements Closeable {
           // bytes should be skipped when writing the data and checksum
           // buffers out to disk.
           long partialChunkSizeOnDisk = onDiskLen % bytesPerChecksum;
+          long lastChunkBoundary = onDiskLen - partialChunkSizeOnDisk;
           boolean alignedOnDisk = partialChunkSizeOnDisk == 0;
           boolean alignedInPacket = firstByteInBlock % bytesPerChecksum == 0;
 
-          // Since data is always appended, not overwritten, partial CRC
-          // recalculation is necessary if the on-disk data is not chunk-
-          // aligned, regardless of whether the beginning of the data in
-          // the packet is chunk-aligned.
-          boolean doPartialCrc = !alignedOnDisk && !shouldNotWriteChecksum;
+          // If the end of the on-disk data is not chunk-aligned, the last
+          // checksum needs to be overwritten.
+          boolean overwriteLastCrc = !alignedOnDisk && !shouldNotWriteChecksum;
+          // If the starting offset of the packat data is at the last chunk
+          // boundary of the data on disk, the partial checksum recalculation
+          // can be skipped and the checksum supplied by the client can be used
+          // instead. This reduces disk reads and cpu load.
+          boolean doCrcRecalc = overwriteLastCrc &&
+              (lastChunkBoundary != firstByteInBlock);
 
           // If this is a partial chunk, then verify that this is the only
           // chunk in the packet. If the starting offset is not chunk
@@ -621,9 +626,10 @@ class BlockReceiver implements Closeable {
           // If the last portion of the block file is not a full chunk,
           // then read in pre-existing partial data chunk and recalculate
           // the checksum so that the checksum calculation can continue
-          // from the right state.
+          // from the right state. If the client provided the checksum for
+          // the whole chunk, this is not necessary.
           Checksum partialCrc = null;
-          if (doPartialCrc) {
+          if (doCrcRecalc) {
             if (LOG.isDebugEnabled()) {
               LOG.debug("receivePacket for " + block 
                   + ": previous write did not end at the chunk boundary."
@@ -659,8 +665,15 @@ class BlockReceiver implements Closeable {
             int skip = 0;
             byte[] crcBytes = null;
 
-            // First, overwrite the partial crc at the end, if necessary.
-            if (doPartialCrc) { // not chunk-aligned on disk
+            // First, prepare to overwrite the partial crc at the end.
+            if (overwriteLastCrc) { // not chunk-aligned on disk
+              // prepare to overwrite last checksum
+              adjustCrcFilePosition();
+            }
+
+            // The CRC was recalculated for the last partial chunk. Update the
+            // CRC by reading the rest of the chunk, then write it out.
+            if (doCrcRecalc) {
               // Calculate new crc for this chunk.
               int bytesToReadForRecalc =
                   (int)(bytesPerChecksum - partialChunkSizeOnDisk);
@@ -673,8 +686,6 @@ class BlockReceiver implements Closeable {
               byte[] buf = FSOutputSummer.convertToByteStream(partialCrc,
                   checksumSize);
               crcBytes = copyLastChunkChecksum(buf, checksumSize, buf.length);
-              // prepare to overwrite last checksum
-              adjustCrcFilePosition();
               checksumOut.write(buf);
               if(LOG.isDebugEnabled()) {
                 LOG.debug("Writing out partial crc for data len " + len +
@@ -687,7 +698,6 @@ class BlockReceiver implements Closeable {
             // boundary. The checksum after the boundary was already counted
             // above. Only count the number of checksums skipped up to the
             // boundary here.
-            long lastChunkBoundary = onDiskLen - (onDiskLen%bytesPerChecksum);
             long skippedDataBytes = lastChunkBoundary - firstByteInBlock;
 
             if (skippedDataBytes > 0) {


[19/50] hadoop git commit: YARN-2194. Addendum patch to fix failing unit test in TestPrivilegedOperationExecutor. Contributed by Sidharta Seethana.

Posted by ar...@apache.org.
YARN-2194. Addendum patch to fix failing unit test in TestPrivilegedOperationExecutor. Contributed by Sidharta Seethana.


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

Branch: refs/heads/HDFS-7240
Commit: 63d0365088ff9fca0baaf3c4c3c01f80c72d3281
Parents: 6b7692c
Author: Varun Vasudev <vv...@apache.org>
Authored: Thu Jul 9 11:49:28 2015 +0530
Committer: Varun Vasudev <vv...@apache.org>
Committed: Thu Jul 9 11:51:59 2015 +0530

----------------------------------------------------------------------
 .../linux/privileged/TestPrivilegedOperationExecutor.java      | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d03650/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/TestPrivilegedOperationExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/TestPrivilegedOperationExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/TestPrivilegedOperationExecutor.java
index 7154d03..8f297ed 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/TestPrivilegedOperationExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/TestPrivilegedOperationExecutor.java
@@ -217,8 +217,10 @@ public class TestPrivilegedOperationExecutor {
           .squashCGroupOperations(ops);
       String expected = new StringBuffer
           (PrivilegedOperation.CGROUP_ARG_PREFIX)
-          .append(cGroupTasks1).append(',')
-          .append(cGroupTasks2).append(',')
+          .append(cGroupTasks1).append(PrivilegedOperation
+              .LINUX_FILE_PATH_SEPARATOR)
+          .append(cGroupTasks2).append(PrivilegedOperation
+              .LINUX_FILE_PATH_SEPARATOR)
           .append(cGroupTasks3).toString();
 
       //We expect axactly one argument


[37/50] hadoop git commit: YARN-3894. RM startup should fail for wrong CS xml NodeLabel capacity configuration. (Bibin A Chundatt via wangda)

Posted by ar...@apache.org.
YARN-3894. RM startup should fail for wrong CS xml NodeLabel capacity configuration. (Bibin A Chundatt via wangda)


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

Branch: refs/heads/HDFS-7240
Commit: 5ed1fead6b5ec24bb0ce1a3db033c2ee1ede4bb4
Parents: d667560
Author: Wangda Tan <wa...@apache.org>
Authored: Sun Jul 12 21:51:34 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Sun Jul 12 21:52:11 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../scheduler/capacity/ParentQueue.java         |  2 +-
 .../scheduler/capacity/TestQueueParsing.java    | 38 ++++++++++++++++++++
 3 files changed, 42 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ed1fead/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 0415ae4..f23effa 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -622,6 +622,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3917. getResourceCalculatorPlugin for the default should intercept all
     exceptions. (gera)
 
+    YARN-3894. RM startup should fail for wrong CS xml NodeLabel capacity
+    configuration. (Bibin A Chundatt via wangda)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ed1fead/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
index 97a5d27..5807dd1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
@@ -147,7 +147,7 @@ public class ParentQueue extends AbstractCSQueue {
       		" for children of queue " + queueName);
     }
     // check label capacities
-    for (String nodeLabel : labelManager.getClusterNodeLabelNames()) {
+    for (String nodeLabel : queueCapacities.getExistingNodeLabels()) {
       float capacityByLabel = queueCapacities.getCapacity(nodeLabel);
       // check children's labels
       float sum = 0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ed1fead/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.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/TestQueueParsing.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/TestQueueParsing.java
index 198bd4a..92baa85 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/TestQueueParsing.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/TestQueueParsing.java
@@ -22,12 +22,15 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.service.ServiceOperations;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
 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.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
@@ -898,4 +901,39 @@ public class TestQueueParsing {
     
     capacityScheduler.reinitialize(csConf, rmContext);
   }
+
+  @Test(timeout = 60000, expected = java.lang.IllegalArgumentException.class)
+  public void testRMStartWrongNodeCapacity() throws Exception {
+    YarnConfiguration config = new YarnConfiguration();
+    nodeLabelManager = new NullRMNodeLabelsManager();
+    nodeLabelManager.init(config);
+    config.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    CapacitySchedulerConfiguration conf =
+        new CapacitySchedulerConfiguration(config);
+
+    // Define top-level queues
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a" });
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100);
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "y", 100);
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "z", 100);
+
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    conf.setCapacity(A, 100);
+    conf.setAccessibleNodeLabels(A, ImmutableSet.of("x", "y", "z"));
+    conf.setCapacityByLabel(A, "x", 100);
+    conf.setCapacityByLabel(A, "y", 100);
+    conf.setCapacityByLabel(A, "z", 70);
+    MockRM rm = null;
+    try {
+      rm = new MockRM(conf) {
+        @Override
+        public RMNodeLabelsManager createNodeLabelManager() {
+          return nodeLabelManager;
+        }
+      };
+    } finally {
+      IOUtils.closeStream(rm);
+    }
+  }
 }


[40/50] hadoop git commit: Revert "YARN-3878. AsyncDispatcher can hang while stopping if it is configured for draining events on stop. (Varun Saxena via kasha)"

Posted by ar...@apache.org.
Revert "YARN-3878. AsyncDispatcher can hang while stopping if it is configured for draining events on stop. (Varun Saxena via kasha)"

This reverts commit aa067c6aa47b4c79577096817acc00ad6421180c.


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

Branch: refs/heads/HDFS-7240
Commit: 2466460d4cd13ad5837c044476b26e63082c1d37
Parents: 19295b3
Author: Jian He <ji...@apache.org>
Authored: Mon Jul 13 14:30:35 2015 -0700
Committer: Jian He <ji...@apache.org>
Committed: Mon Jul 13 14:30:35 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 -
 .../hadoop/yarn/event/AsyncDispatcher.java      | 24 ++++----
 .../hadoop/yarn/event/DrainDispatcher.java      | 13 +----
 .../hadoop/yarn/event/TestAsyncDispatcher.java  | 61 --------------------
 4 files changed, 14 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2466460d/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 98f0e8d..5c17f04 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -648,9 +648,6 @@ Release 2.7.2 - UNRELEASED
 
     YARN-3690. [JDK8] 'mvn site' fails. (Brahma Reddy Battula via aajisaka)
 
-    YARN-3878. AsyncDispatcher can hang while stopping if it is configured for 
-    draining events on stop. (Varun Saxena via kasha)
-
 Release 2.7.1 - 2015-07-06 
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2466460d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
index 646611f..c54b9c7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
@@ -55,6 +55,9 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
   // stop functionality.
   private volatile boolean drainEventsOnStop = false;
 
+  // Indicates all the remaining dispatcher's events on stop have been drained
+  // and processed.
+  private volatile boolean drained = true;
   private Object waitForDrained = new Object();
 
   // For drainEventsOnStop enabled only, block newly coming events into the
@@ -81,12 +84,13 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
       @Override
       public void run() {
         while (!stopped && !Thread.currentThread().isInterrupted()) {
+          drained = eventQueue.isEmpty();
           // blockNewEvents is only set when dispatcher is draining to stop,
           // adding this check is to avoid the overhead of acquiring the lock
           // and calling notify every time in the normal run of the loop.
           if (blockNewEvents) {
             synchronized (waitForDrained) {
-              if (eventQueue.isEmpty()) {
+              if (drained) {
                 waitForDrained.notify();
               }
             }
@@ -135,7 +139,7 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
       blockNewEvents = true;
       LOG.info("AsyncDispatcher is draining to stop, igonring any new events.");
       synchronized (waitForDrained) {
-        while (!eventQueue.isEmpty() && eventHandlingThread.isAlive()) {
+        while (!drained && eventHandlingThread.isAlive()) {
           waitForDrained.wait(1000);
           LOG.info("Waiting for AsyncDispatcher to drain. Thread state is :" +
               eventHandlingThread.getState());
@@ -219,21 +223,12 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
     return handlerInstance;
   }
 
-  @VisibleForTesting
-  protected boolean hasPendingEvents() {
-    return !eventQueue.isEmpty();
-  }
-
-  @VisibleForTesting
-  protected boolean isEventThreadWaiting() {
-    return eventHandlingThread.getState() == Thread.State.WAITING;
-  }
-
   class GenericEventHandler implements EventHandler<Event> {
     public void handle(Event event) {
       if (blockNewEvents) {
         return;
       }
+      drained = false;
 
       /* all this method does is enqueue all the events onto the queue */
       int qSize = eventQueue.size();
@@ -290,4 +285,9 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
       }
     };
   }
+
+  @VisibleForTesting
+  protected boolean isDrained() {
+    return this.drained;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2466460d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
index d1f4fe9..da5ae44 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
@@ -27,24 +27,15 @@ public class DrainDispatcher extends AsyncDispatcher {
     this(new LinkedBlockingQueue<Event>());
   }
 
-  public DrainDispatcher(BlockingQueue<Event> eventQueue) {
+  private DrainDispatcher(BlockingQueue<Event> eventQueue) {
     super(eventQueue);
   }
 
   /**
-   *  Wait till event thread enters WAITING state (i.e. waiting for new events).
-   */
-  public void waitForEventThreadToWait() {
-    while (!isEventThreadWaiting()) {
-      Thread.yield();
-    }
-  }
-
-  /**
    * Busy loop waiting for all queued events to drain.
    */
   public void await() {
-    while (hasPendingEvents()) {
+    while (!isDrained()) {
       Thread.yield();
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2466460d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/TestAsyncDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/TestAsyncDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/TestAsyncDispatcher.java
deleted file mode 100644
index ee17ddd..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/TestAsyncDispatcher.java
+++ /dev/null
@@ -1,61 +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.event;
-
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class TestAsyncDispatcher {
-
-  /* This test checks whether dispatcher hangs on close if following two things
-   * happen :
-   * 1. A thread which was putting event to event queue is interrupted.
-   * 2. Event queue is empty on close.
-   */
-  @SuppressWarnings({ "unchecked", "rawtypes" })
-  @Test(timeout=10000)
-  public void testDispatcherOnCloseIfQueueEmpty() throws Exception {
-    BlockingQueue<Event> eventQueue = spy(new LinkedBlockingQueue<Event>());
-    Event event = mock(Event.class);
-    doThrow(new InterruptedException()).when(eventQueue).put(event);
-    DrainDispatcher disp = new DrainDispatcher(eventQueue);
-    disp.init(new Configuration());
-    disp.setDrainEventsOnStop();
-    disp.start();
-    // Wait for event handler thread to start and begin waiting for events.
-    disp.waitForEventThreadToWait();
-    try {
-      disp.getEventHandler().handle(event);
-    } catch (YarnRuntimeException e) {
-    }
-    // Queue should be empty and dispatcher should not hang on close
-    Assert.assertTrue("Event Queue should have been empty",
-        eventQueue.isEmpty());
-    disp.close();
-  }
-}


[22/50] hadoop git commit: YARN-3878. AsyncDispatcher can hang while stopping if it is configured for draining events on stop. (Varun Saxena via kasha)

Posted by ar...@apache.org.
YARN-3878. AsyncDispatcher can hang while stopping if it is configured for draining events on stop. (Varun Saxena via kasha)


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

Branch: refs/heads/HDFS-7240
Commit: aa067c6aa47b4c79577096817acc00ad6421180c
Parents: 527c40e
Author: Karthik Kambatla <ka...@apache.org>
Authored: Thu Jul 9 09:48:29 2015 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Thu Jul 9 09:48:29 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../hadoop/yarn/event/AsyncDispatcher.java      | 24 ++++----
 .../hadoop/yarn/event/DrainDispatcher.java      | 13 ++++-
 .../hadoop/yarn/event/TestAsyncDispatcher.java  | 61 ++++++++++++++++++++
 4 files changed, 87 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa067c6a/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 19f0854..3c232eb 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -624,6 +624,9 @@ Release 2.7.2 - UNRELEASED
 
     YARN-3690. [JDK8] 'mvn site' fails. (Brahma Reddy Battula via aajisaka)
 
+    YARN-3878. AsyncDispatcher can hang while stopping if it is configured for 
+    draining events on stop. (Varun Saxena via kasha)
+
 Release 2.7.1 - 2015-07-06 
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa067c6a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
index c54b9c7..646611f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
@@ -55,9 +55,6 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
   // stop functionality.
   private volatile boolean drainEventsOnStop = false;
 
-  // Indicates all the remaining dispatcher's events on stop have been drained
-  // and processed.
-  private volatile boolean drained = true;
   private Object waitForDrained = new Object();
 
   // For drainEventsOnStop enabled only, block newly coming events into the
@@ -84,13 +81,12 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
       @Override
       public void run() {
         while (!stopped && !Thread.currentThread().isInterrupted()) {
-          drained = eventQueue.isEmpty();
           // blockNewEvents is only set when dispatcher is draining to stop,
           // adding this check is to avoid the overhead of acquiring the lock
           // and calling notify every time in the normal run of the loop.
           if (blockNewEvents) {
             synchronized (waitForDrained) {
-              if (drained) {
+              if (eventQueue.isEmpty()) {
                 waitForDrained.notify();
               }
             }
@@ -139,7 +135,7 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
       blockNewEvents = true;
       LOG.info("AsyncDispatcher is draining to stop, igonring any new events.");
       synchronized (waitForDrained) {
-        while (!drained && eventHandlingThread.isAlive()) {
+        while (!eventQueue.isEmpty() && eventHandlingThread.isAlive()) {
           waitForDrained.wait(1000);
           LOG.info("Waiting for AsyncDispatcher to drain. Thread state is :" +
               eventHandlingThread.getState());
@@ -223,12 +219,21 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
     return handlerInstance;
   }
 
+  @VisibleForTesting
+  protected boolean hasPendingEvents() {
+    return !eventQueue.isEmpty();
+  }
+
+  @VisibleForTesting
+  protected boolean isEventThreadWaiting() {
+    return eventHandlingThread.getState() == Thread.State.WAITING;
+  }
+
   class GenericEventHandler implements EventHandler<Event> {
     public void handle(Event event) {
       if (blockNewEvents) {
         return;
       }
-      drained = false;
 
       /* all this method does is enqueue all the events onto the queue */
       int qSize = eventQueue.size();
@@ -285,9 +290,4 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
       }
     };
   }
-
-  @VisibleForTesting
-  protected boolean isDrained() {
-    return this.drained;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa067c6a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
index da5ae44..d1f4fe9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
@@ -27,15 +27,24 @@ public class DrainDispatcher extends AsyncDispatcher {
     this(new LinkedBlockingQueue<Event>());
   }
 
-  private DrainDispatcher(BlockingQueue<Event> eventQueue) {
+  public DrainDispatcher(BlockingQueue<Event> eventQueue) {
     super(eventQueue);
   }
 
   /**
+   *  Wait till event thread enters WAITING state (i.e. waiting for new events).
+   */
+  public void waitForEventThreadToWait() {
+    while (!isEventThreadWaiting()) {
+      Thread.yield();
+    }
+  }
+
+  /**
    * Busy loop waiting for all queued events to drain.
    */
   public void await() {
-    while (!isDrained()) {
+    while (hasPendingEvents()) {
       Thread.yield();
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa067c6a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/TestAsyncDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/TestAsyncDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/TestAsyncDispatcher.java
new file mode 100644
index 0000000..ee17ddd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/TestAsyncDispatcher.java
@@ -0,0 +1,61 @@
+/**
+ * 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.event;
+
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestAsyncDispatcher {
+
+  /* This test checks whether dispatcher hangs on close if following two things
+   * happen :
+   * 1. A thread which was putting event to event queue is interrupted.
+   * 2. Event queue is empty on close.
+   */
+  @SuppressWarnings({ "unchecked", "rawtypes" })
+  @Test(timeout=10000)
+  public void testDispatcherOnCloseIfQueueEmpty() throws Exception {
+    BlockingQueue<Event> eventQueue = spy(new LinkedBlockingQueue<Event>());
+    Event event = mock(Event.class);
+    doThrow(new InterruptedException()).when(eventQueue).put(event);
+    DrainDispatcher disp = new DrainDispatcher(eventQueue);
+    disp.init(new Configuration());
+    disp.setDrainEventsOnStop();
+    disp.start();
+    // Wait for event handler thread to start and begin waiting for events.
+    disp.waitForEventThreadToWait();
+    try {
+      disp.getEventHandler().handle(event);
+    } catch (YarnRuntimeException e) {
+    }
+    // Queue should be empty and dispatcher should not hang on close
+    Assert.assertTrue("Event Queue should have been empty",
+        eventQueue.isEmpty());
+    disp.close();
+  }
+}


[27/50] hadoop git commit: HADOOP-12210. Collect network usage on the node. Contributed by Robert Grandl

Posted by ar...@apache.org.
HADOOP-12210. Collect network usage on the node. Contributed by Robert Grandl


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

Branch: refs/heads/HDFS-7240
Commit: 1a0752d85a15499d120b4a79af9bd740fcd1f8e0
Parents: 0e602fa
Author: Chris Douglas <cd...@apache.org>
Authored: Mon Jul 6 17:28:20 2015 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Thu Jul 9 17:48:43 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  2 +
 .../java/org/apache/hadoop/util/SysInfo.java    | 12 +++
 .../org/apache/hadoop/util/SysInfoLinux.java    | 93 +++++++++++++++++++-
 .../org/apache/hadoop/util/SysInfoWindows.java  | 15 ++++
 .../apache/hadoop/util/TestSysInfoLinux.java    | 40 ++++++++-
 .../gridmix/DummyResourceCalculatorPlugin.java  | 19 ++++
 .../yarn/util/ResourceCalculatorPlugin.java     | 16 ++++
 7 files changed, 195 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a0752d8/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 d9a9eba..3d4f1e4 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -693,6 +693,8 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12180. Move ResourceCalculatorPlugin from YARN to Common. 
     (Chris Douglas via kasha)
 
+    HADOOP-12210. Collect network usage on the node (Robert Grandl via cdouglas)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a0752d8/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfo.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfo.java
index ec7fb24..24b339d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfo.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfo.java
@@ -108,4 +108,16 @@ public abstract class SysInfo {
    */
   public abstract float getCpuUsage();
 
+  /**
+   * Obtain the aggregated number of bytes read over the network.
+   * @return total number of bytes read.
+   */
+  public abstract long getNetworkBytesRead();
+
+  /**
+   * Obtain the aggregated number of bytes written to the network.
+   * @return total number of bytes written.
+   */
+  public abstract long getNetworkBytesWritten();
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a0752d8/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoLinux.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoLinux.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoLinux.java
index 055298d..8801985 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoLinux.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoLinux.java
@@ -83,9 +83,22 @@ public class SysInfoLinux extends SysInfo {
                       "[ \t]*([0-9]*)[ \t]*([0-9]*)[ \t].*");
   private CpuTimeTracker cpuTimeTracker;
 
+  /**
+   * Pattern for parsing /proc/net/dev.
+   */
+  private static final String PROCFS_NETFILE = "/proc/net/dev";
+  private static final Pattern PROCFS_NETFILE_FORMAT =
+      Pattern .compile("^[ \t]*([a-zA-Z]+[0-9]*):" +
+               "[ \t]*([0-9]+)[ \t]*([0-9]+)[ \t]*([0-9]+)[ \t]*([0-9]+)" +
+               "[ \t]*([0-9]+)[ \t]*([0-9]+)[ \t]*([0-9]+)[ \t]*([0-9]+)" +
+               "[ \t]*([0-9]+)[ \t]*([0-9]+)[ \t]*([0-9]+)[ \t]*([0-9]+)" +
+               "[ \t]*([0-9]+)[ \t]*([0-9]+)[ \t]*([0-9]+)[ \t]*([0-9]+).*");
+
+
   private String procfsMemFile;
   private String procfsCpuFile;
   private String procfsStatFile;
+  private String procfsNetFile;
   private long jiffyLengthInMillis;
 
   private long ramSize = 0;
@@ -98,6 +111,8 @@ public class SysInfoLinux extends SysInfo {
   /* number of physical cores on the system. */
   private int numCores = 0;
   private long cpuFrequency = 0L; // CPU frequency on the system (kHz)
+  private long numNetBytesRead = 0L; // aggregated bytes read from network
+  private long numNetBytesWritten = 0L; // aggregated bytes written to network
 
   private boolean readMemInfoFile = false;
   private boolean readCpuInfoFile = false;
@@ -130,7 +145,7 @@ public class SysInfoLinux extends SysInfo {
 
   public SysInfoLinux() {
     this(PROCFS_MEMFILE, PROCFS_CPUINFO, PROCFS_STAT,
-        JIFFY_LENGTH_IN_MILLIS);
+         PROCFS_NETFILE, JIFFY_LENGTH_IN_MILLIS);
   }
 
   /**
@@ -139,16 +154,19 @@ public class SysInfoLinux extends SysInfo {
    * @param procfsMemFile fake file for /proc/meminfo
    * @param procfsCpuFile fake file for /proc/cpuinfo
    * @param procfsStatFile fake file for /proc/stat
+   * @param procfsNetFile fake file for /proc/net/dev
    * @param jiffyLengthInMillis fake jiffy length value
    */
   @VisibleForTesting
   public SysInfoLinux(String procfsMemFile,
                                        String procfsCpuFile,
                                        String procfsStatFile,
+                                       String procfsNetFile,
                                        long jiffyLengthInMillis) {
     this.procfsMemFile = procfsMemFile;
     this.procfsCpuFile = procfsCpuFile;
     this.procfsStatFile = procfsStatFile;
+    this.procfsNetFile = procfsNetFile;
     this.jiffyLengthInMillis = jiffyLengthInMillis;
     this.cpuTimeTracker = new CpuTimeTracker(jiffyLengthInMillis);
   }
@@ -338,6 +356,61 @@ public class SysInfoLinux extends SysInfo {
     }
   }
 
+  /**
+   * Read /proc/net/dev file, parse and calculate amount
+   * of bytes read and written through the network.
+   */
+  private void readProcNetInfoFile() {
+
+    numNetBytesRead = 0L;
+    numNetBytesWritten = 0L;
+
+    // Read "/proc/net/dev" file
+    BufferedReader in;
+    InputStreamReader fReader;
+    try {
+      fReader = new InputStreamReader(
+          new FileInputStream(procfsNetFile), Charset.forName("UTF-8"));
+      in = new BufferedReader(fReader);
+    } catch (FileNotFoundException f) {
+      return;
+    }
+
+    Matcher mat;
+    try {
+      String str = in.readLine();
+      while (str != null) {
+        mat = PROCFS_NETFILE_FORMAT.matcher(str);
+        if (mat.find()) {
+          assert mat.groupCount() >= 16;
+
+          // ignore loopback interfaces
+          if (mat.group(1).equals("lo")) {
+            str = in.readLine();
+            continue;
+          }
+          numNetBytesRead += Long.parseLong(mat.group(2));
+          numNetBytesWritten += Long.parseLong(mat.group(10));
+        }
+        str = in.readLine();
+      }
+    } catch (IOException io) {
+      LOG.warn("Error reading the stream " + io);
+    } finally {
+      // Close the streams
+      try {
+        fReader.close();
+        try {
+          in.close();
+        } catch (IOException i) {
+          LOG.warn("Error closing the stream " + in);
+        }
+      } catch (IOException i) {
+        LOG.warn("Error closing the stream " + fReader);
+      }
+    }
+  }
+
   /** {@inheritDoc} */
   @Override
   public long getPhysicalMemorySize() {
@@ -405,6 +478,20 @@ public class SysInfoLinux extends SysInfo {
     return overallCpuUsage;
   }
 
+  /** {@inheritDoc} */
+  @Override
+  public long getNetworkBytesRead() {
+    readProcNetInfoFile();
+    return numNetBytesRead;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getNetworkBytesWritten() {
+    readProcNetInfoFile();
+    return numNetBytesWritten;
+  }
+
   /**
    * Test the {@link SysInfoLinux}.
    *
@@ -424,6 +511,10 @@ public class SysInfoLinux extends SysInfo {
     System.out.println("CPU frequency (kHz) : " + plugin.getCpuFrequency());
     System.out.println("Cumulative CPU time (ms) : " +
             plugin.getCumulativeCpuTime());
+    System.out.println("Total network read (bytes) : "
+            + plugin.getNetworkBytesRead());
+    System.out.println("Total network written (bytes) : "
+            + plugin.getNetworkBytesWritten());
     try {
       // Sleep so we can compute the CPU usage
       Thread.sleep(500L);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a0752d8/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java
index da4c1c5..f8542a3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java
@@ -178,4 +178,19 @@ public class SysInfoWindows extends SysInfo {
     refreshIfNeeded();
     return cpuUsage;
   }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getNetworkBytesRead() {
+    // TODO unimplemented
+    return 0L;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getNetworkBytesWritten() {
+    // TODO unimplemented
+    return 0L;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a0752d8/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoLinux.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoLinux.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoLinux.java
index 73edc77..2a31f31 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoLinux.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoLinux.java
@@ -44,8 +44,10 @@ public class TestSysInfoLinux {
     public FakeLinuxResourceCalculatorPlugin(String procfsMemFile,
                                              String procfsCpuFile,
                                              String procfsStatFile,
+			                                       String procfsNetFile,
                                              long jiffyLengthInMillis) {
-      super(procfsMemFile, procfsCpuFile, procfsStatFile, jiffyLengthInMillis);
+      super(procfsMemFile, procfsCpuFile, procfsStatFile, procfsNetFile,
+          jiffyLengthInMillis);
     }
     @Override
     long getCurrentTime() {
@@ -61,14 +63,17 @@ public class TestSysInfoLinux {
   private static final String FAKE_MEMFILE;
   private static final String FAKE_CPUFILE;
   private static final String FAKE_STATFILE;
+  private static final String FAKE_NETFILE;
   private static final long FAKE_JIFFY_LENGTH = 10L;
   static {
     int randomNum = (new Random()).nextInt(1000000000);
     FAKE_MEMFILE = TEST_ROOT_DIR + File.separator + "MEMINFO_" + randomNum;
     FAKE_CPUFILE = TEST_ROOT_DIR + File.separator + "CPUINFO_" + randomNum;
     FAKE_STATFILE = TEST_ROOT_DIR + File.separator + "STATINFO_" + randomNum;
+    FAKE_NETFILE = TEST_ROOT_DIR + File.separator + "NETINFO_" + randomNum;
     plugin = new FakeLinuxResourceCalculatorPlugin(FAKE_MEMFILE, FAKE_CPUFILE,
                                                    FAKE_STATFILE,
+                                                   FAKE_NETFILE,
                                                    FAKE_JIFFY_LENGTH);
   }
   static final String MEMINFO_FORMAT =
@@ -141,6 +146,17 @@ public class TestSysInfoLinux {
     "procs_running 1\n" +
     "procs_blocked 0\n";
 
+  static final String NETINFO_FORMAT =
+    "Inter-|   Receive                                                |  Transmit\n"+
+    "face |bytes    packets errs drop fifo frame compressed multicast|bytes    packets"+
+    "errs drop fifo colls carrier compressed\n"+
+    "   lo: 42236310  563003    0    0    0     0          0         0 42236310  563003    " +
+    "0    0    0     0       0          0\n"+
+    " eth0: %d 3452527    0    0    0     0          0    299787 %d 1866280    0    0    " +
+    "0     0       0          0\n"+
+    " eth1: %d 3152521    0    0    0     0          0    219781 %d 1866290    0    0    " +
+    "0     0       0          0\n";
+
   /**
    * Test parsing /proc/stat and /proc/cpuinfo
    * @throws IOException
@@ -320,4 +336,26 @@ public class TestSysInfoLinux {
       IOUtils.closeQuietly(fWriter);
     }
   }
+
+  /**
+   * Test parsing /proc/net/dev
+   * @throws IOException
+   */
+  @Test
+  public void parsingProcNetFile() throws IOException {
+    long numBytesReadIntf1 = 2097172468L;
+    long numBytesWrittenIntf1 = 1355620114L;
+    long numBytesReadIntf2 = 1097172460L;
+    long numBytesWrittenIntf2 = 1055620110L;
+    File tempFile = new File(FAKE_NETFILE);
+    tempFile.deleteOnExit();
+    FileWriter fWriter = new FileWriter(FAKE_NETFILE);
+    fWriter.write(String.format(NETINFO_FORMAT,
+                            numBytesReadIntf1, numBytesWrittenIntf1,
+                            numBytesReadIntf2, numBytesWrittenIntf2));
+    fWriter.close();
+    assertEquals(plugin.getNetworkBytesRead(), numBytesReadIntf1 + numBytesReadIntf2);
+    assertEquals(plugin.getNetworkBytesWritten(), numBytesWrittenIntf1 + numBytesWrittenIntf2);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a0752d8/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/DummyResourceCalculatorPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/DummyResourceCalculatorPlugin.java b/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/DummyResourceCalculatorPlugin.java
index fd4cb83..b86303b 100644
--- a/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/DummyResourceCalculatorPlugin.java
+++ b/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/DummyResourceCalculatorPlugin.java
@@ -48,6 +48,12 @@ public class DummyResourceCalculatorPlugin extends ResourceCalculatorPlugin {
       "mapred.tasktracker.cumulativecputime.testing";
   /** CPU usage percentage for testing */
   public static final String CPU_USAGE = "mapred.tasktracker.cpuusage.testing";
+  /** cumulative number of bytes read over the network */
+  public static final String NETWORK_BYTES_READ =
+      "mapred.tasktracker.networkread.testing";
+  /** cumulative number of bytes written over the network */
+  public static final String NETWORK_BYTES_WRITTEN =
+      "mapred.tasktracker.networkwritten.testing";
   /** process cumulative CPU usage time for testing */
   public static final String PROC_CUMULATIVE_CPU_TIME =
       "mapred.tasktracker.proccumulativecputime.testing";
@@ -111,4 +117,17 @@ public class DummyResourceCalculatorPlugin extends ResourceCalculatorPlugin {
   public float getCpuUsage() {
     return getConf().getFloat(CPU_USAGE, -1);
   }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getNetworkBytesRead() {
+    return getConf().getLong(NETWORK_BYTES_READ, -1);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getNetworkBytesWritten() {
+    return getConf().getLong(NETWORK_BYTES_WRITTEN, -1);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a0752d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
index 5e5f1b4..21724a9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
@@ -124,6 +124,22 @@ public class ResourceCalculatorPlugin extends Configured {
     return sys.getCpuUsage();
   }
 
+   /**
+   * Obtain the aggregated number of bytes read over the network.
+   * @return total number of bytes read.
+   */
+  public long getNetworkBytesRead() {
+    return sys.getNetworkBytesRead();
+  }
+
+  /**
+   * Obtain the aggregated number of bytes written to the network.
+   * @return total number of bytes written.
+   */
+  public long getNetworkBytesWritten() {
+    return sys.getNetworkBytesWritten();
+  }
+
   /**
    * Create the ResourceCalculatorPlugin from the class name and configure it. If
    * class name is null, this method will try and return a memory calculator


[46/50] hadoop git commit: HDFS-7608: hdfs dfsclient newConnectedPeer has no write timeout (Xiaoyu Yao via Colin P. McCabe)

Posted by ar...@apache.org.
HDFS-7608: hdfs dfsclient newConnectedPeer has no write timeout (Xiaoyu Yao 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/1d74ccec
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1d74ccec
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1d74ccec

Branch: refs/heads/HDFS-7240
Commit: 1d74ccececaefffaa90c0c18b40a3645dbc819d9
Parents: 4084eaf
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Tue Jul 14 10:57:59 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Tue Jul 14 10:57:59 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  1 +
 .../hadoop/hdfs/TestDistributedFileSystem.java  | 43 +++++++++++++++++---
 2 files changed, 38 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d74ccec/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 4923a50..6629a83 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
@@ -3127,6 +3127,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       peer = TcpPeerServer.peerFromSocketAndKey(saslClient, sock, this,
           blockToken, datanodeId);
       peer.setReadTimeout(socketTimeout);
+      peer.setWriteTimeout(socketTimeout);
       success = true;
       return peer;
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d74ccec/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
index f9da472..0b77210 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
@@ -1132,10 +1132,9 @@ public class TestDistributedFileSystem {
       cluster.shutdown();
     }
   }
-  
-  
+
   @Test(timeout=10000)
-  public void testDFSClientPeerTimeout() throws IOException {
+  public void testDFSClientPeerReadTimeout() throws IOException {
     final int timeout = 1000;
     final Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, timeout);
@@ -1152,11 +1151,11 @@ public class TestDistributedFileSystem {
       long start = Time.now();
       try {
         peer.getInputStream().read();
-        Assert.fail("should timeout");
+        Assert.fail("read should timeout");
       } catch (SocketTimeoutException ste) {
         long delta = Time.now() - start;
-        Assert.assertTrue("timedout too soon", delta >= timeout*0.9);
-        Assert.assertTrue("timedout too late", delta <= timeout*1.1);
+        Assert.assertTrue("read timedout too soon", delta >= timeout*0.9);
+        Assert.assertTrue("read timedout too late", delta <= timeout*1.1);
       } catch (Throwable t) {
         Assert.fail("wrong exception:"+t);
       }
@@ -1178,4 +1177,36 @@ public class TestDistributedFileSystem {
       cluster.shutdown();
     }
   }
+
+  @Test(timeout=10000)
+  public void testDFSClientPeerWriteTimeout() throws IOException {
+    final int timeout = 1000;
+    final Configuration conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, timeout);
+
+    // only need cluster to create a dfs client to get a peer
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+    try {
+      cluster.waitActive();
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      // Write 1 MB to a dummy socket to ensure the write times out
+      ServerSocket socket = new ServerSocket(0);
+      Peer peer = dfs.getClient().newConnectedPeer(
+        (InetSocketAddress) socket.getLocalSocketAddress(), null, null);
+      long start = Time.now();
+      try {
+        byte[] buf = new byte[1024 * 1024];
+        peer.getOutputStream().write(buf);
+        Assert.fail("write should timeout");
+      } catch (SocketTimeoutException ste) {
+        long delta = Time.now() - start;
+        Assert.assertTrue("write timedout too soon", delta >= timeout * 0.9);
+        Assert.assertTrue("write timedout too late", delta <= timeout * 1.1);
+      } catch (Throwable t) {
+        Assert.fail("wrong exception:" + t);
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
 }


[31/50] hadoop git commit: YARN-3445. Cache runningApps in RMNode for getting running apps on given NodeId. (Junping Du via mingma)

Posted by ar...@apache.org.
YARN-3445. Cache runningApps in RMNode for getting running apps on given NodeId. (Junping Du via mingma)


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

Branch: refs/heads/HDFS-7240
Commit: 08244264c0583472b9c4e16591cfde72c6db62a2
Parents: b489080
Author: Ming Ma <mi...@apache.org>
Authored: Fri Jul 10 08:30:10 2015 -0700
Committer: Ming Ma <mi...@apache.org>
Committed: Fri Jul 10 08:30:10 2015 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/sls/nodemanager/NodeInfo.java   |  8 +++-
 .../yarn/sls/scheduler/RMNodeWrapper.java       |  5 +++
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../server/resourcemanager/rmnode/RMNode.java   |  2 +
 .../resourcemanager/rmnode/RMNodeImpl.java      | 43 ++++++++++++++++----
 .../yarn/server/resourcemanager/MockNodes.java  |  5 +++
 .../resourcemanager/TestRMNodeTransitions.java  | 36 ++++++++++++++--
 7 files changed, 91 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/08244264/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
index ee6eb7b..440779c 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
@@ -62,7 +62,8 @@ public class NodeInfo {
     private NodeState state;
     private List<ContainerId> toCleanUpContainers;
     private List<ApplicationId> toCleanUpApplications;
-    
+    private List<ApplicationId> runningApplications;
+
     public FakeRMNodeImpl(NodeId nodeId, String nodeAddr, String httpAddress,
         Resource perNode, String rackName, String healthReport,
         int cmdPort, String hostName, NodeState state) {
@@ -77,6 +78,7 @@ public class NodeInfo {
       this.state = state;
       toCleanUpApplications = new ArrayList<ApplicationId>();
       toCleanUpContainers = new ArrayList<ContainerId>();
+      runningApplications = new ArrayList<ApplicationId>();
     }
 
     public NodeId getNodeID() {
@@ -135,6 +137,10 @@ public class NodeInfo {
       return toCleanUpApplications;
     }
 
+    public List<ApplicationId> getRunningApps() {
+      return runningApplications;
+    }
+
     public void updateNodeHeartbeatResponseForCleanup(
             NodeHeartbeatResponse response) {
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08244264/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
index b64be1b..a6633ae 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
@@ -119,6 +119,11 @@ public class RMNodeWrapper implements RMNode {
   }
 
   @Override
+  public List<ApplicationId> getRunningApps() {
+    return node.getRunningApps();
+  }
+
+  @Override
   public void updateNodeHeartbeatResponseForCleanup(
           NodeHeartbeatResponse nodeHeartbeatResponse) {
     node.updateNodeHeartbeatResponseForCleanup(nodeHeartbeatResponse);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08244264/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 2a9ff98..db000d7 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -1678,6 +1678,9 @@ Release 2.6.0 - 2014-11-18
     YARN-2811. In Fair Scheduler, reservation fulfillments shouldn't ignore max
     share (Siqi Li via Sandy Ryza)
 
+    YARN-3445. Cache runningApps in RMNode for getting running apps on given
+    NodeId. (Junping Du via mingma)
+
   IMPROVEMENTS
 
     YARN-2197. Add a link to YARN CHANGES.txt in the left side of doc

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08244264/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.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/rmnode/RMNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
index 95eeaf6..0386be6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
@@ -119,6 +119,8 @@ public interface RMNode {
 
   public List<ApplicationId> getAppsToCleanup();
 
+  List<ApplicationId> getRunningApps();
+
   /**
    * Update a {@link NodeHeartbeatResponse} with the list of containers and
    * applications to clean up for this node.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08244264/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.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/rmnode/RMNodeImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
index d1e6190..9bc91c7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
@@ -123,11 +123,16 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
       new HashSet<ContainerId>();
 
   /* the list of applications that have finished and need to be purged */
-  private final List<ApplicationId> finishedApplications = new ArrayList<ApplicationId>();
+  private final List<ApplicationId> finishedApplications =
+      new ArrayList<ApplicationId>();
+
+  /* the list of applications that are running on this node */
+  private final List<ApplicationId> runningApplications =
+      new ArrayList<ApplicationId>();
 
   private NodeHeartbeatResponse latestNodeHeartBeatResponse = recordFactory
       .newRecordInstance(NodeHeartbeatResponse.class);
-  
+
   private static final StateMachineFactory<RMNodeImpl,
                                            NodeState,
                                            RMNodeEventType,
@@ -136,7 +141,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
                                            NodeState,
                                            RMNodeEventType,
                                            RMNodeEvent>(NodeState.NEW)
-  
+
      //Transitions from NEW state
      .addTransition(NodeState.NEW, NodeState.RUNNING, 
          RMNodeEventType.STARTED, new AddNodeTransition())
@@ -383,6 +388,16 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
   }
   
   @Override
+  public List<ApplicationId> getRunningApps() {
+    this.readLock.lock();
+    try {
+      return new ArrayList<ApplicationId>(this.runningApplications);
+    } finally {
+      this.readLock.unlock();
+    }
+  }
+
+  @Override
   public List<ContainerId> getContainersToCleanUp() {
 
     this.readLock.lock();
@@ -519,9 +534,12 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
       LOG.warn("Cannot get RMApp by appId=" + appId
           + ", just added it to finishedApplications list for cleanup");
       rmNode.finishedApplications.add(appId);
+      rmNode.runningApplications.remove(appId);
       return;
     }
 
+    // Add running applications back due to Node add or Node reconnection.
+    rmNode.runningApplications.add(appId);
     context.getDispatcher().getEventHandler()
         .handle(new RMAppRunningOnNodeEvent(appId, nodeId));
   }
@@ -707,8 +725,9 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
 
     @Override
     public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
-      rmNode.finishedApplications.add(((
-          RMNodeCleanAppEvent) event).getAppId());
+      ApplicationId appId = ((RMNodeCleanAppEvent) event).getAppId();
+      rmNode.finishedApplications.add(appId);
+      rmNode.runningApplications.remove(appId);
     }
   }
 
@@ -910,12 +929,22 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
             + "cleanup, no further processing");
         continue;
       }
-      if (finishedApplications.contains(containerId.getApplicationAttemptId()
-          .getApplicationId())) {
+
+      ApplicationId containerAppId =
+          containerId.getApplicationAttemptId().getApplicationId();
+
+      if (finishedApplications.contains(containerAppId)) {
         LOG.info("Container " + containerId
             + " belongs to an application that is already killed,"
             + " no further processing");
         continue;
+      } else if (!runningApplications.contains(containerAppId)) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Container " + containerId
+              + " is the first container get launched for application "
+              + containerAppId);
+        }
+        runningApplications.add(containerAppId);
       }
 
       // Process running containers

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08244264/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.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/MockNodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
index 2d863d1..095fe28 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
@@ -187,6 +187,11 @@ public class MockNodes {
     }
 
     @Override
+    public List<ApplicationId> getRunningApps() {
+      return null;
+    }
+
+    @Override
     public void updateNodeHeartbeatResponseForCleanup(NodeHeartbeatResponse response) {
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08244264/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.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/TestRMNodeTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
index 01f4357..ece896b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
@@ -33,6 +33,7 @@ import java.util.List;
 import org.apache.hadoop.util.HostsFileReader;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 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.api.records.NodeState;
@@ -485,9 +486,9 @@ public class TestRMNodeTransitions {
     NodeId nodeId = node.getNodeID();
 
     // Expire a container
-		ContainerId completedContainerId = BuilderUtils.newContainerId(
-				BuilderUtils.newApplicationAttemptId(
-						BuilderUtils.newApplicationId(0, 0), 0), 0);
+    ContainerId completedContainerId = BuilderUtils.newContainerId(
+        BuilderUtils.newApplicationAttemptId(
+            BuilderUtils.newApplicationId(0, 0), 0), 0);
     node.handle(new RMNodeCleanContainerEvent(nodeId, completedContainerId));
     Assert.assertEquals(1, node.getContainersToCleanUp().size());
 
@@ -512,6 +513,35 @@ public class TestRMNodeTransitions {
     Assert.assertEquals(finishedAppId, hbrsp.getApplicationsToCleanup().get(0));
   }
 
+  @Test(timeout=20000)
+  public void testUpdateHeartbeatResponseForAppLifeCycle() {
+    RMNodeImpl node = getRunningNode();
+    NodeId nodeId = node.getNodeID();
+
+    ApplicationId runningAppId = BuilderUtils.newApplicationId(0, 1);
+    // Create a running container
+    ContainerId runningContainerId = BuilderUtils.newContainerId(
+        BuilderUtils.newApplicationAttemptId(
+        runningAppId, 0), 0);
+
+    ContainerStatus status = ContainerStatus.newInstance(runningContainerId,
+        ContainerState.RUNNING, "", 0);
+    List<ContainerStatus> statusList = new ArrayList<ContainerStatus>();
+    statusList.add(status);
+    NodeHealthStatus nodeHealth = NodeHealthStatus.newInstance(true,
+        "", System.currentTimeMillis());
+    node.handle(new RMNodeStatusEvent(nodeId, nodeHealth,
+        statusList, null, null));
+
+    Assert.assertEquals(1, node.getRunningApps().size());
+
+    // Finish an application
+    ApplicationId finishedAppId = runningAppId;
+    node.handle(new RMNodeCleanAppEvent(nodeId, finishedAppId));
+    Assert.assertEquals(1, node.getAppsToCleanup().size());
+    Assert.assertEquals(0, node.getRunningApps().size());
+  }
+
   private RMNodeImpl getRunningNode() {
     return getRunningNode(null, 0);
   }


[08/50] hadoop git commit: YARN-3892. Fixed NPE on RMStateStore#serviceStop when CapacityScheduler#serviceInit fails. Contributed by Bibin A Chundatt

Posted by ar...@apache.org.
YARN-3892. Fixed NPE on RMStateStore#serviceStop when CapacityScheduler#serviceInit fails. Contributed by Bibin A Chundatt


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

Branch: refs/heads/HDFS-7240
Commit: c9dd2cada055c0beffd04bad0ded8324f66ad1b7
Parents: c0b8e4e
Author: Jian He <ji...@apache.org>
Authored: Tue Jul 7 14:16:21 2015 -0700
Committer: Jian He <ji...@apache.org>
Committed: Tue Jul 7 14:16:21 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 3 +++
 .../yarn/server/resourcemanager/recovery/ZKRMStateStore.java      | 3 ++-
 2 files changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9dd2cad/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 2d1d6a2..8b5cc0c 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -598,6 +598,9 @@ Release 2.8.0 - UNRELEASED
     YARN-2194. Fix bug causing CGroups functionality to fail on RHEL7.
     (Wei Yan via vvasudev)
 
+    YARN-3892. Fixed NPE on RMStateStore#serviceStop when
+    CapacityScheduler#serviceInit fails. (Bibin A Chundatt via jianhe)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9dd2cad/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.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/recovery/ZKRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
index bca5348..8f096d8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
@@ -40,6 +40,7 @@ import org.apache.curator.retry.RetryNTimes;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
 import org.apache.hadoop.util.ZKUtil;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -312,7 +313,7 @@ public class ZKRMStateStore extends RMStateStore {
       verifyActiveStatusThread.interrupt();
       verifyActiveStatusThread.join(1000);
     }
-    curatorFramework.close();
+    IOUtils.closeStream(curatorFramework);
   }
 
   @Override


[30/50] hadoop git commit: HDFS-2956. calling fetchdt without a --renewer argument throws NPE (Contributed by Vinayakumar B)HDFS-2956. calling fetchdt without a --renewer argument throws NPE (Contributed by Vinayakumar B)

Posted by ar...@apache.org.
HDFS-2956. calling fetchdt without a --renewer argument throws NPE (Contributed by Vinayakumar B)HDFS-2956. calling fetchdt without a --renewer argument throws NPE (Contributed by Vinayakumar B)


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

Branch: refs/heads/HDFS-7240
Commit: b48908033fcac7a4bd4313c1fd1457999fba08e1
Parents: d66302e
Author: Vinayakumar B <vi...@apache.org>
Authored: Fri Jul 10 15:47:04 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Fri Jul 10 15:47:04 2015 +0530

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../ClientNamenodeProtocolTranslatorPB.java     |  2 +-
 .../hdfs/tools/TestDelegationTokenFetcher.java  | 39 ++++++++++++++++++++
 3 files changed, 43 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b4890803/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 5c1208d..13b2621 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1029,6 +1029,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8749. Fix findbugs warnings in BlockManager.java.
     (Brahma Reddy Battula via aajisaka)
 
+    HDFS-2956. calling fetchdt without a --renewer argument throws NPE
+    (vinayakumarb)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b4890803/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 4ec6f9e..566d54f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -929,7 +929,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throws IOException {
     GetDelegationTokenRequestProto req = GetDelegationTokenRequestProto
         .newBuilder()
-        .setRenewer(renewer.toString())
+        .setRenewer(renewer == null ? "" : renewer.toString())
         .build();
     try {
       GetDelegationTokenResponseProto resp = rpcProxy.getDelegationToken(null, req);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b4890803/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDelegationTokenFetcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDelegationTokenFetcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDelegationTokenFetcher.java
index ab3933b..80a1a6c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDelegationTokenFetcher.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDelegationTokenFetcher.java
@@ -18,7 +18,10 @@
 
 package org.apache.hadoop.hdfs.tools;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.doThrow;
@@ -28,12 +31,18 @@ import java.io.IOException;
 import java.util.Iterator;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.tools.FakeRenewer;
 import org.junit.Assert;
 import org.junit.Rule;
@@ -105,4 +114,34 @@ public class TestDelegationTokenFetcher {
     Assert.assertFalse(p.getFileSystem(conf).exists(p));
 
   }
+
+  @Test
+  public void testDelegationTokenWithoutRenewerViaRPC() throws Exception {
+    conf.setBoolean(DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+        .build();
+    try {
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      // Should be able to fetch token without renewer.
+      LocalFileSystem localFileSystem = FileSystem.getLocal(conf);
+      Path p = new Path(f.getRoot().getAbsolutePath(), tokenFile);
+      p = localFileSystem.makeQualified(p);
+      DelegationTokenFetcher.saveDelegationToken(conf, fs, null, p);
+      Credentials creds = Credentials.readTokenStorageFile(p, conf);
+      Iterator<Token<?>> itr = creds.getAllTokens().iterator();
+      assertTrue("token not exist error", itr.hasNext());
+      assertNotNull("Token should be there without renewer", itr.next());
+      try {
+        // Without renewer renewal of token should fail.
+        DelegationTokenFetcher.renewTokens(conf, p);
+        fail("Should have failed to renew");
+      } catch (AccessControlException e) {
+        GenericTestUtils.assertExceptionContains(
+            "tried to renew a token without a renewer", e);
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
 }


[18/50] hadoop git commit: HADOOP-12200. TestCryptoStreamsWithOpensslAesCtrCryptoCodec should be skipped in non-native profile. Contributed by Masatake Iwasaki.

Posted by ar...@apache.org.
HADOOP-12200. TestCryptoStreamsWithOpensslAesCtrCryptoCodec should be skipped in non-native profile. Contributed by Masatake Iwasaki.


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

Branch: refs/heads/HDFS-7240
Commit: 6b7692c468931215ec0de7e11f0bdf15b80e68ce
Parents: b8832fc
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Jul 9 13:14:53 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Jul 9 13:14:53 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                   | 3 +++
 .../crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java     | 3 +++
 2 files changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b7692c4/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 c99fb5e..a906a63 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -948,6 +948,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12117. Potential NPE from Configuration#loadProperty with
     allowNullValueProperties set. (zhihai xu via vinayakumarb)
 
+    HADOOP-12200. TestCryptoStreamsWithOpensslAesCtrCryptoCodec should be
+    skipped in non-native profile. (Masatake Iwasaki via aajisaka)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b7692c4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java
index 684ec09..d5f25b8 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java
@@ -19,16 +19,19 @@ package org.apache.hadoop.crypto;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.util.NativeCodeLoader;
 import org.junit.BeforeClass;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assume.assumeTrue;
 
 public class TestCryptoStreamsWithOpensslAesCtrCryptoCodec 
     extends TestCryptoStreams {
   
   @BeforeClass
   public static void init() throws Exception {
+    assumeTrue(NativeCodeLoader.isNativeCodeLoaded());
     Configuration conf = new Configuration();
     conf.set(
         CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_KEY,


[12/50] hadoop git commit: HDFS-8642. Make TestFileTruncate more reliable. (Contributed by Rakesh R)

Posted by ar...@apache.org.
HDFS-8642. Make TestFileTruncate more reliable. (Contributed by Rakesh R)


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

Branch: refs/heads/HDFS-7240
Commit: 4119ad3112dcfb7286ca68288489bbcb6235cf53
Parents: 98e5926
Author: Arpit Agarwal <ar...@apache.org>
Authored: Wed Jul 8 09:31:02 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Wed Jul 8 09:31:02 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt       |  3 +++
 .../hdfs/server/namenode/TestFileTruncate.java    | 18 ++++++------------
 2 files changed, 9 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4119ad31/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 b88b42a..a7c9e7c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1017,6 +1017,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8686. WebHdfsFileSystem#getXAttr(Path p, final String name) doesn't
     work if namespace is in capitals (kanaka kumar avvaru via vinayakumarb)
 
+    HDFS-8642. Make TestFileTruncate more reliable. (Rakesh R via
+    Arpit Agarwal)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4119ad31/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index e0f9ad2..a91d6c9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -62,9 +62,8 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.log4j.Level;
-import org.junit.AfterClass;
+import org.junit.After;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class TestFileTruncate {
@@ -90,8 +89,8 @@ public class TestFileTruncate {
 
  private Path parent;
 
-  @BeforeClass
-  public static void startUp() throws IOException {
+  @Before
+  public void setUp() throws IOException {
     conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY, BLOCK_SIZE);
     conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, BLOCK_SIZE);
@@ -105,20 +104,15 @@ public class TestFileTruncate {
         .waitSafeMode(true)
         .build();
     fs = cluster.getFileSystem();
+    parent = new Path("/test");
   }
 
-  @AfterClass
-  public static void tearDown() throws IOException {
+  @After
+  public void tearDown() throws IOException {
     if(fs != null)      fs.close();
     if(cluster != null) cluster.shutdown();
   }
 
-  @Before
-  public void setup() throws IOException {
-    parent = new Path("/test");
-    fs.delete(parent, true);
-  }
-
   /**
    * Truncate files of different sizes byte by byte.
    */


[44/50] hadoop git commit: YARN-3453. Ensure preemption logic in FairScheduler uses DominantResourceCalculator in DRF queues to prevent unnecessary thrashing. (asuresh)

Posted by ar...@apache.org.
YARN-3453. Ensure preemption logic in FairScheduler uses DominantResourceCalculator in DRF queues to prevent unnecessary thrashing. (asuresh)


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

Branch: refs/heads/HDFS-7240
Commit: ac94ba3e185115b83351e35c610c2b8ff91b1ebc
Parents: a431ed9
Author: Arun Suresh <as...@apache.org>
Authored: Tue Jul 14 00:23:55 2015 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Tue Jul 14 00:23:55 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../scheduler/fair/FSLeafQueue.java             |   9 +-
 .../scheduler/fair/FairScheduler.java           |  40 ++--
 .../scheduler/fair/SchedulingPolicy.java        |  11 +
 .../DominantResourceFairnessPolicy.java         |  18 +-
 .../fair/policies/FairSharePolicy.java          |  11 +-
 .../scheduler/fair/policies/FifoPolicy.java     |  15 +-
 .../scheduler/fair/TestFSLeafQueue.java         |  64 ++++++
 .../scheduler/fair/TestFairScheduler.java       | 207 ++++++++++++++++---
 9 files changed, 317 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac94ba3e/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 5c17f04..780c667 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -628,6 +628,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3381. Fix typo InvalidStateTransitonException.
     (Brahma Reddy Battula via aajisaka)
 
+    YARN-3453. Ensure preemption logic in FairScheduler uses DominantResourceCalculator
+    in DRF queues to prevent unnecessary thrashing. (asuresh)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac94ba3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
index 6779a1b..f90a198 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
@@ -560,9 +560,10 @@ public class FSLeafQueue extends FSQueue {
   }
 
   private boolean isStarved(Resource share) {
-    Resource desiredShare = Resources.min(scheduler.getResourceCalculator(),
-        scheduler.getClusterResource(), share, getDemand());
-    return Resources.lessThan(scheduler.getResourceCalculator(),
-        scheduler.getClusterResource(), getResourceUsage(), desiredShare);
+    Resource desiredShare = Resources.min(policy.getResourceCalculator(),
+            scheduler.getClusterResource(), share, getDemand());
+    Resource resourceUsage = getResourceUsage();
+    return Resources.lessThan(policy.getResourceCalculator(),
+            scheduler.getClusterResource(), resourceUsage, desiredShare);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac94ba3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index cbc10e7..efe6544 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -371,10 +371,9 @@ public class FairScheduler extends
 
     Resource resToPreempt = Resources.clone(Resources.none());
     for (FSLeafQueue sched : queueMgr.getLeafQueues()) {
-      Resources.addTo(resToPreempt, resToPreempt(sched, curTime));
+      Resources.addTo(resToPreempt, resourceDeficit(sched, curTime));
     }
-    if (Resources.greaterThan(RESOURCE_CALCULATOR, clusterResource, resToPreempt,
-        Resources.none())) {
+    if (isResourceGreaterThanNone(resToPreempt)) {
       preemptResources(resToPreempt);
     }
   }
@@ -404,8 +403,7 @@ public class FairScheduler extends
       RMContainer container = warnedIter.next();
       if ((container.getState() == RMContainerState.RUNNING ||
               container.getState() == RMContainerState.ALLOCATED) &&
-          Resources.greaterThan(RESOURCE_CALCULATOR, clusterResource,
-              toPreempt, Resources.none())) {
+              isResourceGreaterThanNone(toPreempt)) {
         warnOrKillContainer(container);
         Resources.subtractFrom(toPreempt, container.getContainer().getResource());
       } else {
@@ -419,8 +417,7 @@ public class FairScheduler extends
         queue.resetPreemptedResources();
       }
 
-      while (Resources.greaterThan(RESOURCE_CALCULATOR, clusterResource,
-          toPreempt, Resources.none())) {
+      while (isResourceGreaterThanNone(toPreempt)) {
         RMContainer container =
             getQueueManager().getRootQueue().preemptContainer();
         if (container == null) {
@@ -442,7 +439,11 @@ public class FairScheduler extends
     long duration = getClock().getTime() - start;
     fsOpDurations.addPreemptCallDuration(duration);
   }
-  
+
+  private boolean isResourceGreaterThanNone(Resource toPreempt) {
+    return (toPreempt.getMemory() > 0) || (toPreempt.getVirtualCores() > 0);
+  }
+
   protected void warnOrKillContainer(RMContainer container) {
     ApplicationAttemptId appAttemptId = container.getApplicationAttemptId();
     FSAppAttempt app = getSchedulerApp(appAttemptId);
@@ -485,33 +486,34 @@ public class FairScheduler extends
    * max of the two amounts (this shouldn't happen unless someone sets the
    * timeouts to be identical for some reason).
    */
-  protected Resource resToPreempt(FSLeafQueue sched, long curTime) {
+  protected Resource resourceDeficit(FSLeafQueue sched, long curTime) {
     long minShareTimeout = sched.getMinSharePreemptionTimeout();
     long fairShareTimeout = sched.getFairSharePreemptionTimeout();
     Resource resDueToMinShare = Resources.none();
     Resource resDueToFairShare = Resources.none();
+    ResourceCalculator calc = sched.getPolicy().getResourceCalculator();
     if (curTime - sched.getLastTimeAtMinShare() > minShareTimeout) {
-      Resource target = Resources.min(RESOURCE_CALCULATOR, clusterResource,
+      Resource target = Resources.componentwiseMin(
           sched.getMinShare(), sched.getDemand());
-      resDueToMinShare = Resources.max(RESOURCE_CALCULATOR, clusterResource,
+      resDueToMinShare = Resources.max(calc, clusterResource,
           Resources.none(), Resources.subtract(target, sched.getResourceUsage()));
     }
     if (curTime - sched.getLastTimeAtFairShareThreshold() > fairShareTimeout) {
-      Resource target = Resources.min(RESOURCE_CALCULATOR, clusterResource,
-          sched.getFairShare(), sched.getDemand());
-      resDueToFairShare = Resources.max(RESOURCE_CALCULATOR, clusterResource,
+      Resource target = Resources.componentwiseMin(
+              sched.getFairShare(), sched.getDemand());
+      resDueToFairShare = Resources.max(calc, clusterResource,
           Resources.none(), Resources.subtract(target, sched.getResourceUsage()));
     }
-    Resource resToPreempt = Resources.max(RESOURCE_CALCULATOR, clusterResource,
+    Resource deficit = Resources.max(calc, clusterResource,
         resDueToMinShare, resDueToFairShare);
-    if (Resources.greaterThan(RESOURCE_CALCULATOR, clusterResource,
-        resToPreempt, Resources.none())) {
-      String message = "Should preempt " + resToPreempt + " res for queue "
+    if (Resources.greaterThan(calc, clusterResource,
+        deficit, Resources.none())) {
+      String message = "Should preempt " + deficit + " res for queue "
           + sched.getName() + ": resDueToMinShare = " + resDueToMinShare
           + ", resDueToFairShare = " + resDueToFairShare;
       LOG.info(message);
     }
-    return resToPreempt;
+    return deficit;
   }
 
   public synchronized RMContainerTokenSecretManager

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac94ba3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
index abdc834..160ba4b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
@@ -26,6 +26,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.Dom
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FairSharePolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
 
+
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.concurrent.ConcurrentHashMap;
@@ -98,6 +101,14 @@ public abstract class SchedulingPolicy {
   public void initialize(Resource clusterCapacity) {}
 
   /**
+   * The {@link ResourceCalculator} returned by this method should be used
+   * for any calculations involving resources.
+   *
+   * @return ResourceCalculator instance to use
+   */
+  public abstract ResourceCalculator getResourceCalculator();
+
+  /**
    * @return returns the name of {@link SchedulingPolicy}
    */
   public abstract String getName();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac94ba3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
index 86d503b..45fbf98 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
@@ -29,6 +29,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.Schedulable;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SchedulingPolicy;
+
+import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 import static org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceType.*;
@@ -44,8 +47,10 @@ public class DominantResourceFairnessPolicy extends SchedulingPolicy {
 
   public static final String NAME = "DRF";
 
-  private DominantResourceFairnessComparator comparator =
+  private static final DominantResourceFairnessComparator COMPARATOR =
       new DominantResourceFairnessComparator();
+  private static final DominantResourceCalculator CALCULATOR =
+      new DominantResourceCalculator();
 
   @Override
   public String getName() {
@@ -59,9 +64,14 @@ public class DominantResourceFairnessPolicy extends SchedulingPolicy {
 
   @Override
   public Comparator<Schedulable> getComparator() {
-    return comparator;
+    return COMPARATOR;
   }
-  
+
+  @Override
+  public ResourceCalculator getResourceCalculator() {
+    return CALCULATOR;
+  }
+
   @Override
   public void computeShares(Collection<? extends Schedulable> schedulables,
       Resource totalResources) {
@@ -105,7 +115,7 @@ public class DominantResourceFairnessPolicy extends SchedulingPolicy {
 
   @Override
   public void initialize(Resource clusterCapacity) {
-    comparator.setClusterCapacity(clusterCapacity);
+    COMPARATOR.setClusterCapacity(clusterCapacity);
   }
 
   public static class DominantResourceFairnessComparator implements Comparator<Schedulable> {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac94ba3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
index 918db9d..3b9f07f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.Schedulable;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SchedulingPolicy;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -43,7 +44,8 @@ public class FairSharePolicy extends SchedulingPolicy {
   public static final String NAME = "fair";
   private static final DefaultResourceCalculator RESOURCE_CALCULATOR =
       new DefaultResourceCalculator();
-  private FairShareComparator comparator = new FairShareComparator();
+  private static final FairShareComparator COMPARATOR =
+          new FairShareComparator();
 
   @Override
   public String getName() {
@@ -111,7 +113,12 @@ public class FairSharePolicy extends SchedulingPolicy {
 
   @Override
   public Comparator<Schedulable> getComparator() {
-    return comparator;
+    return COMPARATOR;
+  }
+
+  @Override
+  public ResourceCalculator getResourceCalculator() {
+    return RESOURCE_CALCULATOR;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac94ba3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
index 7d88933..a644e58 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
@@ -27,6 +27,10 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.Schedulable;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SchedulingPolicy;
+
+
+import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -36,7 +40,9 @@ import com.google.common.annotations.VisibleForTesting;
 public class FifoPolicy extends SchedulingPolicy {
   @VisibleForTesting
   public static final String NAME = "FIFO";
-  private FifoComparator comparator = new FifoComparator();
+  private static final FifoComparator COMPARATOR = new FifoComparator();
+  private static final DefaultResourceCalculator CALCULATOR =
+          new DefaultResourceCalculator();
 
   @Override
   public String getName() {
@@ -68,7 +74,12 @@ public class FifoPolicy extends SchedulingPolicy {
 
   @Override
   public Comparator<Schedulable> getComparator() {
-    return comparator;
+    return COMPARATOR;
+  }
+
+  @Override
+  public ResourceCalculator getResourceCalculator() {
+    return CALCULATOR;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac94ba3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSLeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSLeafQueue.java
index 385ea0b..7637410 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSLeafQueue.java
@@ -233,6 +233,70 @@ public class TestFSLeafQueue extends FairSchedulerTestBase {
     assertFalse(queueB2.isStarvedForFairShare());
   }
 
+  @Test (timeout = 5000)
+  public void testIsStarvedForFairShareDRF() throws Exception {
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"queueA\">");
+    out.println("<weight>.5</weight>");
+    out.println("</queue>");
+    out.println("<queue name=\"queueB\">");
+    out.println("<weight>.5</weight>");
+    out.println("</queue>");
+    out.println("<defaultFairSharePreemptionThreshold>1</defaultFairSharePreemptionThreshold>");
+    out.println("<defaultQueueSchedulingPolicy>drf</defaultQueueSchedulingPolicy>");
+    out.println("</allocations>");
+    out.close();
+
+    resourceManager = new MockRM(conf);
+    resourceManager.start();
+    scheduler = (FairScheduler) resourceManager.getResourceScheduler();
+
+    // Add one big node (only care about aggregate capacity)
+    RMNode node1 =
+            MockNodes.newNodeInfo(1, Resources.createResource(10 * 1024, 10), 1,
+                    "127.0.0.1");
+    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
+    scheduler.handle(nodeEvent1);
+
+    scheduler.update();
+
+    // Queue A wants 7 * 1024, 1. Node update gives this all to A
+    createSchedulingRequest(7 * 1024, 1, "queueA", "user1", 1);
+    scheduler.update();
+    NodeUpdateSchedulerEvent nodeEvent2 = new NodeUpdateSchedulerEvent(node1);
+    scheduler.handle(nodeEvent2);
+
+    QueueManager queueMgr = scheduler.getQueueManager();
+    FSLeafQueue queueA = queueMgr.getLeafQueue("queueA", false);
+    assertEquals(7 * 1024, queueA.getResourceUsage().getMemory());
+    assertEquals(1, queueA.getResourceUsage().getVirtualCores());
+
+    // Queue B has 3 reqs :
+    // 1) 2 * 1024, 5 .. which will be granted
+    // 2) 1 * 1024, 1 .. which will be granted
+    // 3) 1 * 1024, 1 .. which wont
+    createSchedulingRequest(2 * 1024, 5, "queueB", "user1", 1);
+    createSchedulingRequest(1 * 1024, 2, "queueB", "user1", 2);
+    scheduler.update();
+    for (int i = 0; i < 3; i ++) {
+      scheduler.handle(nodeEvent2);
+    }
+
+    FSLeafQueue queueB = queueMgr.getLeafQueue("queueB", false);
+    assertEquals(3 * 1024, queueB.getResourceUsage().getMemory());
+    assertEquals(6, queueB.getResourceUsage().getVirtualCores());
+
+    scheduler.update();
+
+    // Verify that Queue us not starved for fair share..
+    // Since the Starvation logic now uses DRF when the policy = drf, The
+    // Queue should not be starved
+    assertFalse(queueB.isStarvedForFairShare());
+  }
+
   @Test
   public void testConcurrentAccess() {
     conf.set(FairSchedulerConfiguration.ASSIGN_MULTIPLE, "false");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac94ba3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
index 56e8adc..2260f73 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
@@ -100,7 +100,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.Dom
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ControlledClock;
-import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.After;
 import org.junit.Assert;
@@ -1706,8 +1705,8 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     clock.tickSec(11);
 
     scheduler.update();
-    Resource toPreempt = scheduler.resToPreempt(scheduler.getQueueManager()
-        .getLeafQueue("queueA.queueA2", false), clock.getTime());
+    Resource toPreempt = scheduler.resourceDeficit(scheduler.getQueueManager()
+            .getLeafQueue("queueA.queueA2", false), clock.getTime());
     assertEquals(3277, toPreempt.getMemory());
 
     // verify if the 3 containers required by queueA2 are preempted in the same
@@ -1829,25 +1828,173 @@ public class TestFairScheduler extends FairSchedulerTestBase {
         scheduler.getQueueManager().getLeafQueue("queueD", true);
 
     assertTrue(Resources.equals(
-        Resources.none(), scheduler.resToPreempt(schedC, clock.getTime())));
+        Resources.none(), scheduler.resourceDeficit(schedC, clock.getTime())));
     assertTrue(Resources.equals(
-        Resources.none(), scheduler.resToPreempt(schedD, clock.getTime())));
+        Resources.none(), scheduler.resourceDeficit(schedD, clock.getTime())));
     // After minSharePreemptionTime has passed, they should want to preempt min
     // share.
     clock.tickSec(6);
     assertEquals(
-        1024, scheduler.resToPreempt(schedC, clock.getTime()).getMemory());
+        1024, scheduler.resourceDeficit(schedC, clock.getTime()).getMemory());
     assertEquals(
-        1024, scheduler.resToPreempt(schedD, clock.getTime()).getMemory());
+        1024, scheduler.resourceDeficit(schedD, clock.getTime()).getMemory());
 
     // After fairSharePreemptionTime has passed, they should want to preempt
     // fair share.
     scheduler.update();
     clock.tickSec(6);
     assertEquals(
-        1536 , scheduler.resToPreempt(schedC, clock.getTime()).getMemory());
+        1536 , scheduler.resourceDeficit(schedC, clock.getTime()).getMemory());
     assertEquals(
-        1536, scheduler.resToPreempt(schedD, clock.getTime()).getMemory());
+        1536, scheduler.resourceDeficit(schedD, clock.getTime()).getMemory());
+  }
+
+  @Test
+/**
+ * Tests the timing of decision to preempt tasks.
+ */
+  public void testPreemptionDecisionWithDRF() throws Exception {
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+    ControlledClock clock = new ControlledClock();
+    scheduler.setClock(clock);
+
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"default\">");
+    out.println("<maxResources>0mb,0vcores</maxResources>");
+    out.println("</queue>");
+    out.println("<queue name=\"queueA\">");
+    out.println("<weight>.25</weight>");
+    out.println("<minResources>1024mb,1vcores</minResources>");
+    out.println("</queue>");
+    out.println("<queue name=\"queueB\">");
+    out.println("<weight>.25</weight>");
+    out.println("<minResources>1024mb,2vcores</minResources>");
+    out.println("</queue>");
+    out.println("<queue name=\"queueC\">");
+    out.println("<weight>.25</weight>");
+    out.println("<minResources>1024mb,3vcores</minResources>");
+    out.println("</queue>");
+    out.println("<queue name=\"queueD\">");
+    out.println("<weight>.25</weight>");
+    out.println("<minResources>1024mb,2vcores</minResources>");
+    out.println("</queue>");
+    out.println("<defaultMinSharePreemptionTimeout>5</defaultMinSharePreemptionTimeout>");
+    out.println("<defaultFairSharePreemptionTimeout>10</defaultFairSharePreemptionTimeout>");
+    out.println("<defaultFairSharePreemptionThreshold>.5</defaultFairSharePreemptionThreshold>");
+    out.println("<defaultQueueSchedulingPolicy>drf</defaultQueueSchedulingPolicy>");
+    out.println("</allocations>");
+    out.close();
+
+    scheduler.init(conf);
+    scheduler.start();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
+    // Create four nodes
+    RMNode node1 =
+            MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024, 4), 1,
+                    "127.0.0.1");
+    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
+    scheduler.handle(nodeEvent1);
+
+    RMNode node2 =
+            MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024, 4), 2,
+                    "127.0.0.2");
+    NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
+    scheduler.handle(nodeEvent2);
+
+    RMNode node3 =
+            MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024, 4), 3,
+                    "127.0.0.3");
+    NodeAddedSchedulerEvent nodeEvent3 = new NodeAddedSchedulerEvent(node3);
+    scheduler.handle(nodeEvent3);
+
+    // Queue A and B each request three containers
+    ApplicationAttemptId app1 =
+            createSchedulingRequest(1 * 1024, "queueA", "user1", 1, 1);
+    ApplicationAttemptId app2 =
+            createSchedulingRequest(1 * 1024, "queueA", "user1", 1, 2);
+    ApplicationAttemptId app3 =
+            createSchedulingRequest(1 * 1024, "queueA", "user1", 1, 3);
+
+    ApplicationAttemptId app4 =
+            createSchedulingRequest(1 * 1024, "queueB", "user1", 1, 1);
+    ApplicationAttemptId app5 =
+            createSchedulingRequest(1 * 1024, "queueB", "user1", 1, 2);
+    ApplicationAttemptId app6 =
+            createSchedulingRequest(1 * 1024, "queueB", "user1", 1, 3);
+
+    scheduler.update();
+
+    // Sufficient node check-ins to fully schedule containers
+    for (int i = 0; i < 2; i++) {
+      NodeUpdateSchedulerEvent nodeUpdate1 = new NodeUpdateSchedulerEvent(node1);
+      scheduler.handle(nodeUpdate1);
+
+      NodeUpdateSchedulerEvent nodeUpdate2 = new NodeUpdateSchedulerEvent(node2);
+      scheduler.handle(nodeUpdate2);
+
+      NodeUpdateSchedulerEvent nodeUpdate3 = new NodeUpdateSchedulerEvent(node3);
+      scheduler.handle(nodeUpdate3);
+    }
+
+    // Now new requests arrive from queues C and D
+    ApplicationAttemptId app7 =
+            createSchedulingRequest(1 * 1024, "queueC", "user1", 1, 1);
+    ApplicationAttemptId app8 =
+            createSchedulingRequest(1 * 1024, "queueC", "user1", 1, 2);
+    ApplicationAttemptId app9 =
+            createSchedulingRequest(1 * 1024, "queueC", "user1", 1, 3);
+
+    ApplicationAttemptId app10 =
+            createSchedulingRequest(1 * 1024, "queueD", "user1", 2, 1);
+    ApplicationAttemptId app11 =
+            createSchedulingRequest(1 * 1024, "queueD", "user1", 2, 2);
+    ApplicationAttemptId app12 =
+            createSchedulingRequest(1 * 1024, "queueD", "user1", 2, 3);
+
+    scheduler.update();
+
+    FSLeafQueue schedC =
+            scheduler.getQueueManager().getLeafQueue("queueC", true);
+    FSLeafQueue schedD =
+            scheduler.getQueueManager().getLeafQueue("queueD", true);
+
+    assertTrue(Resources.equals(
+            Resources.none(), scheduler.resourceDeficit(schedC, clock.getTime())));
+    assertTrue(Resources.equals(
+            Resources.none(), scheduler.resourceDeficit(schedD, clock.getTime())));
+
+    // Test :
+    // 1) whether componentWise min works as expected.
+    // 2) DRF calculator is used
+
+    // After minSharePreemptionTime has passed, they should want to preempt min
+    // share.
+    clock.tickSec(6);
+    Resource res = scheduler.resourceDeficit(schedC, clock.getTime());
+    assertEquals(1024, res.getMemory());
+    // Demand = 3
+    assertEquals(3, res.getVirtualCores());
+
+    res = scheduler.resourceDeficit(schedD, clock.getTime());
+    assertEquals(1024, res.getMemory());
+    // Demand = 6, but min share = 2
+    assertEquals(2, res.getVirtualCores());
+
+    // After fairSharePreemptionTime has passed, they should want to preempt
+    // fair share.
+    scheduler.update();
+    clock.tickSec(6);
+    res = scheduler.resourceDeficit(schedC, clock.getTime());
+    assertEquals(1536, res.getMemory());
+    assertEquals(3, res.getVirtualCores());
+
+    res = scheduler.resourceDeficit(schedD, clock.getTime());
+    assertEquals(1536, res.getMemory());
+    // Demand = 6, but fair share = 3
+    assertEquals(3, res.getVirtualCores());
   }
 
   @Test
@@ -1964,71 +2111,71 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     FSLeafQueue queueC = queueMgr.getLeafQueue("queueC", true);
 
     assertTrue(Resources.equals(
-        Resources.none(), scheduler.resToPreempt(queueB1, clock.getTime())));
+        Resources.none(), scheduler.resourceDeficit(queueB1, clock.getTime())));
     assertTrue(Resources.equals(
-        Resources.none(), scheduler.resToPreempt(queueB2, clock.getTime())));
+        Resources.none(), scheduler.resourceDeficit(queueB2, clock.getTime())));
     assertTrue(Resources.equals(
-        Resources.none(), scheduler.resToPreempt(queueC, clock.getTime())));
+        Resources.none(), scheduler.resourceDeficit(queueC, clock.getTime())));
 
     // After 5 seconds, queueB1 wants to preempt min share
     scheduler.update();
     clock.tickSec(6);
     assertEquals(
-       1024, scheduler.resToPreempt(queueB1, clock.getTime()).getMemory());
+       1024, scheduler.resourceDeficit(queueB1, clock.getTime()).getMemory());
     assertEquals(
-        0, scheduler.resToPreempt(queueB2, clock.getTime()).getMemory());
+        0, scheduler.resourceDeficit(queueB2, clock.getTime()).getMemory());
     assertEquals(
-        0, scheduler.resToPreempt(queueC, clock.getTime()).getMemory());
+        0, scheduler.resourceDeficit(queueC, clock.getTime()).getMemory());
 
     // After 10 seconds, queueB2 wants to preempt min share
     scheduler.update();
     clock.tickSec(5);
     assertEquals(
-        1024, scheduler.resToPreempt(queueB1, clock.getTime()).getMemory());
+        1024, scheduler.resourceDeficit(queueB1, clock.getTime()).getMemory());
     assertEquals(
-        1024, scheduler.resToPreempt(queueB2, clock.getTime()).getMemory());
+        1024, scheduler.resourceDeficit(queueB2, clock.getTime()).getMemory());
     assertEquals(
-        0, scheduler.resToPreempt(queueC, clock.getTime()).getMemory());
+        0, scheduler.resourceDeficit(queueC, clock.getTime()).getMemory());
 
     // After 15 seconds, queueC wants to preempt min share
     scheduler.update();
     clock.tickSec(5);
     assertEquals(
-        1024, scheduler.resToPreempt(queueB1, clock.getTime()).getMemory());
+        1024, scheduler.resourceDeficit(queueB1, clock.getTime()).getMemory());
     assertEquals(
-        1024, scheduler.resToPreempt(queueB2, clock.getTime()).getMemory());
+        1024, scheduler.resourceDeficit(queueB2, clock.getTime()).getMemory());
     assertEquals(
-        1024, scheduler.resToPreempt(queueC, clock.getTime()).getMemory());
+        1024, scheduler.resourceDeficit(queueC, clock.getTime()).getMemory());
 
     // After 20 seconds, queueB2 should want to preempt fair share
     scheduler.update();
     clock.tickSec(5);
     assertEquals(
-        1024, scheduler.resToPreempt(queueB1, clock.getTime()).getMemory());
+        1024, scheduler.resourceDeficit(queueB1, clock.getTime()).getMemory());
     assertEquals(
-        1536, scheduler.resToPreempt(queueB2, clock.getTime()).getMemory());
+        1536, scheduler.resourceDeficit(queueB2, clock.getTime()).getMemory());
     assertEquals(
-        1024, scheduler.resToPreempt(queueC, clock.getTime()).getMemory());
+        1024, scheduler.resourceDeficit(queueC, clock.getTime()).getMemory());
 
     // After 25 seconds, queueB1 should want to preempt fair share
     scheduler.update();
     clock.tickSec(5);
     assertEquals(
-        1536, scheduler.resToPreempt(queueB1, clock.getTime()).getMemory());
+        1536, scheduler.resourceDeficit(queueB1, clock.getTime()).getMemory());
     assertEquals(
-        1536, scheduler.resToPreempt(queueB2, clock.getTime()).getMemory());
+        1536, scheduler.resourceDeficit(queueB2, clock.getTime()).getMemory());
     assertEquals(
-        1024, scheduler.resToPreempt(queueC, clock.getTime()).getMemory());
+        1024, scheduler.resourceDeficit(queueC, clock.getTime()).getMemory());
 
     // After 30 seconds, queueC should want to preempt fair share
     scheduler.update();
     clock.tickSec(5);
     assertEquals(
-        1536, scheduler.resToPreempt(queueB1, clock.getTime()).getMemory());
+        1536, scheduler.resourceDeficit(queueB1, clock.getTime()).getMemory());
     assertEquals(
-        1536, scheduler.resToPreempt(queueB2, clock.getTime()).getMemory());
+        1536, scheduler.resourceDeficit(queueB2, clock.getTime()).getMemory());
     assertEquals(
-        1536, scheduler.resToPreempt(queueC, clock.getTime()).getMemory());
+        1536, scheduler.resourceDeficit(queueC, clock.getTime()).getMemory());
   }
 
   @Test


[49/50] hadoop git commit: HDFS-8742. Inotify: Support event for OP_TRUNCATE. Contributed by Surendra Singh Lilhore.

Posted by ar...@apache.org.
HDFS-8742. Inotify: Support event for OP_TRUNCATE. 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/979c9ca2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/979c9ca2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/979c9ca2

Branch: refs/heads/HDFS-7240
Commit: 979c9ca2ca89e99dc7165abfa29c78d66de43d9a
Parents: 59388a8
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Jul 15 04:41:54 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Jul 15 04:42:08 2015 +0900

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/inotify/Event.java   | 37 +++++++++++++++++++-
 .../src/main/proto/inotify.proto                |  7 ++++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 17 +++++++++
 .../namenode/InotifyFSEditLogOpTranslator.java  |  4 +++
 .../hdfs/TestDFSInotifyEventInputStream.java    | 17 ++++++++-
 6 files changed, 83 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/979c9ca2/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/Event.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/Event.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/Event.java
index 53eefa0..dee17a9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/Event.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/Event.java
@@ -35,7 +35,7 @@ import java.util.List;
 @InterfaceStability.Unstable
 public abstract class Event {
   public static enum EventType {
-    CREATE, CLOSE, APPEND, RENAME, METADATA, UNLINK
+    CREATE, CLOSE, APPEND, RENAME, METADATA, UNLINK, TRUNCATE
   }
 
   private EventType eventType;
@@ -542,4 +542,39 @@ public abstract class Event {
       return timestamp;
     }
   }
+
+  /**
+   * Sent when a file is truncated.
+   */
+  public static class TruncateEvent extends Event {
+    private String path;
+    private long fileSize;
+    private long timestamp;
+
+
+    public TruncateEvent(String path, long fileSize, long timestamp) {
+      super(EventType.TRUNCATE);
+      this.path = path;
+      this.fileSize = fileSize;
+      this.timestamp = timestamp;
+    }
+
+    public String getPath() {
+      return path;
+    }
+
+    /**
+     * The size of the truncated file in bytes.
+     */
+    public long getFileSize() {
+      return fileSize;
+    }
+
+    /**
+     * The time when this event occurred, in milliseconds since the epoch.
+     */
+    public long getTimestamp() {
+      return timestamp;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/979c9ca2/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/inotify.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/inotify.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/inotify.proto
index 5b78fe6..5339902 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/inotify.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/inotify.proto
@@ -41,6 +41,7 @@ enum EventType {
   EVENT_RENAME = 0x3;
   EVENT_METADATA = 0x4;
   EVENT_UNLINK = 0x5;
+  EVENT_TRUNCATE = 0x6;
 }
 
 message EventProto {
@@ -87,6 +88,12 @@ message CloseEventProto {
   required int64 timestamp = 3;
 }
 
+message TruncateEventProto {
+  required string path = 1;
+  required int64 fileSize = 2;
+  required int64 timestamp = 3;
+}
+
 message AppendEventProto {
   required string path = 1;
   optional bool newBlock = 2 [default = false];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/979c9ca2/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 14f3403..20bdef0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -719,6 +719,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8541. Mover should exit with NO_MOVE_PROGRESS if there is no move
     progress.  (Surendra Singh Lilhore via szetszwo)
 
+    HDFS-8742. Inotify: Support event for OP_TRUNCATE.
+    (Surendra Singh Lilhore via aajisaka)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/979c9ca2/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 32d9614..4ca5b26 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
@@ -2685,6 +2685,12 @@ public class PBHelper {
                   .timestamp(unlink.getTimestamp())
                   .build());
             break;
+          case EVENT_TRUNCATE:
+            InotifyProtos.TruncateEventProto truncate =
+                InotifyProtos.TruncateEventProto.parseFrom(p.getContents());
+            events.add(new Event.TruncateEvent(truncate.getPath(),
+                truncate.getFileSize(), truncate.getTimestamp()));
+            break;
           default:
             throw new RuntimeException("Unexpected inotify event type: " +
                 p.getType());
@@ -2791,6 +2797,17 @@ public class PBHelper {
                         .setTimestamp(ue.getTimestamp()).build().toByteString()
                 ).build());
             break;
+          case TRUNCATE:
+            Event.TruncateEvent te = (Event.TruncateEvent) e;
+            events.add(InotifyProtos.EventProto.newBuilder()
+                .setType(InotifyProtos.EventType.EVENT_TRUNCATE)
+                .setContents(
+                    InotifyProtos.TruncateEventProto.newBuilder()
+                        .setPath(te.getPath())
+                        .setFileSize(te.getFileSize())
+                        .setTimestamp(te.getTimestamp()).build().toByteString()
+                ).build());
+            break;
           default:
             throw new RuntimeException("Unexpected inotify event: " + e);
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/979c9ca2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/InotifyFSEditLogOpTranslator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/InotifyFSEditLogOpTranslator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/InotifyFSEditLogOpTranslator.java
index 5345b46..0918107 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/InotifyFSEditLogOpTranslator.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/InotifyFSEditLogOpTranslator.java
@@ -176,6 +176,10 @@ public class InotifyFSEditLogOpTranslator {
           .metadataType(Event.MetadataUpdateEvent.MetadataType.ACLS)
           .path(saOp.src)
           .acls(saOp.aclEntries).build() });
+    case OP_TRUNCATE:
+      FSEditLogOp.TruncateOp tOp = (FSEditLogOp.TruncateOp) op;
+      return new EventBatch(op.txid, new Event[] {
+          new Event.TruncateEvent(tOp.src, tOp.newLength, tOp.timestamp) });
     default:
       return null;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/979c9ca2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
index ba33bd3..385d653 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
@@ -102,6 +102,8 @@ public class TestDFSInotifyEventInputStream {
       DFSTestUtil.createFile(fs, new Path("/file"), BLOCK_SIZE, (short) 1, 0L);
       DFSTestUtil.createFile(fs, new Path("/file3"), BLOCK_SIZE, (short) 1, 0L);
       DFSTestUtil.createFile(fs, new Path("/file5"), BLOCK_SIZE, (short) 1, 0L);
+      DFSTestUtil.createFile(fs, new Path("/truncate_file"),
+          BLOCK_SIZE * 2, (short) 1, 0L);
       DFSInotifyEventInputStream eis = client.getInotifyEventStream();
       client.rename("/file", "/file4", null); // RenameOp -> RenameEvent
       client.rename("/file4", "/file2"); // RenameOldOp -> RenameEvent
@@ -136,7 +138,8 @@ public class TestDFSInotifyEventInputStream {
           "user::rwx,user:foo:rw-,group::r--,other::---", true));
       client.removeAcl("/file5"); // SetAclOp -> MetadataUpdateEvent
       client.rename("/file5", "/dir"); // RenameOldOp -> RenameEvent
-
+      //TruncateOp -> TruncateEvent
+      client.truncate("/truncate_file", BLOCK_SIZE);
       EventBatch batch = null;
 
       // RenameOp
@@ -354,6 +357,18 @@ public class TestDFSInotifyEventInputStream {
       Assert.assertTrue(re3.getSrcPath().equals("/file5"));
       Assert.assertTrue(re.getTimestamp() > 0);
 
+      // TruncateOp
+      batch = waitForNextEvents(eis);
+      Assert.assertEquals(1, batch.getEvents().length);
+      txid = checkTxid(batch, txid);
+      Assert
+          .assertTrue(batch.getEvents()[0].getEventType() ==
+          Event.EventType.TRUNCATE);
+      Event.TruncateEvent et = ((Event.TruncateEvent) batch.getEvents()[0]);
+      Assert.assertTrue(et.getPath().equals("/truncate_file"));
+      Assert.assertTrue(et.getFileSize() == BLOCK_SIZE);
+      Assert.assertTrue(et.getTimestamp() > 0);
+
       // Returns null when there are no further events
       Assert.assertTrue(eis.poll() == null);
 


[34/50] hadoop git commit: YARN-3849. Too much of preemption activity causing continuos killing of containers across queues. (Sunil G via wangda)

Posted by ar...@apache.org.
YARN-3849. Too much of preemption activity causing continuos killing of containers across queues. (Sunil G via wangda)


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

Branch: refs/heads/HDFS-7240
Commit: 1df39c1efc9ed26d3f1a5887c31c38c873e0b784
Parents: 1ea3629
Author: Wangda Tan <wa...@apache.org>
Authored: Sat Jul 11 10:26:46 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Sat Jul 11 10:26:46 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../ProportionalCapacityPreemptionPolicy.java   |   4 +-
 ...estProportionalCapacityPreemptionPolicy.java | 253 ++++++++++++++-----
 ...pacityPreemptionPolicyForNodePartitions.java | 114 +++++++--
 4 files changed, 289 insertions(+), 85 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1df39c1e/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index f78bbfa..1365747 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -613,6 +613,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3888. ApplicationMaster link is broken in RM WebUI when appstate is NEW.
     (Bibin A Chundatt via xgong)
 
+    YARN-3849. Too much of preemption activity causing continuos killing of
+    containers across queues. (Sunil G via wangda)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1df39c1e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.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/monitor/capacity/ProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
index 5a20a6f..6e661d4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
@@ -840,12 +840,12 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     synchronized (curQueue) {
       String queueName = curQueue.getQueueName();
       QueueCapacities qc = curQueue.getQueueCapacities();
-      float absUsed = qc.getAbsoluteUsedCapacity(partitionToLookAt);
       float absCap = qc.getAbsoluteCapacity(partitionToLookAt);
       float absMaxCap = qc.getAbsoluteMaximumCapacity(partitionToLookAt);
       boolean preemptionDisabled = curQueue.getPreemptionDisabled();
 
-      Resource current = Resources.multiply(partitionResource, absUsed);
+      Resource current = curQueue.getQueueResourceUsage().getUsed(
+          partitionToLookAt);
       Resource guaranteed = Resources.multiply(partitionResource, absCap);
       Resource maxCapacity = Resources.multiply(partitionResource, absMaxCap);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1df39c1e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.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/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
index 2c0c6d7..3057360 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
@@ -81,7 +81,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEv
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -372,7 +374,7 @@ public class TestProportionalCapacityPreemptionPolicy {
             appA.getApplicationId(), appA.getAttemptId());
     assertTrue("appA should be running on queueB",
         mCS.getAppsInQueue("queueB").contains(expectedAttemptOnQueueB));
-    verify(mDisp, times(9)).handle(argThat(new IsPreemptionRequestFor(appA)));
+    verify(mDisp, times(10)).handle(argThat(new IsPreemptionRequestFor(appA)));
 
     // Need to call setup() again to reset mDisp
     setup();
@@ -395,7 +397,7 @@ public class TestProportionalCapacityPreemptionPolicy {
     // Resources should have come from queueE (appC) and neither of queueA's
     // children.
     verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appA)));
-    verify(mDisp, times(9)).handle(argThat(new IsPreemptionRequestFor(appC)));
+    verify(mDisp, times(10)).handle(argThat(new IsPreemptionRequestFor(appC)));
   }
 
   @Test
@@ -470,8 +472,8 @@ public class TestProportionalCapacityPreemptionPolicy {
     // With all queues preemptable, resources should be taken from queueC(appA)
     // and queueD(appB). Resources taken more from queueD(appB) than
     // queueC(appA) because it's over its capacity by a larger percentage.
-    verify(mDisp, times(16)).handle(argThat(new IsPreemptionRequestFor(appA)));
-    verify(mDisp, times(182)).handle(argThat(new IsPreemptionRequestFor(appB)));
+    verify(mDisp, times(17)).handle(argThat(new IsPreemptionRequestFor(appA)));
+    verify(mDisp, times(183)).handle(argThat(new IsPreemptionRequestFor(appB)));
 
     // Turn off preemption for queueA and it's children. queueF(appC)'s request
     // should starve.
@@ -635,7 +637,7 @@ public class TestProportionalCapacityPreemptionPolicy {
     policy.editSchedule();
     // verify capacity taken from A1, not B1 despite B1 being far over
     // its absolute guaranteed capacity
-    verify(mDisp, times(9)).handle(argThat(new IsPreemptionRequestFor(appA)));
+    verify(mDisp, times(10)).handle(argThat(new IsPreemptionRequestFor(appA)));
   }
 
   @Test
@@ -676,7 +678,7 @@ public class TestProportionalCapacityPreemptionPolicy {
     // we verify both that C has priority on B and D (has it has >0 guarantees)
     // and that B and D are force to share their over capacity fairly (as they
     // are both zero-guarantees) hence D sees some of its containers preempted
-    verify(mDisp, times(14)).handle(argThat(new IsPreemptionRequestFor(appC)));
+    verify(mDisp, times(15)).handle(argThat(new IsPreemptionRequestFor(appC)));
   }
   
   
@@ -703,8 +705,8 @@ public class TestProportionalCapacityPreemptionPolicy {
 
     // XXX note: compensating for rounding error in Resources.multiplyTo
     // which is likely triggered since we use small numbers for readability
-    verify(mDisp, times(7)).handle(argThat(new IsPreemptionRequestFor(appA)));
-    verify(mDisp, times(5)).handle(argThat(new IsPreemptionRequestFor(appE)));
+    verify(mDisp, times(9)).handle(argThat(new IsPreemptionRequestFor(appA)));
+    verify(mDisp, times(6)).handle(argThat(new IsPreemptionRequestFor(appE)));
   }
 
   @Test
@@ -868,6 +870,34 @@ public class TestProportionalCapacityPreemptionPolicy {
     setAMContainer = false;
   }
 
+  @Test
+  public void testPreemptionWithVCoreResource() {
+    int[][] qData = new int[][]{
+        // / A B
+        {100, 100, 100}, // maxcap
+        {5, 1, 1}, // apps
+        {2, 0, 0}, // subqueues
+    };
+
+    // Resources can be set like memory:vcores
+    String[][] resData = new String[][]{
+        // / A B
+        {"100:100", "50:50", "50:50"}, // abs
+        {"10:100", "10:100", "0"}, // used
+        {"70:20", "70:20", "10:100"}, // pending
+        {"0", "0", "0"}, // reserved
+        {"-1", "1:10", "1:10"}, // req granularity
+    };
+
+    // Passing last param as TRUE to use DominantResourceCalculator
+    ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData, resData,
+        true);
+    policy.editSchedule();
+
+    // 5 containers will be preempted here
+    verify(mDisp, times(5)).handle(argThat(new IsPreemptionRequestFor(appA)));
+  }
+
   static class IsPreemptionRequestFor
       extends ArgumentMatcher<ContainerPreemptEvent> {
     private final ApplicationAttemptId appAttId;
@@ -892,13 +922,40 @@ public class TestProportionalCapacityPreemptionPolicy {
   }
 
   ProportionalCapacityPreemptionPolicy buildPolicy(int[][] qData) {
-    ProportionalCapacityPreemptionPolicy policy =
-      new ProportionalCapacityPreemptionPolicy(conf, rmContext, mCS, mClock);
+    ProportionalCapacityPreemptionPolicy policy = new ProportionalCapacityPreemptionPolicy(
+        conf, rmContext, mCS, mClock);
+    clusterResources = Resource.newInstance(
+        leafAbsCapacities(qData[0], qData[7]), 0);
     ParentQueue mRoot = buildMockRootQueue(rand, qData);
     when(mCS.getRootQueue()).thenReturn(mRoot);
 
-    clusterResources =
-      Resource.newInstance(leafAbsCapacities(qData[0], qData[7]), 0);
+    setResourceAndNodeDetails();
+    return policy;
+  }
+
+  ProportionalCapacityPreemptionPolicy buildPolicy(int[][] qData,
+      String[][] resData) {
+    return buildPolicy(qData, resData, false);
+  }
+
+  ProportionalCapacityPreemptionPolicy buildPolicy(int[][] qData,
+      String[][] resData, boolean useDominantResourceCalculator) {
+    if (useDominantResourceCalculator) {
+      when(mCS.getResourceCalculator()).thenReturn(
+          new DominantResourceCalculator());
+    }
+    ProportionalCapacityPreemptionPolicy policy =
+        new ProportionalCapacityPreemptionPolicy(conf, rmContext, mCS, mClock);
+    clusterResources = leafAbsCapacities(parseResourceDetails(resData[0]),
+        qData[2]);
+    ParentQueue mRoot = buildMockRootQueue(rand, resData, qData);
+    when(mCS.getRootQueue()).thenReturn(mRoot);
+
+    setResourceAndNodeDetails();
+    return policy;
+  }
+
+  private void setResourceAndNodeDetails() {
     when(mCS.getClusterResource()).thenReturn(clusterResources);
     when(lm.getResourceByLabel(anyString(), any(Resource.class))).thenReturn(
         clusterResources);
@@ -906,35 +963,78 @@ public class TestProportionalCapacityPreemptionPolicy {
     SchedulerNode mNode = mock(SchedulerNode.class);
     when(mNode.getPartition()).thenReturn(RMNodeLabelsManager.NO_LABEL);
     when(mCS.getSchedulerNode(any(NodeId.class))).thenReturn(mNode);
-    return policy;
   }
 
   ParentQueue buildMockRootQueue(Random r, int[]... queueData) {
-    int[] abs      = queueData[0];
-    int[] maxCap   = queueData[1];
-    int[] used     = queueData[2];
-    int[] pending  = queueData[3];
-    int[] reserved = queueData[4];
-    int[] apps     = queueData[5];
-    int[] gran     = queueData[6];
-    int[] queues   = queueData[7];
-
-    return mockNested(abs, maxCap, used, pending,  reserved, apps, gran, queues);
+    Resource[] abs = generateResourceList(queueData[0]);
+    Resource[] used = generateResourceList(queueData[2]);
+    Resource[] pending = generateResourceList(queueData[3]);
+    Resource[] reserved = generateResourceList(queueData[4]);
+    Resource[] gran = generateResourceList(queueData[6]);
+    int[] maxCap = queueData[1];
+    int[] apps = queueData[5];
+    int[] queues = queueData[7];
+
+    return mockNested(abs, maxCap, used, pending, reserved, apps, gran, queues);
   }
 
-  ParentQueue mockNested(int[] abs, int[] maxCap, int[] used,
-      int[] pending, int[] reserved, int[] apps, int[] gran, int[] queues) {
-    float tot = leafAbsCapacities(abs, queues);
+  ParentQueue buildMockRootQueue(Random r, String[][] resData,
+      int[]... queueData) {
+    Resource[] abs = parseResourceDetails(resData[0]);
+    Resource[] used = parseResourceDetails(resData[1]);
+    Resource[] pending = parseResourceDetails(resData[2]);
+    Resource[] reserved = parseResourceDetails(resData[3]);
+    Resource[] gran = parseResourceDetails(resData[4]);
+    int[] maxCap = queueData[0];
+    int[] apps = queueData[1];
+    int[] queues = queueData[2];
+
+    return mockNested(abs, maxCap, used, pending, reserved, apps, gran, queues);
+  }
+
+  Resource[] parseResourceDetails(String[] resData) {
+    List<Resource> resourceList = new ArrayList<Resource>();
+    for (int i = 0; i < resData.length; i++) {
+      String[] resource = resData[i].split(":");
+      if (resource.length == 1) {
+        resourceList.add(Resource.newInstance(Integer.valueOf(resource[0]), 0));
+      } else {
+        resourceList.add(Resource.newInstance(Integer.valueOf(resource[0]),
+            Integer.valueOf(resource[1])));
+      }
+    }
+    return resourceList.toArray(new Resource[resourceList.size()]);
+  }
+
+  Resource[] generateResourceList(int[] qData) {
+    List<Resource> resourceList = new ArrayList<Resource>();
+    for (int i = 0; i < qData.length; i++) {
+      resourceList.add(Resource.newInstance(qData[i], 0));
+    }
+    return resourceList.toArray(new Resource[resourceList.size()]);
+  }
+
+  ParentQueue mockNested(Resource[] abs, int[] maxCap, Resource[] used,
+      Resource[] pending, Resource[] reserved, int[] apps, Resource[] gran,
+      int[] queues) {
+    ResourceCalculator rc = mCS.getResourceCalculator();
+    Resource tot = leafAbsCapacities(abs, queues);
     Deque<ParentQueue> pqs = new LinkedList<ParentQueue>();
     ParentQueue root = mockParentQueue(null, queues[0], pqs);
+    ResourceUsage resUsage = new ResourceUsage();
+    resUsage.setUsed(used[0]);
     when(root.getQueueName()).thenReturn(CapacitySchedulerConfiguration.ROOT);
-    when(root.getAbsoluteUsedCapacity()).thenReturn(used[0] / tot);
-    when(root.getAbsoluteCapacity()).thenReturn(abs[0] / tot);
-    when(root.getAbsoluteMaximumCapacity()).thenReturn(maxCap[0] / tot);
+    when(root.getAbsoluteUsedCapacity()).thenReturn(
+        Resources.divide(rc, tot, used[0], tot));
+    when(root.getAbsoluteCapacity()).thenReturn(
+        Resources.divide(rc, tot, abs[0], tot));
+    when(root.getAbsoluteMaximumCapacity()).thenReturn(
+        maxCap[0] / (float) tot.getMemory());
+    when(root.getQueueResourceUsage()).thenReturn(resUsage);
     QueueCapacities rootQc = new QueueCapacities(true);
-    rootQc.setAbsoluteUsedCapacity(used[0] / tot);
-    rootQc.setAbsoluteCapacity(abs[0] / tot);
-    rootQc.setAbsoluteMaximumCapacity(maxCap[0] / tot);
+    rootQc.setAbsoluteUsedCapacity(Resources.divide(rc, tot, used[0], tot));
+    rootQc.setAbsoluteCapacity(Resources.divide(rc, tot, abs[0], tot));
+    rootQc.setAbsoluteMaximumCapacity(maxCap[0] / (float) tot.getMemory());
     when(root.getQueueCapacities()).thenReturn(rootQc);
     when(root.getQueuePath()).thenReturn(CapacitySchedulerConfiguration.ROOT);
     boolean preemptionDisabled = mockPreemptionStatus("root");
@@ -943,28 +1043,35 @@ public class TestProportionalCapacityPreemptionPolicy {
     for (int i = 1; i < queues.length; ++i) {
       final CSQueue q;
       final ParentQueue p = pqs.removeLast();
-      final String queueName = "queue" + ((char)('A' + i - 1));
+      final String queueName = "queue" + ((char) ('A' + i - 1));
       if (queues[i] > 0) {
         q = mockParentQueue(p, queues[i], pqs);
+        ResourceUsage resUsagePerQueue = new ResourceUsage();
+        resUsagePerQueue.setUsed(used[i]);
+        when(q.getQueueResourceUsage()).thenReturn(resUsagePerQueue);
       } else {
         q = mockLeafQueue(p, tot, i, abs, used, pending, reserved, apps, gran);
       }
       when(q.getParent()).thenReturn(p);
       when(q.getQueueName()).thenReturn(queueName);
-      when(q.getAbsoluteUsedCapacity()).thenReturn(used[i] / tot);
-      when(q.getAbsoluteCapacity()).thenReturn(abs[i] / tot);
-      when(q.getAbsoluteMaximumCapacity()).thenReturn(maxCap[i] / tot);
+      when(q.getAbsoluteUsedCapacity()).thenReturn(
+          Resources.divide(rc, tot, used[i], tot));
+      when(q.getAbsoluteCapacity()).thenReturn(
+          Resources.divide(rc, tot, abs[i], tot));
+      when(q.getAbsoluteMaximumCapacity()).thenReturn(
+          maxCap[i] / (float) tot.getMemory());
 
       // We need to make these fields to QueueCapacities
       QueueCapacities qc = new QueueCapacities(false);
-      qc.setAbsoluteUsedCapacity(used[i] / tot);
-      qc.setAbsoluteCapacity(abs[i] / tot);
-      qc.setAbsoluteMaximumCapacity(maxCap[i] / tot);
+      qc.setAbsoluteUsedCapacity(Resources.divide(rc, tot, used[i], tot));
+      qc.setAbsoluteCapacity(Resources.divide(rc, tot, abs[i], tot));
+      qc.setAbsoluteMaximumCapacity(maxCap[i] / (float) tot.getMemory());
       when(q.getQueueCapacities()).thenReturn(qc);
 
       String parentPathName = p.getQueuePath();
       parentPathName = (parentPathName == null) ? "root" : parentPathName;
-      String queuePathName = (parentPathName+"."+queueName).replace("/","root");
+      String queuePathName = (parentPathName + "." + queueName).replace("/",
+          "root");
       when(q.getQueuePath()).thenReturn(queuePathName);
       preemptionDisabled = mockPreemptionStatus(queuePathName);
       when(q.getPreemptionDisabled()).thenReturn(preemptionDisabled);
@@ -1004,16 +1111,18 @@ public class TestProportionalCapacityPreemptionPolicy {
   }
 
   @SuppressWarnings("rawtypes")
-  LeafQueue mockLeafQueue(ParentQueue p, float tot, int i, int[] abs, 
-      int[] used, int[] pending, int[] reserved, int[] apps, int[] gran) {
+  LeafQueue mockLeafQueue(ParentQueue p, Resource tot, int i, Resource[] abs,
+      Resource[] used, Resource[] pending, Resource[] reserved, int[] apps,
+      Resource[] gran) {
     LeafQueue lq = mock(LeafQueue.class);
+    ResourceCalculator rc = mCS.getResourceCalculator();
     List<ApplicationAttemptId> appAttemptIdList = 
         new ArrayList<ApplicationAttemptId>();
-    when(lq.getTotalResourcePending()).thenReturn(
-        Resource.newInstance(pending[i], 0));
+    when(lq.getTotalResourcePending()).thenReturn(pending[i]);
     // need to set pending resource in resource usage as well
     ResourceUsage ru = new ResourceUsage();
-    ru.setPending(Resource.newInstance(pending[i], 0));
+    ru.setPending(pending[i]);
+    ru.setUsed(used[i]);
     when(lq.getQueueResourceUsage()).thenReturn(ru);
     // consider moving where CapacityScheduler::comparator accessible
     final NavigableSet<FiCaSchedulerApp> qApps = new TreeSet<FiCaSchedulerApp>(
@@ -1026,9 +1135,9 @@ public class TestProportionalCapacityPreemptionPolicy {
       });
     // applications are added in global L->R order in queues
     if (apps[i] != 0) {
-      int aUsed    = used[i] / apps[i];
-      int aPending = pending[i] / apps[i];
-      int aReserve = reserved[i] / apps[i];
+      Resource aUsed = Resources.divideAndCeil(rc, used[i], apps[i]);
+      Resource aPending = Resources.divideAndCeil(rc, pending[i], apps[i]);
+      Resource aReserve = Resources.divideAndCeil(rc, reserved[i], apps[i]);
       for (int a = 0; a < apps[i]; ++a) {
         FiCaSchedulerApp mockFiCaApp =
             mockApp(i, appAlloc, aUsed, aPending, aReserve, gran[i]);
@@ -1055,9 +1164,10 @@ public class TestProportionalCapacityPreemptionPolicy {
     return lq;
   }
 
-  FiCaSchedulerApp mockApp(int qid, int id, int used, int pending, int reserved,
-      int gran) {
+  FiCaSchedulerApp mockApp(int qid, int id, Resource used, Resource pending,
+      Resource reserved, Resource gran) {
     FiCaSchedulerApp app = mock(FiCaSchedulerApp.class);
+    ResourceCalculator rc = mCS.getResourceCalculator();
 
     ApplicationId appId = ApplicationId.newInstance(TS, id);
     ApplicationAttemptId appAttId = ApplicationAttemptId.newInstance(appId, 0);
@@ -1065,30 +1175,35 @@ public class TestProportionalCapacityPreemptionPolicy {
     when(app.getApplicationAttemptId()).thenReturn(appAttId);
 
     int cAlloc = 0;
-    Resource unit = Resource.newInstance(gran, 0);
+    Resource unit = gran;
     List<RMContainer> cReserved = new ArrayList<RMContainer>();
-    for (int i = 0; i < reserved; i += gran) {
-      cReserved.add(mockContainer(appAttId, cAlloc, unit, priority.CONTAINER
-          .getValue()));
+    Resource resIter = Resource.newInstance(0, 0);
+    for (; Resources.lessThan(rc, clusterResources, resIter, reserved); Resources
+        .addTo(resIter, gran)) {
+      cReserved.add(mockContainer(appAttId, cAlloc, unit,
+          priority.CONTAINER.getValue()));
       ++cAlloc;
     }
     when(app.getReservedContainers()).thenReturn(cReserved);
 
     List<RMContainer> cLive = new ArrayList<RMContainer>();
-    for (int i = 0; i < used; i += gran) {
-      if(setAMContainer && i == 0){
-        cLive.add(mockContainer(appAttId, cAlloc, unit, priority.AMCONTAINER
-            .getValue()));
-      }else if(setLabeledContainer && i ==1){
+    Resource usedIter = Resource.newInstance(0, 0);
+    int i = 0;
+    for (; Resources.lessThan(rc, clusterResources, usedIter, used); Resources
+        .addTo(usedIter, gran)) {
+      if (setAMContainer && i == 0) {
+        cLive.add(mockContainer(appAttId, cAlloc, unit,
+            priority.AMCONTAINER.getValue()));
+      } else if (setLabeledContainer && i == 1) {
         cLive.add(mockContainer(appAttId, cAlloc, unit,
             priority.LABELEDCONTAINER.getValue()));
-        ++used;
-      }
-      else{
-        cLive.add(mockContainer(appAttId, cAlloc, unit, priority.CONTAINER
-            .getValue()));
+        Resources.addTo(used, Resource.newInstance(1, 1));
+      } else {
+        cLive.add(mockContainer(appAttId, cAlloc, unit,
+            priority.CONTAINER.getValue()));
       }
       ++cAlloc;
+      ++i;
     }
     when(app.getLiveContainers()).thenReturn(cLive);
     return app;
@@ -1124,6 +1239,16 @@ public class TestProportionalCapacityPreemptionPolicy {
     return ret;
   }
 
+  static Resource leafAbsCapacities(Resource[] abs, int[] subqueues) {
+    Resource ret = Resource.newInstance(0, 0);
+    for (int i = 0; i < abs.length; ++i) {
+      if (0 == subqueues[i]) {
+        Resources.addTo(ret, abs[i]);
+      }
+    }
+    return ret;
+  }
+
   void printString(CSQueue nq, String indent) {
     if (nq instanceof ParentQueue) {
       System.out.println(indent + nq.getQueueName()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1df39c1e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.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/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
index 114769c..b3ac79b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
@@ -70,6 +70,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEv
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
@@ -771,6 +772,60 @@ public class TestProportionalCapacityPreemptionPolicyForNodePartitions {
         argThat(new IsPreemptionRequestFor(getAppAttemptId(2))));
   }
 
+  @Test
+  public void testNodePartitionPreemptionWithVCoreResource() throws IOException {
+    /**
+     * Queue structure is:
+     *
+     * <pre>
+     *       root
+     *       /  \
+     *      a    b
+     * </pre>
+     *
+     * Both a/b can access x, and guaranteed capacity of them is 50:50. Two
+     * nodes, n1 has 100 x, n2 has 100 NO_LABEL 4 applications in the cluster,
+     * app1/app2 in a, and app3/app4 in b. app1 uses 80 x, app2 uses 20
+     * NO_LABEL, app3 uses 20 x, app4 uses 80 NO_LABEL. Both a/b have 50 pending
+     * resource for x and NO_LABEL
+     *
+     * After preemption, it should preempt 30 from app1, and 30 from app4.
+     */
+    String labelsConfig = "=100:200,true;" + // default partition
+        "x=100:200,true"; // partition=x
+    String nodesConfig = "n1=x;" + // n1 has partition=x
+        "n2="; // n2 is default partition
+    String queuesConfig =
+    // guaranteed,max,used,pending
+    "root(=[100:200 100:200 100:200 100:200],x=[100:200 100:200 100:200 100:200]);"
+        + // root
+        "-a(=[50:100 100:200 20:40 50:100],x=[50:100 100:200 80:160 50:100]);" + // a
+        "-b(=[50:100 100:200 80:160 50:100],x=[50:100 100:200 20:40 50:100])"; // b
+    String appsConfig =
+    // queueName\t(priority,resource,host,expression,#repeat,reserved)
+    "a\t" // app1 in a
+        + "(1,1:2,n1,x,80,false);" + // 80 * x in n1
+        "a\t" // app2 in a
+        + "(1,1:2,n2,,20,false);" + // 20 default in n2
+        "b\t" // app3 in b
+        + "(1,1:2,n1,x,20,false);" + // 20 * x in n1
+        "b\t" // app4 in b
+        + "(1,1:2,n2,,80,false)"; // 80 default in n2
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig, true);
+    policy.editSchedule();
+
+    // 30 preempted from app1, 30 preempted from app4, and nothing preempted
+    // from app2/app3
+    verify(mDisp, times(30)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(1))));
+    verify(mDisp, times(30)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(4))));
+    verify(mDisp, never()).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(2))));
+    verify(mDisp, never()).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(3))));
+  }
 
   private ApplicationAttemptId getAppAttemptId(int id) {
     ApplicationId appId = ApplicationId.newInstance(0L, id);
@@ -821,6 +876,16 @@ public class TestProportionalCapacityPreemptionPolicyForNodePartitions {
 
   private void buildEnv(String labelsConfig, String nodesConfig,
       String queuesConfig, String appsConfig) throws IOException {
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig, false);
+  }
+
+  private void buildEnv(String labelsConfig, String nodesConfig,
+      String queuesConfig, String appsConfig,
+      boolean useDominantResourceCalculator) throws IOException {
+    if (useDominantResourceCalculator) {
+      when(cs.getResourceCalculator()).thenReturn(
+          new DominantResourceCalculator());
+    }
     mockNodeLabelsManager(labelsConfig);
     mockSchedulerNodes(nodesConfig);
     for (NodeId nodeId : nodeIdToSchedulerNodes.keySet()) {
@@ -832,7 +897,8 @@ public class TestProportionalCapacityPreemptionPolicyForNodePartitions {
     when(cs.getClusterResource()).thenReturn(clusterResource);
     mockApplications(appsConfig);
 
-    policy = new ProportionalCapacityPreemptionPolicy(conf, rmContext, cs, mClock);
+    policy = new ProportionalCapacityPreemptionPolicy(conf, rmContext, cs,
+        mClock);
   }
 
   private void mockContainers(String containersConfig, ApplicationAttemptId attemptId,
@@ -868,7 +934,7 @@ public class TestProportionalCapacityPreemptionPolicyForNodePartitions {
             + "(priority,resource,host,expression,repeat,reserved)");
       }
       Priority pri = Priority.newInstance(Integer.valueOf(values[0]));
-      Resource res = Resources.createResource(Integer.valueOf(values[1]));
+      Resource res = parseResourceFromString(values[1]);
       NodeId host = NodeId.newInstance(values[2], 1);
       String exp = values[3];
       int repeat = Integer.valueOf(values[4]);
@@ -1002,11 +1068,10 @@ public class TestProportionalCapacityPreemptionPolicyForNodePartitions {
     clusterResource = Resources.createResource(0);
     for (String p : partitionConfigArr) {
       String partitionName = p.substring(0, p.indexOf("="));
-      int totalResource =
-          Integer.valueOf(p.substring(p.indexOf("=") + 1, p.indexOf(",")));
-      boolean exclusivity =
+      Resource res = parseResourceFromString(p.substring(p.indexOf("=") + 1,
+          p.indexOf(",")));
+     boolean exclusivity =
           Boolean.valueOf(p.substring(p.indexOf(",") + 1, p.length()));
-      Resource res = Resources.createResource(totalResource);
       when(nlm.getResourceByLabel(eq(partitionName), any(Resource.class)))
           .thenReturn(res);
       when(nlm.isExclusiveNodeLabel(eq(partitionName))).thenReturn(exclusivity);
@@ -1022,6 +1087,18 @@ public class TestProportionalCapacityPreemptionPolicyForNodePartitions {
         partitionToResource.keySet());
   }
 
+  private Resource parseResourceFromString(String p) {
+    String[] resource = p.split(":");
+    Resource res = Resources.createResource(0);
+    if (resource.length == 1) {
+      res = Resources.createResource(Integer.valueOf(resource[0]));
+    } else {
+      res = Resources.createResource(Integer.valueOf(resource[0]),
+          Integer.valueOf(resource[1]));
+    }
+    return res;
+  }
+
   /**
    * Format is:
    * <pre>
@@ -1120,23 +1197,22 @@ public class TestProportionalCapacityPreemptionPolicyForNodePartitions {
       // Add a small epsilon to capacities to avoid truncate when doing
       // Resources.multiply
       float epsilon = 1e-6f;
-      float absGuaranteed =
-          Integer.valueOf(values[0].trim())
-              / (float) (partitionToResource.get(partitionName).getMemory())
-              + epsilon;
-      float absMax =
-          Integer.valueOf(values[1].trim())
-              / (float) (partitionToResource.get(partitionName).getMemory())
-              + epsilon;
-      float absUsed =
-          Integer.valueOf(values[2].trim())
-              / (float) (partitionToResource.get(partitionName).getMemory())
-              + epsilon;
-      Resource pending = Resources.createResource(Integer.valueOf(values[3].trim()));
+      Resource totResoucePerPartition = partitionToResource.get(partitionName);
+      float absGuaranteed = Resources.divide(rc, totResoucePerPartition,
+          parseResourceFromString(values[0].trim()), totResoucePerPartition)
+          + epsilon;
+      float absMax = Resources.divide(rc, totResoucePerPartition,
+          parseResourceFromString(values[1].trim()), totResoucePerPartition)
+          + epsilon;
+      float absUsed = Resources.divide(rc, totResoucePerPartition,
+          parseResourceFromString(values[2].trim()), totResoucePerPartition)
+          + epsilon;
+      Resource pending = parseResourceFromString(values[3].trim());
       qc.setAbsoluteCapacity(partitionName, absGuaranteed);
       qc.setAbsoluteMaximumCapacity(partitionName, absMax);
       qc.setAbsoluteUsedCapacity(partitionName, absUsed);
       ru.setPending(partitionName, pending);
+      ru.setUsed(partitionName, parseResourceFromString(values[2].trim()));
       LOG.debug("Setup queue=" + queueName + " partition=" + partitionName
           + " [abs_guaranteed=" + absGuaranteed + ",abs_max=" + absMax
           + ",abs_used" + absUsed + ",pending_resource=" + pending + "]");


[28/50] hadoop git commit: YARN-3888. ApplicationMaster link is broken in RM WebUI when appstate is NEW. Contributed by Bibin A Chundatt

Posted by ar...@apache.org.
YARN-3888. ApplicationMaster link is broken in RM WebUI when appstate is
NEW. Contributed by Bibin A Chundatt


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

Branch: refs/heads/HDFS-7240
Commit: 52148767924baf423172d26f2c6d8a4cfc6e143f
Parents: 1a0752d
Author: Xuan <xg...@apache.org>
Authored: Thu Jul 9 21:37:33 2015 -0700
Committer: Xuan <xg...@apache.org>
Committed: Thu Jul 9 21:37:33 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                           |  3 +++
 .../yarn/server/resourcemanager/webapp/RMAppsBlock.java   | 10 ++++++----
 2 files changed, 9 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/52148767/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 89b5e9f..2a9ff98 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -607,6 +607,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3892. Fixed NPE on RMStateStore#serviceStop when
     CapacityScheduler#serviceInit fails. (Bibin A Chundatt via jianhe)
 
+    YARN-3888. ApplicationMaster link is broken in RM WebUI when appstate is NEW.
+    (Bibin A Chundatt via xgong)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52148767/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
index d252c30..5e80d23 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
@@ -131,13 +131,15 @@ public class RMAppsBlock extends AppsBlock {
 
       String trackingURL =
           app.getTrackingUrl() == null
-              || app.getTrackingUrl().equals(UNAVAILABLE) ? null : app
-            .getTrackingUrl();
+              || app.getTrackingUrl().equals(UNAVAILABLE)
+              || app.getAppState() == YarnApplicationState.NEW ? null : app
+              .getTrackingUrl();
 
       String trackingUI =
           app.getTrackingUrl() == null
-              || app.getTrackingUrl().equals(UNAVAILABLE) ? "Unassigned" : app
-            .getAppState() == YarnApplicationState.FINISHED
+              || app.getTrackingUrl().equals(UNAVAILABLE)
+              || app.getAppState() == YarnApplicationState.NEW ? "Unassigned"
+              : app.getAppState() == YarnApplicationState.FINISHED
               || app.getAppState() == YarnApplicationState.FAILED
               || app.getAppState() == YarnApplicationState.KILLED ? "History"
               : "ApplicationMaster";


[41/50] hadoop git commit: Add HDFS-8143 to CHANGES.txt.

Posted by ar...@apache.org.
Add HDFS-8143 to CHANGES.txt.


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

Branch: refs/heads/HDFS-7240
Commit: f7c8311e9836ad1a1a2ef6eca8b42fd61a688164
Parents: 2466460
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jul 13 14:59:45 2015 -0700
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Mon Jul 13 14:59:45 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7c8311e/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 4fa566d..1491990 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1081,6 +1081,9 @@ Release 2.7.1 - 2015-07-06
 
     HDFS-7164. Feature documentation for HDFS-6581. (Arpit Agarwal)
 
+    HDFS-8143. Mover should exit after some retry when failed to move blocks.
+    (Surendra Singh Lilhore via szetszwo)
+
   OPTIMIZATIONS
     HDFS-8480. Fix performance and timeout issues in HDFS-7929 by using
     hard-links to preserve old edit logs, instead of copying them. (Zhe Zhang


[25/50] hadoop git commit: HDFS-8729. Fix TestFileTruncate#testTruncateWithDataNodesRestartImmediately which occasionally failed. Contributed by Walter Su.

Posted by ar...@apache.org.
HDFS-8729. Fix TestFileTruncate#testTruncateWithDataNodesRestartImmediately which occasionally failed. Contributed by Walter Su.


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

Branch: refs/heads/HDFS-7240
Commit: f4ca530c1cc9ece25c5ef01f99a94eb9e678e890
Parents: ac60483
Author: Jing Zhao <ji...@apache.org>
Authored: Thu Jul 9 13:17:52 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Thu Jul 9 13:17:52 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4ca530c/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 f595751..e26e061 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1023,6 +1023,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8642. Make TestFileTruncate more reliable. (Rakesh R via
     Arpit Agarwal)
 
+    HDFS-8729. Fix TestFileTruncate#testTruncateWithDataNodesRestartImmediately
+    which occasionally failed. (Walter Su via jing9)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4ca530c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index a91d6c9..8e54edc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -804,6 +804,9 @@ public class TestFileTruncate {
     assertEquals(newBlock.getBlock().getGenerationStamp(),
         oldBlock.getBlock().getGenerationStamp() + 1);
 
+    Thread.sleep(2000);
+    // trigger the second time BR to delete the corrupted replica if there's one
+    cluster.triggerBlockReports();
     // Wait replicas come to 3
     DFSTestUtil.waitReplication(fs, p, REPLICATION);
     // Old replica is disregarded and replaced with the truncated one on dn0


[43/50] hadoop git commit: HADOOP-12211. Collect disk usage on the node. Contributed by Robert Grandl

Posted by ar...@apache.org.
HADOOP-12211. Collect disk usage on the node. Contributed by Robert Grandl


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

Branch: refs/heads/HDFS-7240
Commit: a431ed9075cf6f467be5ff10f4ffb131cb1d3216
Parents: 9ef03a4
Author: Chris Douglas <cd...@apache.org>
Authored: Mon Jul 13 15:36:11 2015 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Mon Jul 13 15:36:11 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   2 +
 .../java/org/apache/hadoop/util/SysInfo.java    |  14 ++
 .../org/apache/hadoop/util/SysInfoLinux.java    | 157 ++++++++++++++++++-
 .../org/apache/hadoop/util/SysInfoWindows.java  |  12 ++
 .../apache/hadoop/util/TestSysInfoLinux.java    |  77 ++++++++-
 .../gridmix/DummyResourceCalculatorPlugin.java  |  17 ++
 .../yarn/util/ResourceCalculatorPlugin.java     |  18 +++
 7 files changed, 293 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a431ed90/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 3d4f1e4..3121430 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -695,6 +695,8 @@ Release 2.8.0 - UNRELEASED
 
     HADOOP-12210. Collect network usage on the node (Robert Grandl via cdouglas)
 
+    HADOOP-12211. Collect disk usage on the node (Robert Grandl via cdouglas)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a431ed90/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfo.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfo.java
index 24b339d..b75a8d8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfo.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfo.java
@@ -120,4 +120,18 @@ public abstract class SysInfo {
    */
   public abstract long getNetworkBytesWritten();
 
+  /**
+   * Obtain the aggregated number of bytes read from disks.
+   *
+   * @return total number of bytes read.
+   */
+  public abstract long getStorageBytesRead();
+
+  /**
+   * Obtain the aggregated number of bytes written to disks.
+   *
+   * @return total number of bytes written.
+   */
+  public abstract long getStorageBytesWritten();
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a431ed90/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoLinux.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoLinux.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoLinux.java
index 8801985..6b21f18 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoLinux.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoLinux.java
@@ -25,6 +25,7 @@ import java.io.InputStreamReader;
 import java.io.IOException;
 import java.math.BigInteger;
 import java.nio.charset.Charset;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -94,11 +95,27 @@ public class SysInfoLinux extends SysInfo {
                "[ \t]*([0-9]+)[ \t]*([0-9]+)[ \t]*([0-9]+)[ \t]*([0-9]+)" +
                "[ \t]*([0-9]+)[ \t]*([0-9]+)[ \t]*([0-9]+)[ \t]*([0-9]+).*");
 
+  /**
+   * Pattern for parsing /proc/diskstats.
+   */
+  private static final String PROCFS_DISKSFILE = "/proc/diskstats";
+  private static final Pattern PROCFS_DISKSFILE_FORMAT =
+      Pattern.compile("^[ \t]*([0-9]+)[ \t]*([0-9 ]+)" +
+              "(?!([a-zA-Z]+[0-9]+))([a-zA-Z]+)" +
+              "[ \t]*([0-9]+)[ \t]*([0-9]+)[ \t]*([0-9]+)[ \t]*([0-9]+)" +
+              "[ \t]*([0-9]+)[ \t]*([0-9]+)[ \t]*([0-9]+)[ \t]*([0-9]+)" +
+              "[ \t]*([0-9]+)[ \t]*([0-9]+)[ \t]*([0-9]+)");
+  /**
+   * Pattern for parsing /sys/block/partition_name/queue/hw_sector_size.
+   */
+  private static final Pattern PROCFS_DISKSECTORFILE_FORMAT =
+      Pattern.compile("^([0-9]+)");
 
   private String procfsMemFile;
   private String procfsCpuFile;
   private String procfsStatFile;
   private String procfsNetFile;
+  private String procfsDisksFile;
   private long jiffyLengthInMillis;
 
   private long ramSize = 0;
@@ -113,10 +130,15 @@ public class SysInfoLinux extends SysInfo {
   private long cpuFrequency = 0L; // CPU frequency on the system (kHz)
   private long numNetBytesRead = 0L; // aggregated bytes read from network
   private long numNetBytesWritten = 0L; // aggregated bytes written to network
+  private long numDisksBytesRead = 0L; // aggregated bytes read from disks
+  private long numDisksBytesWritten = 0L; // aggregated bytes written to disks
 
   private boolean readMemInfoFile = false;
   private boolean readCpuInfoFile = false;
 
+  /* map for every disk its sector size */
+  private HashMap<String, Integer> perDiskSectorSize = null;
+
   public static final long PAGE_SIZE = getConf("PAGESIZE");
   public static final long JIFFY_LENGTH_IN_MILLIS =
       Math.max(Math.round(1000D / getConf("CLK_TCK")), -1);
@@ -145,7 +167,7 @@ public class SysInfoLinux extends SysInfo {
 
   public SysInfoLinux() {
     this(PROCFS_MEMFILE, PROCFS_CPUINFO, PROCFS_STAT,
-         PROCFS_NETFILE, JIFFY_LENGTH_IN_MILLIS);
+         PROCFS_NETFILE, PROCFS_DISKSFILE, JIFFY_LENGTH_IN_MILLIS);
   }
 
   /**
@@ -155,6 +177,7 @@ public class SysInfoLinux extends SysInfo {
    * @param procfsCpuFile fake file for /proc/cpuinfo
    * @param procfsStatFile fake file for /proc/stat
    * @param procfsNetFile fake file for /proc/net/dev
+   * @param procfsDisksFile fake file for /proc/diskstats
    * @param jiffyLengthInMillis fake jiffy length value
    */
   @VisibleForTesting
@@ -162,13 +185,16 @@ public class SysInfoLinux extends SysInfo {
                                        String procfsCpuFile,
                                        String procfsStatFile,
                                        String procfsNetFile,
+                                       String procfsDisksFile,
                                        long jiffyLengthInMillis) {
     this.procfsMemFile = procfsMemFile;
     this.procfsCpuFile = procfsCpuFile;
     this.procfsStatFile = procfsStatFile;
     this.procfsNetFile = procfsNetFile;
+    this.procfsDisksFile = procfsDisksFile;
     this.jiffyLengthInMillis = jiffyLengthInMillis;
     this.cpuTimeTracker = new CpuTimeTracker(jiffyLengthInMillis);
+    this.perDiskSectorSize = new HashMap<String, Integer>();
   }
 
   /**
@@ -411,6 +437,119 @@ public class SysInfoLinux extends SysInfo {
     }
   }
 
+  /**
+   * Read /proc/diskstats file, parse and calculate amount
+   * of bytes read and written from/to disks.
+   */
+  private void readProcDisksInfoFile() {
+
+    numDisksBytesRead = 0L;
+    numDisksBytesWritten = 0L;
+
+    // Read "/proc/diskstats" file
+    BufferedReader in;
+    try {
+      in = new BufferedReader(new InputStreamReader(
+            new FileInputStream(procfsDisksFile), Charset.forName("UTF-8")));
+    } catch (FileNotFoundException f) {
+      return;
+    }
+
+    Matcher mat;
+    try {
+      String str = in.readLine();
+      while (str != null) {
+        mat = PROCFS_DISKSFILE_FORMAT.matcher(str);
+        if (mat.find()) {
+          String diskName = mat.group(4);
+          assert diskName != null;
+          // ignore loop or ram partitions
+          if (diskName.contains("loop") || diskName.contains("ram")) {
+            str = in.readLine();
+            continue;
+          }
+
+          Integer sectorSize;
+          synchronized (perDiskSectorSize) {
+            sectorSize = perDiskSectorSize.get(diskName);
+            if (null == sectorSize) {
+              // retrieve sectorSize
+              // if unavailable or error, assume 512
+              sectorSize = readDiskBlockInformation(diskName, 512);
+              perDiskSectorSize.put(diskName, sectorSize);
+            }
+          }
+
+          String sectorsRead = mat.group(7);
+          String sectorsWritten = mat.group(11);
+          if (null == sectorsRead || null == sectorsWritten) {
+            return;
+          }
+          numDisksBytesRead += Long.parseLong(sectorsRead) * sectorSize;
+          numDisksBytesWritten += Long.parseLong(sectorsWritten) * sectorSize;
+        }
+        str = in.readLine();
+      }
+    } catch (IOException e) {
+      LOG.warn("Error reading the stream " + procfsDisksFile, e);
+    } finally {
+      // Close the streams
+      try {
+        in.close();
+      } catch (IOException e) {
+        LOG.warn("Error closing the stream " + procfsDisksFile, e);
+      }
+    }
+  }
+
+  /**
+   * Read /sys/block/diskName/queue/hw_sector_size file, parse and calculate
+   * sector size for a specific disk.
+   * @return sector size of specified disk, or defSector
+   */
+  int readDiskBlockInformation(String diskName, int defSector) {
+
+    assert perDiskSectorSize != null && diskName != null;
+
+    String procfsDiskSectorFile =
+            "/sys/block/" + diskName + "/queue/hw_sector_size";
+
+    BufferedReader in;
+    try {
+      in = new BufferedReader(new InputStreamReader(
+            new FileInputStream(procfsDiskSectorFile),
+              Charset.forName("UTF-8")));
+    } catch (FileNotFoundException f) {
+      return defSector;
+    }
+
+    Matcher mat;
+    try {
+      String str = in.readLine();
+      while (str != null) {
+        mat = PROCFS_DISKSECTORFILE_FORMAT.matcher(str);
+        if (mat.find()) {
+          String secSize = mat.group(1);
+          if (secSize != null) {
+            return Integer.parseInt(secSize);
+          }
+        }
+        str = in.readLine();
+      }
+      return defSector;
+    } catch (IOException|NumberFormatException e) {
+      LOG.warn("Error reading the stream " + procfsDiskSectorFile, e);
+      return defSector;
+    } finally {
+      // Close the streams
+      try {
+        in.close();
+      } catch (IOException e) {
+        LOG.warn("Error closing the stream " + procfsDiskSectorFile, e);
+      }
+    }
+  }
+
   /** {@inheritDoc} */
   @Override
   public long getPhysicalMemorySize() {
@@ -492,6 +631,18 @@ public class SysInfoLinux extends SysInfo {
     return numNetBytesWritten;
   }
 
+  @Override
+  public long getStorageBytesRead() {
+    readProcDisksInfoFile();
+    return numDisksBytesRead;
+  }
+
+  @Override
+  public long getStorageBytesWritten() {
+    readProcDisksInfoFile();
+    return numDisksBytesWritten;
+  }
+
   /**
    * Test the {@link SysInfoLinux}.
    *
@@ -515,6 +666,10 @@ public class SysInfoLinux extends SysInfo {
             + plugin.getNetworkBytesRead());
     System.out.println("Total network written (bytes) : "
             + plugin.getNetworkBytesWritten());
+    System.out.println("Total storage read (bytes) : "
+            + plugin.getStorageBytesRead());
+    System.out.println("Total storage written (bytes) : "
+            + plugin.getStorageBytesWritten());
     try {
       // Sleep so we can compute the CPU usage
       Thread.sleep(500L);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a431ed90/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java
index f8542a3..f3fb364 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java
@@ -193,4 +193,16 @@ public class SysInfoWindows extends SysInfo {
     return 0L;
   }
 
+  @Override
+  public long getStorageBytesRead() {
+    // TODO unimplemented
+    return 0L;
+  }
+
+  @Override
+  public long getStorageBytesWritten() {
+    // TODO unimplemented
+    return 0L;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a431ed90/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoLinux.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoLinux.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoLinux.java
index 2a31f31..83f2e86 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoLinux.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoLinux.java
@@ -37,17 +37,18 @@ public class TestSysInfoLinux {
   /**
    * LinuxResourceCalculatorPlugin with a fake timer
    */
-  static class FakeLinuxResourceCalculatorPlugin extends
-      SysInfoLinux {
+  static class FakeLinuxResourceCalculatorPlugin extends SysInfoLinux {
+    static final int SECTORSIZE = 4096;
 
     long currentTime = 0;
     public FakeLinuxResourceCalculatorPlugin(String procfsMemFile,
                                              String procfsCpuFile,
                                              String procfsStatFile,
 			                                       String procfsNetFile,
+                                             String procfsDisksFile,
                                              long jiffyLengthInMillis) {
       super(procfsMemFile, procfsCpuFile, procfsStatFile, procfsNetFile,
-          jiffyLengthInMillis);
+          procfsDisksFile, jiffyLengthInMillis);
     }
     @Override
     long getCurrentTime() {
@@ -56,6 +57,10 @@ public class TestSysInfoLinux {
     public void advanceTime(long adv) {
       currentTime += adv * this.getJiffyLengthInMillis();
     }
+    @Override
+    int readDiskBlockInformation(String diskName, int defSector) {
+      return SECTORSIZE;
+    }
   }
   private static final FakeLinuxResourceCalculatorPlugin plugin;
   private static String TEST_ROOT_DIR = new Path(System.getProperty(
@@ -64,6 +69,7 @@ public class TestSysInfoLinux {
   private static final String FAKE_CPUFILE;
   private static final String FAKE_STATFILE;
   private static final String FAKE_NETFILE;
+  private static final String FAKE_DISKSFILE;
   private static final long FAKE_JIFFY_LENGTH = 10L;
   static {
     int randomNum = (new Random()).nextInt(1000000000);
@@ -71,9 +77,11 @@ public class TestSysInfoLinux {
     FAKE_CPUFILE = TEST_ROOT_DIR + File.separator + "CPUINFO_" + randomNum;
     FAKE_STATFILE = TEST_ROOT_DIR + File.separator + "STATINFO_" + randomNum;
     FAKE_NETFILE = TEST_ROOT_DIR + File.separator + "NETINFO_" + randomNum;
+    FAKE_DISKSFILE = TEST_ROOT_DIR + File.separator + "DISKSINFO_" + randomNum;
     plugin = new FakeLinuxResourceCalculatorPlugin(FAKE_MEMFILE, FAKE_CPUFILE,
                                                    FAKE_STATFILE,
                                                    FAKE_NETFILE,
+                                                   FAKE_DISKSFILE,
                                                    FAKE_JIFFY_LENGTH);
   }
   static final String MEMINFO_FORMAT =
@@ -157,6 +165,38 @@ public class TestSysInfoLinux {
     " eth1: %d 3152521    0    0    0     0          0    219781 %d 1866290    0    0    " +
     "0     0       0          0\n";
 
+  static final String DISKSINFO_FORMAT =
+      "1       0 ram0 0 0 0 0 0 0 0 0 0 0 0\n"+
+      "1       1 ram1 0 0 0 0 0 0 0 0 0 0 0\n"+
+      "1       2 ram2 0 0 0 0 0 0 0 0 0 0 0\n"+
+      "1       3 ram3 0 0 0 0 0 0 0 0 0 0 0\n"+
+      "1       4 ram4 0 0 0 0 0 0 0 0 0 0 0\n"+
+      "1       5 ram5 0 0 0 0 0 0 0 0 0 0 0\n"+
+      "1       6 ram6 0 0 0 0 0 0 0 0 0 0 0\n"+
+      "7       0 loop0 0 0 0 0 0 0 0 0 0 0 0\n"+
+      "7       1 loop1 0 0 0 0 0 0 0 0 0 0 0\n"+
+      "8       0 sda 82575678 2486518 %d 59876600 3225402 19761924 %d " +
+      "6407705 4 48803346 66227952\n"+
+      "8       1 sda1 732 289 21354 787 7 3 32 4 0 769 791"+
+      "8       2 sda2 744272 2206315 23605200 6742762 336830 2979630 " +
+      "26539520 1424776 4 1820130 8165444\n"+
+      "8       3 sda3 81830497 279914 17881852954 53132969 2888558 16782291 " +
+      "157367552 4982925 0 47077660 58061635\n"+
+      "8      32 sdc 10148118 693255 %d 122125461 6090515 401630172 %d 2696685590 " +
+      "0 26848216 2818793840\n"+
+      "8      33 sdc1 10147917 693230 2054138426 122125426 6090506 401630172 " +
+      "3261765880 2696685589 0 26848181 2818793804\n"+
+      "8      64 sde 9989771 553047 %d 93407551 5978572 391997273 %d 2388274325 " +
+      "0 24396646 2481664818\n"+
+      "8      65 sde1 9989570 553022 1943973346 93407489 5978563 391997273 3183807264 " +
+      "2388274325 0 24396584 2481666274\n"+
+      "8      80 sdf 10197163 693995 %d 144374395 6216644 408395438 %d 2669389056 0 " +
+      "26164759 2813746348\n"+
+      "8      81 sdf1 10196962 693970 2033452794 144374355 6216635 408395438 3316897064 " +
+      "2669389056 0 26164719 2813746308\n"+
+      "8     129 sdi1 10078602 657936 2056552626 108362198 6134036 403851153 3279882064 " +
+      "2639256086 0 26260432 2747601085\n";
+
   /**
    * Test parsing /proc/stat and /proc/cpuinfo
    * @throws IOException
@@ -358,4 +398,35 @@ public class TestSysInfoLinux {
     assertEquals(plugin.getNetworkBytesWritten(), numBytesWrittenIntf1 + numBytesWrittenIntf2);
   }
 
+  /**
+   * Test parsing /proc/diskstats
+   * @throws IOException
+   */
+  @Test
+  public void parsingProcDisksFile() throws IOException {
+    long numSectorsReadsda = 1790549L; long numSectorsWrittensda = 1839071L;
+    long numSectorsReadsdc = 20541402L; long numSectorsWrittensdc = 32617658L;
+    long numSectorsReadsde = 19439751L; long numSectorsWrittensde = 31838072L;
+    long numSectorsReadsdf = 20334546L; long numSectorsWrittensdf = 33168970L;
+    File tempFile = new File(FAKE_DISKSFILE);
+    tempFile.deleteOnExit();
+    FileWriter fWriter = new FileWriter(FAKE_DISKSFILE);
+    fWriter.write(String.format(DISKSINFO_FORMAT,
+             numSectorsReadsda, numSectorsWrittensda,
+             numSectorsReadsdc, numSectorsWrittensdc,
+             numSectorsReadsde, numSectorsWrittensde,
+             numSectorsReadsdf, numSectorsWrittensdf));
+
+    fWriter.close();
+    long expectedNumSectorsRead = numSectorsReadsda + numSectorsReadsdc +
+                                  numSectorsReadsde + numSectorsReadsdf;
+    long expectedNumSectorsWritten = numSectorsWrittensda + numSectorsWrittensdc +
+                                     numSectorsWrittensde + numSectorsWrittensdf;
+    // use non-default sector size
+    int diskSectorSize = FakeLinuxResourceCalculatorPlugin.SECTORSIZE;
+    assertEquals(expectedNumSectorsRead * diskSectorSize,
+        plugin.getStorageBytesRead());
+    assertEquals(expectedNumSectorsWritten * diskSectorSize,
+        plugin.getStorageBytesWritten());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a431ed90/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/DummyResourceCalculatorPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/DummyResourceCalculatorPlugin.java b/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/DummyResourceCalculatorPlugin.java
index b86303b..4999f14 100644
--- a/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/DummyResourceCalculatorPlugin.java
+++ b/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/DummyResourceCalculatorPlugin.java
@@ -54,6 +54,12 @@ public class DummyResourceCalculatorPlugin extends ResourceCalculatorPlugin {
   /** cumulative number of bytes written over the network */
   public static final String NETWORK_BYTES_WRITTEN =
       "mapred.tasktracker.networkwritten.testing";
+  /** cumulative number of bytes read from disks */
+  public static final String STORAGE_BYTES_READ =
+      "mapred.tasktracker.storageread.testing";
+  /** cumulative number of bytes written to disks */
+  public static final String STORAGE_BYTES_WRITTEN =
+      "mapred.tasktracker.storagewritten.testing";
   /** process cumulative CPU usage time for testing */
   public static final String PROC_CUMULATIVE_CPU_TIME =
       "mapred.tasktracker.proccumulativecputime.testing";
@@ -130,4 +136,15 @@ public class DummyResourceCalculatorPlugin extends ResourceCalculatorPlugin {
     return getConf().getLong(NETWORK_BYTES_WRITTEN, -1);
   }
 
+  /** {@inheritDoc} */
+  @Override
+  public long getStorageBytesRead() {
+    return getConf().getLong(STORAGE_BYTES_READ, -1);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getStorageBytesWritten() {
+    return getConf().getLong(STORAGE_BYTES_WRITTEN, -1);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a431ed90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
index 3af4aee..691c4ac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
@@ -145,6 +145,24 @@ public class ResourceCalculatorPlugin extends Configured {
   }
 
   /**
+   * Obtain the aggregated number of bytes read from disks.
+   *
+   * @return total number of bytes read.
+   */
+  public long getStorageBytesRead() {
+    return sys.getStorageBytesRead();
+  }
+
+  /**
+   * Obtain the aggregated number of bytes written to disks.
+   *
+   * @return total number of bytes written.
+   */
+  public long getStorageBytesWritten() {
+    return sys.getStorageBytesWritten();
+  }
+
+  /**
    * Create the ResourceCalculatorPlugin from the class name and configure it. If
    * class name is null, this method will try and return a memory calculator
    * plugin available for this system.


[02/50] hadoop git commit: Revert "HDFS-8652. Track BlockInfo instead of Block in CorruptReplicasMap. Contributed by Jing Zhao."

Posted by ar...@apache.org.
Revert "HDFS-8652. Track BlockInfo instead of Block in CorruptReplicasMap. Contributed by Jing Zhao."

This reverts commit d62b63d297bff12d93de560dd50ddd48743b851d.


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

Branch: refs/heads/HDFS-7240
Commit: bc99aaffe7b0ed13b1efc37b6a32cdbd344c2d75
Parents: 559425d
Author: Jing Zhao <ji...@apache.org>
Authored: Tue Jul 7 10:08:30 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Tue Jul 7 10:13:41 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 -
 .../hdfs/server/blockmanagement/BlockInfo.java  |   7 +-
 .../blockmanagement/BlockInfoContiguous.java    |   9 +-
 .../BlockInfoUnderConstruction.java             |  22 +--
 .../BlockInfoUnderConstructionContiguous.java   |  13 +-
 .../server/blockmanagement/BlockManager.java    | 143 ++++++++++---------
 .../hdfs/server/blockmanagement/BlocksMap.java  |   4 +-
 .../ContiguousBlockStorageOp.java               |   7 +-
 .../blockmanagement/CorruptReplicasMap.java     |  62 ++++----
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |   6 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |   2 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |  12 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |   6 +-
 .../blockmanagement/BlockManagerTestUtil.java   |   7 +-
 .../server/blockmanagement/TestBlockInfo.java   |  10 +-
 .../blockmanagement/TestBlockManager.java       |  10 +-
 .../blockmanagement/TestCorruptReplicaInfo.java |  15 +-
 17 files changed, 168 insertions(+), 169 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc99aaff/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 e40ea3d..7294cab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -701,8 +701,6 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8709. Clarify automatic sync in FSEditLog#logEdit. (wang)
 
-    HDFS-8652. Track BlockInfo instead of Block in CorruptReplicasMap. (jing9)
-
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc99aaff/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 4df2f0e..5ad992b 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
@@ -179,7 +179,7 @@ public abstract class  BlockInfo extends Block
    *                      information indicating the index of the block in the
    *                      corresponding block group.
    */
-  abstract void addStorage(DatanodeStorageInfo storage, Block reportedBlock);
+  abstract boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock);
 
   /**
    * Remove {@link DatanodeStorageInfo} location for a block
@@ -193,11 +193,6 @@ public abstract class  BlockInfo extends Block
   abstract void replaceBlock(BlockInfo newBlock);
 
   /**
-   * @return true if there is no storage storing the block
-   */
-  abstract boolean hasEmptyStorage();
-
-  /**
    * Find specified DatanodeStorageInfo.
    * @return DatanodeStorageInfo or null if not found.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc99aaff/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 561faca..de64ad8 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
@@ -45,8 +45,8 @@ public class BlockInfoContiguous extends BlockInfo {
   }
 
   @Override
-  void addStorage(DatanodeStorageInfo storage, Block reportedBlock) {
-    ContiguousBlockStorageOp.addStorage(this, storage);
+  boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock) {
+    return ContiguousBlockStorageOp.addStorage(this, storage);
   }
 
   @Override
@@ -73,9 +73,4 @@ public class BlockInfoContiguous extends BlockInfo {
     ucBlock.setBlockCollection(getBlockCollection());
     return ucBlock;
   }
-
-  @Override
-  boolean hasEmptyStorage() {
-    return ContiguousBlockStorageOp.hasEmptyStorage(this);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc99aaff/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
index 7924709..9cd3987 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
@@ -273,17 +274,18 @@ public abstract class BlockInfoUnderConstruction extends BlockInfo {
           "No blocks found, lease removed.");
     }
     boolean allLiveReplicasTriedAsPrimary = true;
-    for (ReplicaUnderConstruction replica : replicas) {
+    for (int i = 0; i < replicas.size(); i++) {
       // Check if all replicas have been tried or not.
-      if (replica.isAlive()) {
-        allLiveReplicasTriedAsPrimary = allLiveReplicasTriedAsPrimary
-            && replica.getChosenAsPrimary();
+      if (replicas.get(i).isAlive()) {
+        allLiveReplicasTriedAsPrimary =
+            (allLiveReplicasTriedAsPrimary &&
+                replicas.get(i).getChosenAsPrimary());
       }
     }
     if (allLiveReplicasTriedAsPrimary) {
       // Just set all the replicas to be chosen whether they are alive or not.
-      for (ReplicaUnderConstruction replica : replicas) {
-        replica.setChosenAsPrimary(false);
+      for (int i = 0; i < replicas.size(); i++) {
+        replicas.get(i).setChosenAsPrimary(false);
       }
     }
     long mostRecentLastUpdate = 0;
@@ -343,6 +345,10 @@ public abstract class BlockInfoUnderConstruction extends BlockInfo {
    * Convert an under construction block to a complete block.
    *
    * @return 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.
    */
   public abstract BlockInfo convertToCompleteBlock();
 
@@ -380,8 +386,8 @@ public abstract class BlockInfoUnderConstruction extends BlockInfo {
   }
 
   private void appendUCParts(StringBuilder sb) {
-    sb.append("{UCState=").append(blockUCState).append(", truncateBlock=")
-      .append(truncateBlock)
+    sb.append("{UCState=").append(blockUCState)
+      .append(", truncateBlock=" + truncateBlock)
       .append(", primaryNodeIndex=").append(primaryNodeIndex)
       .append(", replicas=[");
     if (replicas != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc99aaff/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java
index 963f247..d3cb337 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java
@@ -55,6 +55,10 @@ public class BlockInfoUnderConstructionContiguous extends
    * 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.
    */
   @Override
   public BlockInfoContiguous convertToCompleteBlock() {
@@ -65,8 +69,8 @@ public class BlockInfoUnderConstructionContiguous extends
   }
 
   @Override
-  void addStorage(DatanodeStorageInfo storage, Block reportedBlock) {
-    ContiguousBlockStorageOp.addStorage(this, storage);
+  boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock) {
+    return ContiguousBlockStorageOp.addStorage(this, storage);
   }
 
   @Override
@@ -85,11 +89,6 @@ public class BlockInfoUnderConstructionContiguous extends
   }
 
   @Override
-  boolean hasEmptyStorage() {
-    return ContiguousBlockStorageOp.hasEmptyStorage(this);
-  }
-
-  @Override
   public void setExpectedLocations(DatanodeStorageInfo[] targets) {
     int numLocations = targets == null ? 0 : targets.length;
     this.replicas = new ArrayList<>(numLocations);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc99aaff/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 6ae3ee2..0b60a97 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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
 import static org.apache.hadoop.util.ExitUtil.terminate;
 
 import java.io.IOException;
@@ -196,7 +197,7 @@ public class BlockManager implements BlockStatsMXBean {
    * notified of all block deletions that might have been pending
    * when the failover happened.
    */
-  private final Set<BlockInfo> postponedMisreplicatedBlocks = Sets.newHashSet();
+  private final Set<Block> postponedMisreplicatedBlocks = Sets.newHashSet();
 
   /**
    * Maps a StorageID to the set of blocks that are "extra" for this
@@ -337,7 +338,8 @@ public class BlockManager implements BlockStatsMXBean {
             DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_KEY,
             DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_DEFAULT);
     this.shouldCheckForEnoughRacks =
-        conf.get(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY) != null;
+        conf.get(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY) == null
+            ? false : true;
 
     this.blocksInvalidateWorkPct = DFSUtil.getInvalidateWorkPctPerIteration(conf);
     this.blocksReplWorkMultiplier = DFSUtil.getReplWorkMultiplier(conf);
@@ -463,7 +465,8 @@ public class BlockManager implements BlockStatsMXBean {
 
   /** Should the access keys be updated? */
   boolean shouldUpdateBlockKey(final long updateTime) throws IOException {
-    return isBlockTokenEnabled() && blockTokenSecretManager.updateKeys(updateTime);
+    return isBlockTokenEnabled()? blockTokenSecretManager.updateKeys(updateTime)
+        : false;
   }
 
   public void activate(Configuration conf) {
@@ -516,14 +519,14 @@ public class BlockManager implements BlockStatsMXBean {
     synchronized (neededReplications) {
       out.println("Metasave: Blocks waiting for replication: " + 
                   neededReplications.size());
-      for (BlockInfo block : neededReplications) {
+      for (Block block : neededReplications) {
         dumpBlockMeta(block, out);
       }
     }
     
     // Dump any postponed over-replicated blocks
     out.println("Mis-replicated blocks that have been postponed:");
-    for (BlockInfo block : postponedMisreplicatedBlocks) {
+    for (Block block : postponedMisreplicatedBlocks) {
       dumpBlockMeta(block, out);
     }
 
@@ -541,9 +544,11 @@ public class BlockManager implements BlockStatsMXBean {
    * Dump the metadata for the given block in a human-readable
    * form.
    */
-  private void dumpBlockMeta(BlockInfo block, PrintWriter out) {
-    List<DatanodeDescriptor> containingNodes = new ArrayList<>();
-    List<DatanodeStorageInfo> containingLiveReplicasNodes = new ArrayList<>();
+  private void dumpBlockMeta(Block block, PrintWriter out) {
+    List<DatanodeDescriptor> containingNodes =
+                                      new ArrayList<DatanodeDescriptor>();
+    List<DatanodeStorageInfo> containingLiveReplicasNodes =
+      new ArrayList<>();
 
     NumberReplicas numReplicas = new NumberReplicas();
     // source node returned is not used
@@ -551,16 +556,17 @@ public class BlockManager implements BlockStatsMXBean {
         containingLiveReplicasNodes, numReplicas,
         UnderReplicatedBlocks.LEVEL);
     
-    // containingLiveReplicasNodes can include READ_ONLY_SHARED replicas which
-    // are not included in the numReplicas.liveReplicas() count
+    // containingLiveReplicasNodes can include READ_ONLY_SHARED replicas which are 
+    // not included in the numReplicas.liveReplicas() count
     assert containingLiveReplicasNodes.size() >= numReplicas.liveReplicas();
     int usableReplicas = numReplicas.liveReplicas() +
                          numReplicas.decommissionedAndDecommissioning();
-
-    BlockCollection bc = block.getBlockCollection();
-    String fileName = (bc == null) ? "[orphaned]" : bc.getName();
-    out.print(fileName + ": ");
-
+    
+    if (block instanceof BlockInfo) {
+      BlockCollection bc = ((BlockInfo) block).getBlockCollection();
+      String fileName = (bc == null) ? "[orphaned]" : bc.getName();
+      out.print(fileName + ": ");
+    }
     // l: == live:, d: == decommissioned c: == corrupt e: == excess
     out.print(block + ((usableReplicas > 0)? "" : " MISSING") + 
               " (replicas:" +
@@ -569,8 +575,8 @@ public class BlockManager implements BlockStatsMXBean {
               " c: " + numReplicas.corruptReplicas() +
               " e: " + numReplicas.excessReplicas() + ") "); 
 
-    Collection<DatanodeDescriptor> corruptNodes =
-        corruptReplicas.getNodes(block);
+    Collection<DatanodeDescriptor> corruptNodes = 
+                                  corruptReplicas.getNodes(block);
     
     for (DatanodeStorageInfo storage : getStorages(block)) {
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
@@ -807,8 +813,7 @@ public class BlockManager implements BlockStatsMXBean {
       final long offset, final long length, final int nrBlocksToReturn,
       final AccessMode mode) throws IOException {
     int curBlk;
-    long curPos = 0;
-    long blkSize;
+    long curPos = 0, blkSize = 0;
     int nrBlocks = (blocks[0].getNumBytes() == 0) ? 0 : blocks.length;
     for (curBlk = 0; curBlk < nrBlocks; curBlk++) {
       blkSize = blocks[curBlk].getNumBytes();
@@ -1199,11 +1204,10 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
-   * Mark a replica (of a contiguous block) or an internal block (of a striped
-   * block group) as corrupt.
-   * @param b Indicating the reported bad block and the corresponding BlockInfo
-   *          stored in blocksMap.
+   * 
+   * @param b
    * @param storageInfo storage that contains the block, if known. null otherwise.
+   * @throws IOException
    */
   private void markBlockAsCorrupt(BlockToMarkCorrupt b,
       DatanodeStorageInfo storageInfo,
@@ -1224,7 +1228,7 @@ public class BlockManager implements BlockStatsMXBean {
     }
 
     // Add this replica to corruptReplicas Map
-    corruptReplicas.addToCorruptReplicasMap(b.stored, node, b.reason,
+    corruptReplicas.addToCorruptReplicasMap(b.corrupted, node, b.reason,
         b.reasonCode);
 
     NumberReplicas numberOfReplicas = countNodes(b.stored);
@@ -1246,7 +1250,7 @@ public class BlockManager implements BlockStatsMXBean {
     if (hasEnoughLiveReplicas || hasMoreCorruptReplicas
         || corruptedDuringWrite) {
       // the block is over-replicated so invalidate the replicas immediately
-      invalidateBlock(b, node, numberOfReplicas);
+      invalidateBlock(b, node);
     } else if (namesystem.isPopulatingReplQueues()) {
       // add the block to neededReplication
       updateNeededReplications(b.stored, -1, 0);
@@ -1254,15 +1258,12 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
-   * Invalidates the given block on the given datanode. Note that before this
-   * call we have already checked the current live replicas of the block and
-   * make sure it's safe to invalidate the replica.
-   *
-   * @return true if the replica was successfully invalidated and no longer
-   *         associated with the DataNode.
+   * Invalidates the given block on the given datanode.
+   * @return true if the block was successfully invalidated and no longer
+   * present in the BlocksMap
    */
-  private boolean invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn,
-      NumberReplicas nr) throws IOException {
+  private boolean invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn
+      ) throws IOException {
     blockLog.info("BLOCK* invalidateBlock: {} on {}", b, dn);
     DatanodeDescriptor node = getDatanodeManager().getDatanode(dn);
     if (node == null) {
@@ -1271,30 +1272,35 @@ public class BlockManager implements BlockStatsMXBean {
     }
 
     // Check how many copies we have of the block
+    NumberReplicas nr = countNodes(b.stored);
     if (nr.replicasOnStaleNodes() > 0) {
       blockLog.info("BLOCK* invalidateBlocks: postponing " +
           "invalidation of {} on {} because {} replica(s) are located on " +
           "nodes with potentially out-of-date block reports", b, dn,
           nr.replicasOnStaleNodes());
-      postponeBlock(b.stored);
+      postponeBlock(b.corrupted);
       return false;
-    } else {
-      // we already checked the number of replicas in the caller of this
-      // function and we know there is at least one copy on a live node, so we
-      // can delete it.
+    } else if (nr.liveReplicas() >= 1) {
+      // If we have at least one copy on a live node, then we can delete it.
       addToInvalidates(b.corrupted, dn);
       removeStoredBlock(b.stored, node);
       blockLog.debug("BLOCK* invalidateBlocks: {} on {} listed for deletion.",
           b, dn);
       return true;
+    } else {
+      blockLog.info("BLOCK* invalidateBlocks: {} on {} is the only copy and" +
+          " was not deleted", b, dn);
+      return false;
     }
   }
 
+
   public void setPostponeBlocksFromFuture(boolean postpone) {
     this.shouldPostponeBlocksFromFuture  = postpone;
   }
 
-  private void postponeBlock(BlockInfo blk) {
+
+  private void postponeBlock(Block blk) {
     if (postponedMisreplicatedBlocks.add(blk)) {
       postponedMisreplicatedBlocksCount.incrementAndGet();
     }
@@ -1368,7 +1374,7 @@ public class BlockManager implements BlockStatsMXBean {
     int requiredReplication, numEffectiveReplicas;
     List<DatanodeDescriptor> containingNodes;
     DatanodeDescriptor srcNode;
-    BlockCollection bc;
+    BlockCollection bc = null;
     int additionalReplRequired;
 
     int scheduledWork = 0;
@@ -1529,9 +1535,9 @@ public class BlockManager implements BlockStatsMXBean {
         DatanodeStorageInfo[] targets = rw.targets;
         if (targets != null && targets.length != 0) {
           StringBuilder targetList = new StringBuilder("datanode(s)");
-          for (DatanodeStorageInfo target : targets) {
+          for (int k = 0; k < targets.length; k++) {
             targetList.append(' ');
-            targetList.append(target.getDatanodeDescriptor());
+            targetList.append(targets[k].getDatanodeDescriptor());
           }
           blockLog.info("BLOCK* ask {} to replicate {} to {}", rw.srcNode,
               rw.block, targetList);
@@ -1608,8 +1614,8 @@ public class BlockManager implements BlockStatsMXBean {
     List<DatanodeDescriptor> datanodeDescriptors = null;
     if (nodes != null) {
       datanodeDescriptors = new ArrayList<>(nodes.size());
-      for (String nodeStr : nodes) {
-        DatanodeDescriptor node = datanodeManager.getDatanodeDescriptor(nodeStr);
+      for (int i = 0; i < nodes.size(); i++) {
+        DatanodeDescriptor node = datanodeManager.getDatanodeDescriptor(nodes.get(i));
         if (node != null) {
           datanodeDescriptors.add(node);
         }
@@ -1648,7 +1654,7 @@ public class BlockManager implements BlockStatsMXBean {
    *         the given block
    */
    @VisibleForTesting
-   DatanodeDescriptor chooseSourceDatanode(BlockInfo block,
+   DatanodeDescriptor chooseSourceDatanode(Block block,
        List<DatanodeDescriptor> containingNodes,
        List<DatanodeStorageInfo>  nodesContainingLiveReplicas,
        NumberReplicas numReplicas,
@@ -1728,16 +1734,16 @@ public class BlockManager implements BlockStatsMXBean {
     if (timedOutItems != null) {
       namesystem.writeLock();
       try {
-        for (BlockInfo timedOutItem : timedOutItems) {
+        for (int i = 0; i < timedOutItems.length; i++) {
           /*
            * Use the blockinfo from the blocksmap to be certain we're working
            * with the most up-to-date block information (e.g. genstamp).
            */
-          BlockInfo bi = getStoredBlock(timedOutItem);
+          BlockInfo bi = getStoredBlock(timedOutItems[i]);
           if (bi == null) {
             continue;
           }
-          NumberReplicas num = countNodes(timedOutItem);
+          NumberReplicas num = countNodes(timedOutItems[i]);
           if (isNeededReplication(bi, getReplication(bi), num.liveReplicas())) {
             neededReplications.add(bi, num.liveReplicas(),
                 num.decommissionedAndDecommissioning(), getReplication(bi));
@@ -1754,7 +1760,7 @@ public class BlockManager implements BlockStatsMXBean {
 
   public long requestBlockReportLeaseId(DatanodeRegistration nodeReg) {
     assert namesystem.hasReadLock();
-    DatanodeDescriptor node;
+    DatanodeDescriptor node = null;
     try {
       node = datanodeManager.getDatanode(nodeReg);
     } catch (UnregisteredNodeException e) {
@@ -2016,7 +2022,7 @@ public class BlockManager implements BlockStatsMXBean {
           startIndex += (base+1);
         }
       }
-      Iterator<BlockInfo> it = postponedMisreplicatedBlocks.iterator();
+      Iterator<Block> it = postponedMisreplicatedBlocks.iterator();
       for (int tmp = 0; tmp < startIndex; tmp++) {
         it.next();
       }
@@ -2111,7 +2117,7 @@ public class BlockManager implements BlockStatsMXBean {
       long oldGenerationStamp, long oldNumBytes,
       DatanodeStorageInfo[] newStorages) throws IOException {
     assert namesystem.hasWriteLock();
-    BlockToMarkCorrupt b;
+    BlockToMarkCorrupt b = null;
     if (block.getGenerationStamp() != oldGenerationStamp) {
       b = new BlockToMarkCorrupt(oldBlock, block, oldGenerationStamp,
           "genstamp does not match " + oldGenerationStamp
@@ -2713,7 +2719,7 @@ public class BlockManager implements BlockStatsMXBean {
           " but corrupt replicas map has " + corruptReplicasCount);
     }
     if ((corruptReplicasCount > 0) && (numLiveReplicas >= fileReplication)) {
-      invalidateCorruptReplicas(storedBlock, reportedBlock, num);
+      invalidateCorruptReplicas(storedBlock, reportedBlock);
     }
     return storedBlock;
   }
@@ -2746,20 +2752,18 @@ public class BlockManager implements BlockStatsMXBean {
    *
    * @param blk Block whose corrupt replicas need to be invalidated
    */
-  private void invalidateCorruptReplicas(BlockInfo blk, Block reported,
-      NumberReplicas numberReplicas) {
+  private void invalidateCorruptReplicas(BlockInfo blk, Block reported) {
     Collection<DatanodeDescriptor> nodes = corruptReplicas.getNodes(blk);
     boolean removedFromBlocksMap = true;
     if (nodes == null)
       return;
     // make a copy of the array of nodes in order to avoid
     // ConcurrentModificationException, when the block is removed from the node
-    DatanodeDescriptor[] nodesCopy = nodes.toArray(
-        new DatanodeDescriptor[nodes.size()]);
+    DatanodeDescriptor[] nodesCopy = nodes.toArray(new DatanodeDescriptor[0]);
     for (DatanodeDescriptor node : nodesCopy) {
       try {
         if (!invalidateBlock(new BlockToMarkCorrupt(reported, blk, null,
-            Reason.ANY), node, numberReplicas)) {
+            Reason.ANY), node)) {
           removedFromBlocksMap = false;
         }
       } catch (IOException e) {
@@ -2809,6 +2813,7 @@ public class BlockManager implements BlockStatsMXBean {
         replicationQueuesInitializer.join();
       } catch (final InterruptedException e) {
         LOG.warn("Interrupted while waiting for replicationQueueInitializer. Returning..");
+        return;
       } finally {
         replicationQueuesInitializer = null;
       }
@@ -3170,7 +3175,8 @@ public class BlockManager implements BlockStatsMXBean {
       CachedBlock cblock = namesystem.getCacheManager().getCachedBlocks()
           .get(new CachedBlock(storedBlock.getBlockId(), (short) 0, false));
       if (cblock != null) {
-        boolean removed = node.getPendingCached().remove(cblock);
+        boolean removed = false;
+        removed |= node.getPendingCached().remove(cblock);
         removed |= node.getCached().remove(cblock);
         removed |= node.getPendingUncached().remove(cblock);
         if (removed) {
@@ -3386,7 +3392,7 @@ public class BlockManager implements BlockStatsMXBean {
     int excess = 0;
     int stale = 0;
     Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b);
-    for (DatanodeStorageInfo storage : blocksMap.getStorages(b, State.NORMAL)) {
+    for(DatanodeStorageInfo storage : blocksMap.getStorages(b, State.NORMAL)) {
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
       if ((nodesCorrupt != null) && (nodesCorrupt.contains(node))) {
         corrupt++;
@@ -3407,8 +3413,7 @@ public class BlockManager implements BlockStatsMXBean {
         stale++;
       }
     }
-    return new NumberReplicas(live, decommissioned, decommissioning, corrupt,
-        excess, stale);
+    return new NumberReplicas(live, decommissioned, decommissioning, corrupt, excess, stale);
   }
 
   /** 
@@ -3591,6 +3596,8 @@ public class BlockManager implements BlockStatsMXBean {
       String src, BlockInfo[] blocks) {
     for (BlockInfo b: blocks) {
       if (!b.isComplete()) {
+        final BlockInfoUnderConstruction uc =
+            (BlockInfoUnderConstruction)b;
         final int numNodes = b.numNodes();
         final int min = getMinStorageNum(b);
         final BlockUCState state = b.getBlockUCState();
@@ -3716,7 +3723,11 @@ public class BlockManager implements BlockStatsMXBean {
     return blocksMap.getBlockCollection(b);
   }
 
-  public void removeBlockFromMap(BlockInfo block) {
+  public int numCorruptReplicas(Block block) {
+    return corruptReplicas.numCorruptReplicas(block);
+  }
+
+  public void removeBlockFromMap(Block block) {
     removeFromExcessReplicateMap(block);
     blocksMap.removeBlock(block);
     // If block is removed from blocksMap remove it from corruptReplicasMap
@@ -3726,7 +3737,7 @@ public class BlockManager implements BlockStatsMXBean {
   /**
    * If a block is removed from blocksMap, remove it from excessReplicateMap.
    */
-  private void removeFromExcessReplicateMap(BlockInfo block) {
+  private void removeFromExcessReplicateMap(Block block) {
     for (DatanodeStorageInfo info : getStorages(block)) {
       String uuid = info.getDatanodeDescriptor().getDatanodeUuid();
       LightWeightLinkedSet<BlockInfo> excessReplicas =
@@ -3757,14 +3768,14 @@ public class BlockManager implements BlockStatsMXBean {
   /**
    * Get the replicas which are corrupt for a given block.
    */
-  public Collection<DatanodeDescriptor> getCorruptReplicas(BlockInfo block) {
+  public Collection<DatanodeDescriptor> getCorruptReplicas(Block block) {
     return corruptReplicas.getNodes(block);
   }
 
  /**
   * Get reason for certain corrupted replicas for a given block and a given dn.
   */
- public String getCorruptReason(BlockInfo block, DatanodeDescriptor node) {
+ public String getCorruptReason(Block block, DatanodeDescriptor node) {
    return corruptReplicas.getCorruptReason(block, node);
  }
 
@@ -3858,7 +3869,7 @@ public class BlockManager implements BlockStatsMXBean {
     datanodeManager.clearPendingQueues();
     postponedMisreplicatedBlocks.clear();
     postponedMisreplicatedBlocksCount.set(0);
-  }
+  };
 
   public static LocatedBlock newLocatedBlock(
       ExtendedBlock b, DatanodeStorageInfo[] storages,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc99aaff/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 85cea5a..0dbf485 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
@@ -117,7 +117,7 @@ class BlocksMap {
    * remove it from all data-node lists it belongs to;
    * and remove all data-node locations associated with the block.
    */
-  void removeBlock(BlockInfo block) {
+  void removeBlock(Block block) {
     BlockInfo blockInfo = blocks.remove(block);
     if (blockInfo == null)
       return;
@@ -190,7 +190,7 @@ class BlocksMap {
     // remove block from the data-node list and the node from the block info
     boolean removed = node.removeBlock(info);
 
-    if (info.hasEmptyStorage()     // no datanodes left
+    if (info.getDatanode(0) == null     // no datanodes left
               && info.isDeleted()) {  // does not belong to a file
       blocks.remove(b);  // remove block from the map
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc99aaff/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ContiguousBlockStorageOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ContiguousBlockStorageOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ContiguousBlockStorageOp.java
index 70251e1..092f65e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ContiguousBlockStorageOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ContiguousBlockStorageOp.java
@@ -45,12 +45,13 @@ class ContiguousBlockStorageOp {
     return last;
   }
 
-  static void addStorage(BlockInfo b, DatanodeStorageInfo storage) {
+  static boolean addStorage(BlockInfo b, DatanodeStorageInfo storage) {
     // find the last null node
     int lastNode = ensureCapacity(b, 1);
     b.setStorageInfo(lastNode, storage);
     b.setNext(lastNode, null);
     b.setPrevious(lastNode, null);
+    return true;
   }
 
   static boolean removeStorage(BlockInfo b,
@@ -102,8 +103,4 @@ class ContiguousBlockStorageOp {
           "newBlock already exists.");
     }
   }
-
-  static boolean hasEmptyStorage(BlockInfo b) {
-    return b.getStorageInfo(0) == null;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc99aaff/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java
index 9a0023d..fc2e234 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java
@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.ipc.Server;
 
@@ -46,12 +46,8 @@ public class CorruptReplicasMap{
     CORRUPTION_REPORTED  // client or datanode reported the corruption
   }
 
-  /**
-   * Used to track corrupted replicas (for contiguous block) or internal blocks
-   * (for striped block) and the corresponding DataNodes. For a striped block,
-   * the key here is the striped block group object stored in the blocksMap.
-   */
-  private final SortedMap<BlockInfo, Map<DatanodeDescriptor, Reason>> corruptReplicasMap = new TreeMap<>();
+  private final SortedMap<Block, Map<DatanodeDescriptor, Reason>> corruptReplicasMap =
+    new TreeMap<Block, Map<DatanodeDescriptor, Reason>>();
 
   /**
    * Mark the block belonging to datanode as corrupt.
@@ -61,21 +57,21 @@ public class CorruptReplicasMap{
    * @param reason a textual reason (for logging purposes)
    * @param reasonCode the enum representation of the reason
    */
-  void addToCorruptReplicasMap(BlockInfo blk, DatanodeDescriptor dn,
+  void addToCorruptReplicasMap(Block blk, DatanodeDescriptor dn,
       String reason, Reason reasonCode) {
     Map <DatanodeDescriptor, Reason> nodes = corruptReplicasMap.get(blk);
     if (nodes == null) {
-      nodes = new HashMap<>();
+      nodes = new HashMap<DatanodeDescriptor, Reason>();
       corruptReplicasMap.put(blk, nodes);
     }
-
+    
     String reasonText;
     if (reason != null) {
       reasonText = " because " + reason;
     } else {
       reasonText = "";
     }
-
+    
     if (!nodes.keySet().contains(dn)) {
       NameNode.blockStateChangeLog.info(
           "BLOCK NameSystem.addToCorruptReplicasMap: {} added as corrupt on "
@@ -96,7 +92,7 @@ public class CorruptReplicasMap{
    *
    * @param blk Block to be removed
    */
-  void removeFromCorruptReplicasMap(BlockInfo blk) {
+  void removeFromCorruptReplicasMap(Block blk) {
     if (corruptReplicasMap != null) {
       corruptReplicasMap.remove(blk);
     }
@@ -109,13 +105,12 @@ public class CorruptReplicasMap{
    * @return true if the removal is successful; 
              false if the replica is not in the map
    */ 
-  boolean removeFromCorruptReplicasMap(BlockInfo blk,
-      DatanodeDescriptor datanode) {
+  boolean removeFromCorruptReplicasMap(Block blk, DatanodeDescriptor datanode) {
     return removeFromCorruptReplicasMap(blk, datanode, Reason.ANY);
   }
 
-  boolean removeFromCorruptReplicasMap(BlockInfo blk,
-      DatanodeDescriptor datanode, Reason reason) {
+  boolean removeFromCorruptReplicasMap(Block blk, DatanodeDescriptor datanode,
+      Reason reason) {
     Map <DatanodeDescriptor, Reason> datanodes = corruptReplicasMap.get(blk);
     if (datanodes==null)
       return false;
@@ -144,9 +139,11 @@ public class CorruptReplicasMap{
    * @param blk Block for which nodes are requested
    * @return collection of nodes. Null if does not exists
    */
-  Collection<DatanodeDescriptor> getNodes(BlockInfo blk) {
-    Map<DatanodeDescriptor, Reason> nodes = corruptReplicasMap.get(blk);
-    return nodes != null ? nodes.keySet() : null;
+  Collection<DatanodeDescriptor> getNodes(Block blk) {
+    Map <DatanodeDescriptor, Reason> nodes = corruptReplicasMap.get(blk);
+    if (nodes == null)
+      return null;
+    return nodes.keySet();
   }
 
   /**
@@ -156,12 +153,12 @@ public class CorruptReplicasMap{
    * @param node DatanodeDescriptor which holds the replica
    * @return true if replica is corrupt, false if does not exists in this map
    */
-  boolean isReplicaCorrupt(BlockInfo blk, DatanodeDescriptor node) {
+  boolean isReplicaCorrupt(Block blk, DatanodeDescriptor node) {
     Collection<DatanodeDescriptor> nodes = getNodes(blk);
     return ((nodes != null) && (nodes.contains(node)));
   }
 
-  int numCorruptReplicas(BlockInfo blk) {
+  int numCorruptReplicas(Block blk) {
     Collection<DatanodeDescriptor> nodes = getNodes(blk);
     return (nodes == null) ? 0 : nodes.size();
   }
@@ -171,9 +168,9 @@ public class CorruptReplicasMap{
   }
 
   /**
-   * Return a range of corrupt replica block ids. Up to numExpectedBlocks
+   * Return a range of corrupt replica block ids. Up to numExpectedBlocks 
    * blocks starting at the next block after startingBlockId are returned
-   * (fewer if numExpectedBlocks blocks are unavailable). If startingBlockId
+   * (fewer if numExpectedBlocks blocks are unavailable). If startingBlockId 
    * is null, up to numExpectedBlocks blocks are returned from the beginning.
    * If startingBlockId cannot be found, null is returned.
    *
@@ -184,39 +181,44 @@ public class CorruptReplicasMap{
    * @return Up to numExpectedBlocks blocks from startingBlockId if it exists
    *
    */
-  @VisibleForTesting
   long[] getCorruptReplicaBlockIds(int numExpectedBlocks,
                                    Long startingBlockId) {
     if (numExpectedBlocks < 0 || numExpectedBlocks > 100) {
       return null;
     }
-    Iterator<BlockInfo> blockIt = corruptReplicasMap.keySet().iterator();
+    
+    Iterator<Block> blockIt = corruptReplicasMap.keySet().iterator();
+    
     // if the starting block id was specified, iterate over keys until
     // we find the matching block. If we find a matching block, break
-    // to leave the iterator on the next block after the specified block.
+    // to leave the iterator on the next block after the specified block. 
     if (startingBlockId != null) {
       boolean isBlockFound = false;
       while (blockIt.hasNext()) {
-        BlockInfo b = blockIt.next();
+        Block b = blockIt.next();
         if (b.getBlockId() == startingBlockId) {
           isBlockFound = true;
-          break;
+          break; 
         }
       }
+      
       if (!isBlockFound) {
         return null;
       }
     }
 
-    ArrayList<Long> corruptReplicaBlockIds = new ArrayList<>();
+    ArrayList<Long> corruptReplicaBlockIds = new ArrayList<Long>();
+
     // append up to numExpectedBlocks blockIds to our list
     for(int i=0; i<numExpectedBlocks && blockIt.hasNext(); i++) {
       corruptReplicaBlockIds.add(blockIt.next().getBlockId());
     }
+    
     long[] ret = new long[corruptReplicaBlockIds.size()];
     for(int i=0; i<ret.length; i++) {
       ret[i] = corruptReplicaBlockIds.get(i);
     }
+    
     return ret;
   }
 
@@ -227,7 +229,7 @@ public class CorruptReplicasMap{
    * @param node datanode that contains this corrupted replica
    * @return reason
    */
-  String getCorruptReason(BlockInfo block, DatanodeDescriptor node) {
+  String getCorruptReason(Block block, DatanodeDescriptor node) {
     Reason reason = null;
     if(corruptReplicasMap.containsKey(block)) {
       if (corruptReplicasMap.get(block).containsKey(node)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc99aaff/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index eebeac0..4830d5d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -71,7 +71,7 @@ class FSDirWriteFileOp {
   private FSDirWriteFileOp() {}
   static boolean unprotectedRemoveBlock(
       FSDirectory fsd, String path, INodesInPath iip, INodeFile fileNode,
-      BlockInfo block) throws IOException {
+      Block block) throws IOException {
     // modify file-> block and blocksMap
     // fileNode should be under construction
     BlockInfoUnderConstruction uc = fileNode.removeLastBlock(block);
@@ -136,9 +136,7 @@ class FSDirWriteFileOp {
     fsd.writeLock();
     try {
       // Remove the block from the pending creates list
-      BlockInfo storedBlock = fsd.getBlockManager().getStoredBlock(localBlock);
-      if (storedBlock != null &&
-          !unprotectedRemoveBlock(fsd, src, iip, file, storedBlock)) {
+      if (!unprotectedRemoveBlock(fsd, src, iip, file, localBlock)) {
         return;
       }
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc99aaff/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 96d6982..63ef985 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
@@ -1035,7 +1035,7 @@ public class FSEditLogLoader {
         throw new IOException("Trying to remove more than one block from file "
             + path);
       }
-      BlockInfo oldBlock = oldBlocks[oldBlocks.length - 1];
+      Block oldBlock = oldBlocks[oldBlocks.length - 1];
       boolean removed = FSDirWriteFileOp.unprotectedRemoveBlock(
           fsDir, path, iip, file, oldBlock);
       if (!removed && !(op instanceof UpdateBlocksOp)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc99aaff/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 2a8231a..ab179b4 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
@@ -267,8 +267,10 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       out.println("No. of corrupted Replica: " +
           numberReplicas.corruptReplicas());
       //record datanodes that have corrupted block replica
-      Collection<DatanodeDescriptor> corruptionRecord =
-          bm.getCorruptReplicas(blockInfo);
+      Collection<DatanodeDescriptor> corruptionRecord = null;
+      if (bm.getCorruptReplicas(block) != null) {
+        corruptionRecord = bm.getCorruptReplicas(block);
+      }
 
       //report block replicas status on datanodes
       for(int idx = (blockInfo.numNodes()-1); idx >= 0; idx--) {
@@ -277,7 +279,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
             dn.getNetworkLocation() + " ");
         if (corruptionRecord != null && corruptionRecord.contains(dn)) {
           out.print(CORRUPT_STATUS+"\t ReasonCode: "+
-            bm.getCorruptReason(blockInfo, dn));
+            bm.getCorruptReason(block,dn));
         } else if (dn.isDecommissioned() ){
           out.print(DECOMMISSIONED_STATUS);
         } else if (dn.isDecommissionInProgress()) {
@@ -648,7 +650,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
               LightWeightLinkedSet<BlockInfo> blocksExcess =
                   bm.excessReplicateMap.get(dnDesc.getDatanodeUuid());
               Collection<DatanodeDescriptor> corruptReplicas =
-                  bm.getCorruptReplicas(storedBlock);
+                  bm.getCorruptReplicas(block.getLocalBlock());
               sb.append("(");
               if (dnDesc.isDecommissioned()) {
                 sb.append("DECOMMISSIONED)");
@@ -656,7 +658,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
                 sb.append("DECOMMISSIONING)");
               } else if (corruptReplicas != null && corruptReplicas.contains(dnDesc)) {
                 sb.append("CORRUPT)");
-              } else if (blocksExcess != null && blocksExcess.contains(storedBlock)) {
+              } else if (blocksExcess != null && blocksExcess.contains(block.getLocalBlock())) {
                 sb.append("EXCESS)");
               } else if (dnDesc.isStale(this.staleInterval)) {
                 sb.append("STALE_NODE)");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc99aaff/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index af1e023..89ee674 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -560,8 +560,7 @@ public class DFSTestUtil {
       throws TimeoutException, InterruptedException {
     int count = 0;
     final int ATTEMPTS = 50;
-    int repls = BlockManagerTestUtil.numCorruptReplicas(ns.getBlockManager(),
-        b.getLocalBlock());
+    int repls = ns.getBlockManager().numCorruptReplicas(b.getLocalBlock());
     while (repls != corruptRepls && count < ATTEMPTS) {
       try {
         IOUtils.copyBytes(fs.open(file), new IOUtils.NullOutputStream(),
@@ -573,8 +572,7 @@ public class DFSTestUtil {
       count++;
       // check more often so corrupt block reports are not easily missed
       for (int i = 0; i < 10; i++) {
-        repls = BlockManagerTestUtil.numCorruptReplicas(ns.getBlockManager(),
-            b.getLocalBlock());
+        repls = ns.getBlockManager().numCorruptReplicas(b.getLocalBlock());
         Thread.sleep(100);
         if (repls == corruptRepls) {
           break;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc99aaff/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
index a899891..148135b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
@@ -24,7 +24,6 @@ import java.util.HashSet;
 import java.util.Set;
 import java.util.concurrent.ExecutionException;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -88,7 +87,7 @@ public class BlockManagerTestUtil {
       final Block b) {
     final Set<String> rackSet = new HashSet<String>(0);
     final Collection<DatanodeDescriptor> corruptNodes = 
-       getCorruptReplicas(blockManager).getNodes(blockManager.getStoredBlock(b));
+       getCorruptReplicas(blockManager).getNodes(b);
     for(DatanodeStorageInfo storage : blockManager.blocksMap.getStorages(b)) {
       final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
       if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
@@ -307,8 +306,4 @@ public class BlockManagerTestUtil {
       throws ExecutionException, InterruptedException {
     dm.getDecomManager().runMonitor();
   }
-
-  public static int numCorruptReplicas(BlockManager bm, Block block) {
-    return bm.corruptReplicas.numCorruptReplicas(bm.getStoredBlock(block));
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc99aaff/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
index c23f3d0..bae4f1d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
@@ -63,7 +63,9 @@ public class TestBlockInfo {
 
     final DatanodeStorageInfo storage = DFSTestUtil.createDatanodeStorageInfo("storageID", "127.0.0.1");
 
-    blockInfo.addStorage(storage, blockInfo);
+    boolean added = blockInfo.addStorage(storage, blockInfo);
+
+    Assert.assertTrue(added);
     Assert.assertEquals(storage, blockInfo.getStorageInfo(0));
   }
 
@@ -71,7 +73,7 @@ public class TestBlockInfo {
   public void testCopyConstructor() {
     BlockInfo old = new BlockInfoContiguous((short) 3);
     try {
-      BlockInfo copy = new BlockInfoContiguous(old);
+      BlockInfo copy = new BlockInfoContiguous((BlockInfoContiguous)old);
       assertEquals(old.getBlockCollection(), copy.getBlockCollection());
       assertEquals(old.getCapacity(), copy.getCapacity());
     } catch (Exception e) {
@@ -108,8 +110,8 @@ public class TestBlockInfo {
     final int MAX_BLOCKS = 10;
 
     DatanodeStorageInfo dd = DFSTestUtil.createDatanodeStorageInfo("s1", "1.1.1.1");
-    ArrayList<Block> blockList = new ArrayList<>(MAX_BLOCKS);
-    ArrayList<BlockInfo> blockInfoList = new ArrayList<>();
+    ArrayList<Block> blockList = new ArrayList<Block>(MAX_BLOCKS);
+    ArrayList<BlockInfo> blockInfoList = new ArrayList<BlockInfo>();
     int headIndex;
     int curIndex;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc99aaff/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
index f6cc747..9e31670 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
@@ -509,7 +509,7 @@ public class TestBlockManager {
         + " even if all available source nodes have reached their replication"
         + " limits below the hard limit.",
         bm.chooseSourceDatanode(
-            bm.getStoredBlock(aBlock),
+            aBlock,
             cntNodes,
             liveNodes,
             new NumberReplicas(),
@@ -519,7 +519,7 @@ public class TestBlockManager {
         + " replication since all available source nodes have reached"
         + " their replication limits.",
         bm.chooseSourceDatanode(
-            bm.getStoredBlock(aBlock),
+            aBlock,
             cntNodes,
             liveNodes,
             new NumberReplicas(),
@@ -532,7 +532,7 @@ public class TestBlockManager {
     assertNull("Does not choose a source node for a highest-priority"
         + " replication when all available nodes exceed the hard limit.",
         bm.chooseSourceDatanode(
-            bm.getStoredBlock(aBlock),
+            aBlock,
             cntNodes,
             liveNodes,
             new NumberReplicas(),
@@ -558,7 +558,7 @@ public class TestBlockManager {
         + " if all available source nodes have reached their replication"
         + " limits below the hard limit.",
         bm.chooseSourceDatanode(
-            bm.getStoredBlock(aBlock),
+            aBlock,
             cntNodes,
             liveNodes,
             new NumberReplicas(),
@@ -572,7 +572,7 @@ public class TestBlockManager {
     assertNull("Does not choose a source decommissioning node for a normal"
         + " replication when all available nodes exceed the hard limit.",
         bm.chooseSourceDatanode(
-            bm.getStoredBlock(aBlock),
+            aBlock,
             cntNodes,
             liveNodes,
             new NumberReplicas(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc99aaff/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCorruptReplicaInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCorruptReplicaInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCorruptReplicaInfo.java
index 1a49bee..21fb54e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCorruptReplicaInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCorruptReplicaInfo.java
@@ -48,19 +48,20 @@ public class TestCorruptReplicaInfo {
   private static final Log LOG = 
                            LogFactory.getLog(TestCorruptReplicaInfo.class);
   
-  private final Map<Long, BlockInfo> block_map = new HashMap<>();
+  private final Map<Long, Block> block_map =
+    new HashMap<Long, Block>();  
     
   // Allow easy block creation by block id
   // Return existing block if one with same block id already exists
-  private BlockInfo getBlock(Long block_id) {
+  private Block getBlock(Long block_id) {
     if (!block_map.containsKey(block_id)) {
-      block_map.put(block_id,
-          new BlockInfoContiguous(new Block(block_id, 0, 0), (short) 1));
+      block_map.put(block_id, new Block(block_id,0,0));
     }
+    
     return block_map.get(block_id);
   }
   
-  private BlockInfo getBlock(int block_id) {
+  private Block getBlock(int block_id) {
     return getBlock((long)block_id);
   }
   
@@ -81,7 +82,7 @@ public class TestCorruptReplicaInfo {
       // create a list of block_ids. A list is used to allow easy validation of the
       // output of getCorruptReplicaBlockIds
       int NUM_BLOCK_IDS = 140;
-      List<Long> block_ids = new LinkedList<>();
+      List<Long> block_ids = new LinkedList<Long>();
       for (int i=0;i<NUM_BLOCK_IDS;i++) {
         block_ids.add((long)i);
       }
@@ -129,7 +130,7 @@ public class TestCorruptReplicaInfo {
   }
   
   private static void addToCorruptReplicasMap(CorruptReplicasMap crm,
-      BlockInfo blk, DatanodeDescriptor dn) {
+      Block blk, DatanodeDescriptor dn) {
     crm.addToCorruptReplicasMap(blk, dn, "TEST", Reason.NONE);
   }
 }


[39/50] hadoop git commit: YARN-3381. Fix typo InvalidStateTransitonException. Contributed by Brahma Reddy Battula.

Posted by ar...@apache.org.
YARN-3381. Fix typo InvalidStateTransitonException. Contributed by Brahma Reddy Battula.


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

Branch: refs/heads/HDFS-7240
Commit: 19295b36d90e26616accee73b1f7743aab5df692
Parents: e04faf8
Author: Akira Ajisaka <aa...@apache.org>
Authored: Mon Jul 13 17:52:13 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Mon Jul 13 17:52:13 2015 +0900

----------------------------------------------------------------------
 .../mapreduce/v2/app/job/impl/JobImpl.java      |  4 +-
 .../v2/app/job/impl/TaskAttemptImpl.java        |  4 +-
 .../mapreduce/v2/app/job/impl/TaskImpl.java     |  4 +-
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../api/async/impl/NMClientAsyncImpl.java       |  4 +-
 .../state/InvalidStateTransitionException.java  | 51 ++++++++++++++++++++
 .../state/InvalidStateTransitonException.java   | 21 +++-----
 .../apache/hadoop/yarn/state/StateMachine.java  |  2 +-
 .../hadoop/yarn/state/StateMachineFactory.java  | 10 ++--
 .../application/ApplicationImpl.java            |  4 +-
 .../container/ContainerImpl.java                |  4 +-
 .../localizer/LocalizedResource.java            |  4 +-
 .../resourcemanager/recovery/RMStateStore.java  |  4 +-
 .../server/resourcemanager/rmapp/RMAppImpl.java |  4 +-
 .../rmapp/attempt/RMAppAttemptImpl.java         |  4 +-
 .../rmcontainer/RMContainerImpl.java            |  4 +-
 .../resourcemanager/rmnode/RMNodeImpl.java      |  4 +-
 17 files changed, 90 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/19295b36/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
index 2c48019..731bcba 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
@@ -122,7 +122,7 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
@@ -994,7 +994,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       JobStateInternal oldState = getInternalState();
       try {
          getStateMachine().doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
+      } catch (InvalidStateTransitionException e) {
         LOG.error("Can't handle this event at current state", e);
         addDiagnostic("Invalid event " + event.getType() + 
             " on Job " + this.jobId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19295b36/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
index 3fa42fe..77a7555 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
@@ -128,7 +128,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
@@ -1188,7 +1188,7 @@ public abstract class TaskAttemptImpl implements
       final TaskAttemptStateInternal oldState = getInternalState()  ;
       try {
         stateMachine.doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
+      } catch (InvalidStateTransitionException e) {
         LOG.error("Can't handle this event at current state for "
             + this.attemptId, e);
         eventHandler.handle(new JobDiagnosticsUpdateEvent(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19295b36/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
index ca81059..5f5f300 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
@@ -85,7 +85,7 @@ import org.apache.hadoop.util.StringInterner;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
@@ -646,7 +646,7 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
       TaskStateInternal oldState = getInternalState();
       try {
         stateMachine.doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
+      } catch (InvalidStateTransitionException e) {
         LOG.error("Can't handle this event at current state for "
             + this.taskId, e);
         internalError(event.getType());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19295b36/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index f23effa..98f0e8d 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -625,6 +625,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3894. RM startup should fail for wrong CS xml NodeLabel capacity
     configuration. (Bibin A Chundatt via wangda)
 
+    YARN-3381. Fix typo InvalidStateTransitonException.
+    (Brahma Reddy Battula via aajisaka)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19295b36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java
index 700a509..39682df 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java
@@ -55,7 +55,7 @@ import org.apache.hadoop.yarn.event.AbstractEvent;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
@@ -496,7 +496,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
       try {
         try {
           this.stateMachine.doTransition(event.getType(), event);
-        } catch (InvalidStateTransitonException e) {
+        } catch (InvalidStateTransitionException e) {
           LOG.error("Can't handle this event at current state", e);
         }
       } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19295b36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitionException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitionException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitionException.java
new file mode 100644
index 0000000..d10902a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitionException.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.state;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+
+/**
+ * The exception that happens when you call invalid state transition.
+ *
+ */
+@Public
+@Evolving
+public class InvalidStateTransitionException extends YarnRuntimeException {
+
+  private static final long serialVersionUID = -6188669113571351684L;
+  private Enum<?> currentState;
+  private Enum<?> event;
+
+  public InvalidStateTransitionException(Enum<?> currentState, Enum<?> event) {
+    super("Invalid event: " + event + " at " + currentState);
+    this.currentState = currentState;
+    this.event = event;
+  }
+
+  public Enum<?> getCurrentState() {
+    return currentState;
+  }
+
+  public Enum<?> getEvent() {
+    return event;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19295b36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitonException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitonException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitonException.java
index a9a8b11..eeb1b97 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitonException.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitonException.java
@@ -20,29 +20,20 @@ package org.apache.hadoop.yarn.state;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+
+/** @deprecated Use {@link InvalidStateTransitionException} instead. */
 
 @Public
 @Evolving
-public class InvalidStateTransitonException extends YarnRuntimeException {
+@Deprecated
+public class InvalidStateTransitonException extends
+    InvalidStateTransitionException {
 
   private static final long serialVersionUID = 8610511635996283691L;
 
-  private Enum<?> currentState;
-  private Enum<?> event;
-
   public InvalidStateTransitonException(Enum<?> currentState, Enum<?> event) {
-    super("Invalid event: " + event + " at " + currentState);
-    this.currentState = currentState;
-    this.event = event;
+    super(currentState, event);
   }
 
-  public Enum<?> getCurrentState() {
-    return currentState;
-  }
-  
-  public Enum<?> getEvent() {
-    return event;
-  }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19295b36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachine.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachine.java
index b5ce6a3..5151559 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachine.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachine.java
@@ -28,5 +28,5 @@ public interface StateMachine
                   EVENTTYPE extends Enum<EVENTTYPE>, EVENT> {
   public STATE getCurrentState();
   public STATE doTransition(EVENTTYPE eventType, EVENT event)
-        throws InvalidStateTransitonException;
+        throws InvalidStateTransitionException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19295b36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java
index 55ac4cf..5b76ce8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java
@@ -289,7 +289,7 @@ final public class StateMachineFactory
    */
   private STATE doTransition
            (OPERAND operand, STATE oldState, EVENTTYPE eventType, EVENT event)
-      throws InvalidStateTransitonException {
+      throws InvalidStateTransitionException {
     // We can assume that stateMachineTable is non-null because we call
     //  maybeMakeStateMachineTable() when we build an InnerStateMachine ,
     //  and this code only gets called from inside a working InnerStateMachine .
@@ -302,7 +302,7 @@ final public class StateMachineFactory
         return transition.doTransition(operand, oldState, event, eventType);
       }
     }
-    throw new InvalidStateTransitonException(oldState, eventType);
+    throw new InvalidStateTransitionException(oldState, eventType);
   }
 
   private synchronized void maybeMakeStateMachineTable() {
@@ -381,11 +381,11 @@ final public class StateMachineFactory
     @Override
     public STATE doTransition(OPERAND operand, STATE oldState,
                               EVENT event, EVENTTYPE eventType)
-        throws InvalidStateTransitonException {
+        throws InvalidStateTransitionException {
       STATE postState = hook.transition(operand, event);
 
       if (!validPostStates.contains(postState)) {
-        throw new InvalidStateTransitonException(oldState, eventType);
+        throw new InvalidStateTransitionException(oldState, eventType);
       }
       return postState;
     }
@@ -444,7 +444,7 @@ final public class StateMachineFactory
 
     @Override
     public synchronized STATE doTransition(EVENTTYPE eventType, EVENT event)
-         throws InvalidStateTransitonException  {
+         throws InvalidStateTransitionException  {
       currentState = StateMachineFactory.this.doTransition
           (operand, currentState, eventType, event);
       return currentState;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19295b36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
----------------------------------------------------------------------
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/application/ApplicationImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
index a73b113..e880c31 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
@@ -49,7 +49,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppFinishedEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppStartedEvent;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
@@ -457,7 +457,7 @@ public class ApplicationImpl implements Application {
       try {
         // queue event requesting init of the same app
         newState = stateMachine.doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
+      } catch (InvalidStateTransitionException e) {
         LOG.warn("Can't handle this event at current state", e);
       }
       if (oldState != newState) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19295b36/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 289d6d0..3c76596 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
@@ -71,7 +71,7 @@ import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredContainerStatus;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
@@ -1121,7 +1121,7 @@ public class ContainerImpl implements Container {
       try {
         newState =
             stateMachine.doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
+      } catch (InvalidStateTransitionException e) {
         LOG.warn("Can't handle this event at current state: Current: ["
             + oldState + "], eventType: [" + event.getType() + "]", e);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19295b36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalizedResource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalizedResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalizedResource.java
index e2d0fe1..04d95f1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalizedResource.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalizedResource.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.even
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceRecoveredEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceReleaseEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceRequestEvent;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
 import org.apache.hadoop.yarn.state.StateMachineFactory;
@@ -196,7 +196,7 @@ public class LocalizedResource implements EventHandler<ResourceEvent> {
       ResourceState newState = null;
       try {
         newState = this.stateMachine.doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
+      } catch (InvalidStateTransitionException e) {
         LOG.warn("Can't handle this event at current state", e);
       }
       if (oldState != newState) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19295b36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.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/recovery/RMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
index cc4edd7..46c2954 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
@@ -62,7 +62,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AggregateAppR
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
 import org.apache.hadoop.yarn.state.StateMachineFactory;
@@ -841,7 +841,7 @@ public abstract class RMStateStore extends AbstractService {
             + getRMStateStoreState());
       }
 
-    } catch (InvalidStateTransitonException e) {
+    } catch (InvalidStateTransitionException e) {
       LOG.error("Can't handle this event at current state", e);
     } finally {
       this.writeLock.unlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19295b36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.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/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index 90e63c1..62d5555 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -93,7 +93,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSch
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.webproxy.ProxyUriUtils;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
@@ -759,7 +759,7 @@ public class RMAppImpl implements RMApp, Recoverable {
       try {
         /* keep the master in sync with the state machine */
         this.stateMachine.doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
+      } catch (InvalidStateTransitionException e) {
         LOG.error("Can't handle this event at current state", e);
         /* TODO fail the application on the failed transition */
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19295b36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
index 5171bba..0914022 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
@@ -96,7 +96,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptA
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.webproxy.ProxyUriUtils;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
@@ -784,7 +784,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       try {
         /* keep the master in sync with the state machine */
         this.stateMachine.doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
+      } catch (InvalidStateTransitionException e) {
         LOG.error("Can't handle this event at current state", e);
         /* TODO fail the application on the failed transition */
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19295b36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.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/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
index 316a450..0ad63b4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
@@ -47,7 +47,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
@@ -385,7 +385,7 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
       RMContainerState oldState = getState();
       try {
          stateMachine.doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
+      } catch (InvalidStateTransitionException e) {
         LOG.error("Can't handle this event at current state", e);
         LOG.error("Invalid event " + event.getType() + 
             " on container " + this.containerId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19295b36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.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/rmnode/RMNodeImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
index 9bc91c7..09b9278 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
@@ -64,7 +64,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemoved
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeResourceUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils.ContainerIdComparator;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
@@ -446,7 +446,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
       NodeState oldState = getState();
       try {
          stateMachine.doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
+      } catch (InvalidStateTransitionException e) {
         LOG.error("Can't handle this event at current state", e);
         LOG.error("Invalid event " + event.getType() + 
             " on Node  " + this.nodeId);


[24/50] hadoop git commit: HADOOP-12180. Move ResourceCalculatorPlugin from YARN to Common. (Chris Douglas via kasha)

Posted by ar...@apache.org.
HADOOP-12180. Move ResourceCalculatorPlugin from YARN to Common. (Chris Douglas via kasha)


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

Branch: refs/heads/HDFS-7240
Commit: ac6048372a58b3a3b57cd5f2702b44a3d4667f3d
Parents: aa067c6
Author: Karthik Kambatla <ka...@apache.org>
Authored: Thu Jul 9 09:56:40 2015 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Thu Jul 9 09:56:40 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../org/apache/hadoop/util/CpuTimeTracker.java  | 115 +++++
 .../java/org/apache/hadoop/util/SysInfo.java    | 111 +++++
 .../org/apache/hadoop/util/SysInfoLinux.java    | 444 +++++++++++++++++++
 .../org/apache/hadoop/util/SysInfoWindows.java  | 181 ++++++++
 .../apache/hadoop/util/TestSysInfoLinux.java    | 323 ++++++++++++++
 .../apache/hadoop/util/TestSysInfoWindows.java  | 100 +++++
 .../apache/hadoop/yarn/util/CpuTimeTracker.java | 100 -----
 .../util/LinuxResourceCalculatorPlugin.java     | 392 +---------------
 .../yarn/util/ProcfsBasedProcessTree.java       |  34 +-
 .../yarn/util/ResourceCalculatorPlugin.java     |  68 +--
 .../yarn/util/WindowsBasedProcessTree.java      |   2 +-
 .../util/WindowsResourceCalculatorPlugin.java   | 158 +------
 .../util/TestLinuxResourceCalculatorPlugin.java | 324 --------------
 .../util/TestResourceCalculatorProcessTree.java |   2 +-
 .../TestWindowsResourceCalculatorPlugin.java    |  86 ----
 16 files changed, 1335 insertions(+), 1108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac604837/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 a906a63..d9a9eba 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -690,6 +690,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12201. Add tracing to FileSystem#createFileSystem and Globber#glob
     (cmccabe)
 
+    HADOOP-12180. Move ResourceCalculatorPlugin from YARN to Common. 
+    (Chris Douglas via kasha)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac604837/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CpuTimeTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CpuTimeTracker.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CpuTimeTracker.java
new file mode 100644
index 0000000..3f17c9a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CpuTimeTracker.java
@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.math.BigInteger;
+
+/**
+ * Utility for sampling and computing CPU usage.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class CpuTimeTracker {
+  public static final int UNAVAILABLE = -1;
+  private final long minimumTimeInterval;
+
+  // CPU used time since system is on (ms)
+  private BigInteger cumulativeCpuTime = BigInteger.ZERO;
+
+  // CPU used time read last time (ms)
+  private BigInteger lastCumulativeCpuTime = BigInteger.ZERO;
+
+  // Unix timestamp while reading the CPU time (ms)
+  private long sampleTime;
+  private long lastSampleTime;
+  private float cpuUsage;
+  private BigInteger jiffyLengthInMillis;
+
+  public CpuTimeTracker(long jiffyLengthInMillis) {
+    this.jiffyLengthInMillis = BigInteger.valueOf(jiffyLengthInMillis);
+    this.cpuUsage = UNAVAILABLE;
+    this.sampleTime = UNAVAILABLE;
+    this.lastSampleTime = UNAVAILABLE;
+    minimumTimeInterval =  10 * jiffyLengthInMillis;
+  }
+
+  /**
+   * Return percentage of cpu time spent over the time since last update.
+   * CPU time spent is based on elapsed jiffies multiplied by amount of
+   * time for 1 core. Thus, if you use 2 cores completely you would have spent
+   * twice the actual time between updates and this will return 200%.
+   *
+   * @return Return percentage of cpu usage since last update, {@link
+   * CpuTimeTracker#UNAVAILABLE} if there haven't been 2 updates more than
+   * {@link CpuTimeTracker#minimumTimeInterval} apart
+   */
+  public float getCpuTrackerUsagePercent() {
+    if (lastSampleTime == UNAVAILABLE ||
+        lastSampleTime > sampleTime) {
+      // lastSampleTime > sampleTime may happen when the system time is changed
+      lastSampleTime = sampleTime;
+      lastCumulativeCpuTime = cumulativeCpuTime;
+      return cpuUsage;
+    }
+    // When lastSampleTime is sufficiently old, update cpuUsage.
+    // Also take a sample of the current time and cumulative CPU time for the
+    // use of the next calculation.
+    if (sampleTime > lastSampleTime + minimumTimeInterval) {
+      cpuUsage =
+          ((cumulativeCpuTime.subtract(lastCumulativeCpuTime)).floatValue())
+          * 100F / ((float) (sampleTime - lastSampleTime));
+      lastSampleTime = sampleTime;
+      lastCumulativeCpuTime = cumulativeCpuTime;
+    }
+    return cpuUsage;
+  }
+
+  /**
+   * Obtain the cumulative CPU time since the system is on.
+   * @return cumulative CPU time in milliseconds
+   */
+  public long getCumulativeCpuTime() {
+    return cumulativeCpuTime.longValue();
+  }
+
+  /**
+   * Apply delta to accumulators.
+   * @param elapsedJiffies updated jiffies
+   * @param newTime new sample time
+   */
+  public void updateElapsedJiffies(BigInteger elapsedJiffies, long newTime) {
+    cumulativeCpuTime = elapsedJiffies.multiply(jiffyLengthInMillis);
+    sampleTime = newTime;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("SampleTime " + this.sampleTime);
+    sb.append(" CummulativeCpuTime " + this.cumulativeCpuTime);
+    sb.append(" LastSampleTime " + this.lastSampleTime);
+    sb.append(" LastCummulativeCpuTime " + this.lastCumulativeCpuTime);
+    sb.append(" CpuUsage " + this.cpuUsage);
+    sb.append(" JiffyLengthMillisec " + this.jiffyLengthInMillis);
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac604837/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfo.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfo.java
new file mode 100644
index 0000000..ec7fb24
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfo.java
@@ -0,0 +1,111 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Plugin to calculate resource information on the system.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class SysInfo {
+
+  /**
+   * Return default OS instance.
+   * @throws UnsupportedOperationException If cannot determine OS.
+   * @return Default instance for the detected OS.
+   */
+  public static SysInfo newInstance() {
+    if (Shell.LINUX) {
+      return new SysInfoLinux();
+    }
+    if (Shell.WINDOWS) {
+      return new SysInfoWindows();
+    }
+    throw new UnsupportedOperationException("Could not determine OS");
+  }
+
+  /**
+   * Obtain the total size of the virtual memory present in the system.
+   *
+   * @return virtual memory size in bytes.
+   */
+  public abstract long getVirtualMemorySize();
+
+  /**
+   * Obtain the total size of the physical memory present in the system.
+   *
+   * @return physical memory size bytes.
+   */
+  public abstract long getPhysicalMemorySize();
+
+  /**
+   * Obtain the total size of the available virtual memory present
+   * in the system.
+   *
+   * @return available virtual memory size in bytes.
+   */
+  public abstract long getAvailableVirtualMemorySize();
+
+  /**
+   * Obtain the total size of the available physical memory present
+   * in the system.
+   *
+   * @return available physical memory size bytes.
+   */
+  public abstract long getAvailablePhysicalMemorySize();
+
+  /**
+   * Obtain the total number of logical processors present on the system.
+   *
+   * @return number of logical processors
+   */
+  public abstract int getNumProcessors();
+
+  /**
+   * Obtain total number of physical cores present on the system.
+   *
+   * @return number of physical cores
+   */
+  public abstract int getNumCores();
+
+  /**
+   * Obtain the CPU frequency of on the system.
+   *
+   * @return CPU frequency in kHz
+   */
+  public abstract long getCpuFrequency();
+
+  /**
+   * Obtain the cumulative CPU time since the system is on.
+   *
+   * @return cumulative CPU time in milliseconds
+   */
+  public abstract long getCumulativeCpuTime();
+
+  /**
+   * Obtain the CPU usage % of the machine. Return -1 if it is unavailable
+   *
+   * @return CPU usage as a percentage of available cycles.
+   */
+  public abstract float getCpuUsage();
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac604837/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoLinux.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoLinux.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoLinux.java
new file mode 100644
index 0000000..055298d
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoLinux.java
@@ -0,0 +1,444 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.util;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.InputStreamReader;
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.charset.Charset;
+import java.util.HashSet;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.util.Shell.ShellCommandExecutor;
+
+/**
+ * Plugin to calculate resource information on Linux systems.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class SysInfoLinux extends SysInfo {
+  private static final Log LOG =
+      LogFactory.getLog(SysInfoLinux.class);
+
+  /**
+   * proc's meminfo virtual file has keys-values in the format
+   * "key:[ \t]*value[ \t]kB".
+   */
+  private static final String PROCFS_MEMFILE = "/proc/meminfo";
+  private static final Pattern PROCFS_MEMFILE_FORMAT =
+      Pattern.compile("^([a-zA-Z]*):[ \t]*([0-9]*)[ \t]kB");
+
+  // We need the values for the following keys in meminfo
+  private static final String MEMTOTAL_STRING = "MemTotal";
+  private static final String SWAPTOTAL_STRING = "SwapTotal";
+  private static final String MEMFREE_STRING = "MemFree";
+  private static final String SWAPFREE_STRING = "SwapFree";
+  private static final String INACTIVE_STRING = "Inactive";
+
+  /**
+   * Patterns for parsing /proc/cpuinfo.
+   */
+  private static final String PROCFS_CPUINFO = "/proc/cpuinfo";
+  private static final Pattern PROCESSOR_FORMAT =
+      Pattern.compile("^processor[ \t]:[ \t]*([0-9]*)");
+  private static final Pattern FREQUENCY_FORMAT =
+      Pattern.compile("^cpu MHz[ \t]*:[ \t]*([0-9.]*)");
+  private static final Pattern PHYSICAL_ID_FORMAT =
+      Pattern.compile("^physical id[ \t]*:[ \t]*([0-9]*)");
+  private static final Pattern CORE_ID_FORMAT =
+      Pattern.compile("^core id[ \t]*:[ \t]*([0-9]*)");
+
+  /**
+   * Pattern for parsing /proc/stat.
+   */
+  private static final String PROCFS_STAT = "/proc/stat";
+  private static final Pattern CPU_TIME_FORMAT =
+      Pattern.compile("^cpu[ \t]*([0-9]*)" +
+                      "[ \t]*([0-9]*)[ \t]*([0-9]*)[ \t].*");
+  private CpuTimeTracker cpuTimeTracker;
+
+  private String procfsMemFile;
+  private String procfsCpuFile;
+  private String procfsStatFile;
+  private long jiffyLengthInMillis;
+
+  private long ramSize = 0;
+  private long swapSize = 0;
+  private long ramSizeFree = 0;  // free ram space on the machine (kB)
+  private long swapSizeFree = 0; // free swap space on the machine (kB)
+  private long inactiveSize = 0; // inactive cache memory (kB)
+  /* number of logical processors on the system. */
+  private int numProcessors = 0;
+  /* number of physical cores on the system. */
+  private int numCores = 0;
+  private long cpuFrequency = 0L; // CPU frequency on the system (kHz)
+
+  private boolean readMemInfoFile = false;
+  private boolean readCpuInfoFile = false;
+
+  public static final long PAGE_SIZE = getConf("PAGESIZE");
+  public static final long JIFFY_LENGTH_IN_MILLIS =
+      Math.max(Math.round(1000D / getConf("CLK_TCK")), -1);
+
+  private static long getConf(String attr) {
+    if(Shell.LINUX) {
+      try {
+        ShellCommandExecutor shellExecutorClk = new ShellCommandExecutor(
+            new String[] {"getconf", attr });
+        shellExecutorClk.execute();
+        return Long.parseLong(shellExecutorClk.getOutput().replace("\n", ""));
+      } catch (IOException|NumberFormatException e) {
+        return -1;
+      }
+    }
+    return -1;
+  }
+
+  /**
+   * Get current time.
+   * @return Unix time stamp in millisecond
+   */
+  long getCurrentTime() {
+    return System.currentTimeMillis();
+  }
+
+  public SysInfoLinux() {
+    this(PROCFS_MEMFILE, PROCFS_CPUINFO, PROCFS_STAT,
+        JIFFY_LENGTH_IN_MILLIS);
+  }
+
+  /**
+   * Constructor which allows assigning the /proc/ directories. This will be
+   * used only in unit tests.
+   * @param procfsMemFile fake file for /proc/meminfo
+   * @param procfsCpuFile fake file for /proc/cpuinfo
+   * @param procfsStatFile fake file for /proc/stat
+   * @param jiffyLengthInMillis fake jiffy length value
+   */
+  @VisibleForTesting
+  public SysInfoLinux(String procfsMemFile,
+                                       String procfsCpuFile,
+                                       String procfsStatFile,
+                                       long jiffyLengthInMillis) {
+    this.procfsMemFile = procfsMemFile;
+    this.procfsCpuFile = procfsCpuFile;
+    this.procfsStatFile = procfsStatFile;
+    this.jiffyLengthInMillis = jiffyLengthInMillis;
+    this.cpuTimeTracker = new CpuTimeTracker(jiffyLengthInMillis);
+  }
+
+  /**
+   * Read /proc/meminfo, parse and compute memory information only once.
+   */
+  private void readProcMemInfoFile() {
+    readProcMemInfoFile(false);
+  }
+
+  /**
+   * Read /proc/meminfo, parse and compute memory information.
+   * @param readAgain if false, read only on the first time
+   */
+  private void readProcMemInfoFile(boolean readAgain) {
+
+    if (readMemInfoFile && !readAgain) {
+      return;
+    }
+
+    // Read "/proc/memInfo" file
+    BufferedReader in;
+    InputStreamReader fReader;
+    try {
+      fReader = new InputStreamReader(
+          new FileInputStream(procfsMemFile), Charset.forName("UTF-8"));
+      in = new BufferedReader(fReader);
+    } catch (FileNotFoundException f) {
+      // shouldn't happen....
+      LOG.warn("Couldn't read " + procfsMemFile
+          + "; can't determine memory settings");
+      return;
+    }
+
+    Matcher mat;
+
+    try {
+      String str = in.readLine();
+      while (str != null) {
+        mat = PROCFS_MEMFILE_FORMAT.matcher(str);
+        if (mat.find()) {
+          if (mat.group(1).equals(MEMTOTAL_STRING)) {
+            ramSize = Long.parseLong(mat.group(2));
+          } else if (mat.group(1).equals(SWAPTOTAL_STRING)) {
+            swapSize = Long.parseLong(mat.group(2));
+          } else if (mat.group(1).equals(MEMFREE_STRING)) {
+            ramSizeFree = Long.parseLong(mat.group(2));
+          } else if (mat.group(1).equals(SWAPFREE_STRING)) {
+            swapSizeFree = Long.parseLong(mat.group(2));
+          } else if (mat.group(1).equals(INACTIVE_STRING)) {
+            inactiveSize = Long.parseLong(mat.group(2));
+          }
+        }
+        str = in.readLine();
+      }
+    } catch (IOException io) {
+      LOG.warn("Error reading the stream " + io);
+    } finally {
+      // Close the streams
+      try {
+        fReader.close();
+        try {
+          in.close();
+        } catch (IOException i) {
+          LOG.warn("Error closing the stream " + in);
+        }
+      } catch (IOException i) {
+        LOG.warn("Error closing the stream " + fReader);
+      }
+    }
+
+    readMemInfoFile = true;
+  }
+
+  /**
+   * Read /proc/cpuinfo, parse and calculate CPU information.
+   */
+  private void readProcCpuInfoFile() {
+    // This directory needs to be read only once
+    if (readCpuInfoFile) {
+      return;
+    }
+    HashSet<String> coreIdSet = new HashSet<>();
+    // Read "/proc/cpuinfo" file
+    BufferedReader in;
+    InputStreamReader fReader;
+    try {
+      fReader = new InputStreamReader(
+          new FileInputStream(procfsCpuFile), Charset.forName("UTF-8"));
+      in = new BufferedReader(fReader);
+    } catch (FileNotFoundException f) {
+      // shouldn't happen....
+      LOG.warn("Couldn't read " + procfsCpuFile + "; can't determine cpu info");
+      return;
+    }
+    Matcher mat;
+    try {
+      numProcessors = 0;
+      numCores = 1;
+      String currentPhysicalId = "";
+      String str = in.readLine();
+      while (str != null) {
+        mat = PROCESSOR_FORMAT.matcher(str);
+        if (mat.find()) {
+          numProcessors++;
+        }
+        mat = FREQUENCY_FORMAT.matcher(str);
+        if (mat.find()) {
+          cpuFrequency = (long)(Double.parseDouble(mat.group(1)) * 1000); // kHz
+        }
+        mat = PHYSICAL_ID_FORMAT.matcher(str);
+        if (mat.find()) {
+          currentPhysicalId = str;
+        }
+        mat = CORE_ID_FORMAT.matcher(str);
+        if (mat.find()) {
+          coreIdSet.add(currentPhysicalId + " " + str);
+          numCores = coreIdSet.size();
+        }
+        str = in.readLine();
+      }
+    } catch (IOException io) {
+      LOG.warn("Error reading the stream " + io);
+    } finally {
+      // Close the streams
+      try {
+        fReader.close();
+        try {
+          in.close();
+        } catch (IOException i) {
+          LOG.warn("Error closing the stream " + in);
+        }
+      } catch (IOException i) {
+        LOG.warn("Error closing the stream " + fReader);
+      }
+    }
+    readCpuInfoFile = true;
+  }
+
+  /**
+   * Read /proc/stat file, parse and calculate cumulative CPU.
+   */
+  private void readProcStatFile() {
+    // Read "/proc/stat" file
+    BufferedReader in;
+    InputStreamReader fReader;
+    try {
+      fReader = new InputStreamReader(
+          new FileInputStream(procfsStatFile), Charset.forName("UTF-8"));
+      in = new BufferedReader(fReader);
+    } catch (FileNotFoundException f) {
+      // shouldn't happen....
+      return;
+    }
+
+    Matcher mat;
+    try {
+      String str = in.readLine();
+      while (str != null) {
+        mat = CPU_TIME_FORMAT.matcher(str);
+        if (mat.find()) {
+          long uTime = Long.parseLong(mat.group(1));
+          long nTime = Long.parseLong(mat.group(2));
+          long sTime = Long.parseLong(mat.group(3));
+          cpuTimeTracker.updateElapsedJiffies(
+              BigInteger.valueOf(uTime + nTime + sTime),
+              getCurrentTime());
+          break;
+        }
+        str = in.readLine();
+      }
+    } catch (IOException io) {
+      LOG.warn("Error reading the stream " + io);
+    } finally {
+      // Close the streams
+      try {
+        fReader.close();
+        try {
+          in.close();
+        } catch (IOException i) {
+          LOG.warn("Error closing the stream " + in);
+        }
+      } catch (IOException i) {
+        LOG.warn("Error closing the stream " + fReader);
+      }
+    }
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getPhysicalMemorySize() {
+    readProcMemInfoFile();
+    return ramSize * 1024;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getVirtualMemorySize() {
+    readProcMemInfoFile();
+    return (ramSize + swapSize) * 1024;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getAvailablePhysicalMemorySize() {
+    readProcMemInfoFile(true);
+    return (ramSizeFree + inactiveSize) * 1024;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getAvailableVirtualMemorySize() {
+    readProcMemInfoFile(true);
+    return (ramSizeFree + swapSizeFree + inactiveSize) * 1024;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public int getNumProcessors() {
+    readProcCpuInfoFile();
+    return numProcessors;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public int getNumCores() {
+    readProcCpuInfoFile();
+    return numCores;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getCpuFrequency() {
+    readProcCpuInfoFile();
+    return cpuFrequency;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getCumulativeCpuTime() {
+    readProcStatFile();
+    return cpuTimeTracker.getCumulativeCpuTime();
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public float getCpuUsage() {
+    readProcStatFile();
+    float overallCpuUsage = cpuTimeTracker.getCpuTrackerUsagePercent();
+    if (overallCpuUsage != CpuTimeTracker.UNAVAILABLE) {
+      overallCpuUsage = overallCpuUsage / getNumProcessors();
+    }
+    return overallCpuUsage;
+  }
+
+  /**
+   * Test the {@link SysInfoLinux}.
+   *
+   * @param args - arguments to this calculator test
+   */
+  public static void main(String[] args) {
+    SysInfoLinux plugin = new SysInfoLinux();
+    System.out.println("Physical memory Size (bytes) : "
+        + plugin.getPhysicalMemorySize());
+    System.out.println("Total Virtual memory Size (bytes) : "
+        + plugin.getVirtualMemorySize());
+    System.out.println("Available Physical memory Size (bytes) : "
+        + plugin.getAvailablePhysicalMemorySize());
+    System.out.println("Total Available Virtual memory Size (bytes) : "
+        + plugin.getAvailableVirtualMemorySize());
+    System.out.println("Number of Processors : " + plugin.getNumProcessors());
+    System.out.println("CPU frequency (kHz) : " + plugin.getCpuFrequency());
+    System.out.println("Cumulative CPU time (ms) : " +
+            plugin.getCumulativeCpuTime());
+    try {
+      // Sleep so we can compute the CPU usage
+      Thread.sleep(500L);
+    } catch (InterruptedException e) {
+      // do nothing
+    }
+    System.out.println("CPU usage % : " + plugin.getCpuUsage());
+  }
+
+  @VisibleForTesting
+  void setReadCpuInfoFile(boolean readCpuInfoFileValue) {
+    this.readCpuInfoFile = readCpuInfoFileValue;
+  }
+
+  public long getJiffyLengthInMillis() {
+    return this.jiffyLengthInMillis;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac604837/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java
new file mode 100644
index 0000000..da4c1c5
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java
@@ -0,0 +1,181 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.util;
+
+import java.io.IOException;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.util.Shell.ShellCommandExecutor;
+
+/**
+ * Plugin to calculate resource information on Windows systems.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class SysInfoWindows extends SysInfo {
+
+  private static final Log LOG = LogFactory.getLog(SysInfoWindows.class);
+
+  private long vmemSize;
+  private long memSize;
+  private long vmemAvailable;
+  private long memAvailable;
+  private int numProcessors;
+  private long cpuFrequencyKhz;
+  private long cumulativeCpuTimeMs;
+  private float cpuUsage;
+
+  private long lastRefreshTime;
+  static final int REFRESH_INTERVAL_MS = 1000;
+
+  public SysInfoWindows() {
+    lastRefreshTime = 0;
+    reset();
+  }
+
+  @VisibleForTesting
+  long now() {
+    return System.nanoTime();
+  }
+
+  void reset() {
+    vmemSize = -1;
+    memSize = -1;
+    vmemAvailable = -1;
+    memAvailable = -1;
+    numProcessors = -1;
+    cpuFrequencyKhz = -1;
+    cumulativeCpuTimeMs = -1;
+    cpuUsage = -1;
+  }
+
+  String getSystemInfoInfoFromShell() {
+    ShellCommandExecutor shellExecutor = new ShellCommandExecutor(
+        new String[] {Shell.WINUTILS, "systeminfo" });
+    try {
+      shellExecutor.execute();
+      return shellExecutor.getOutput();
+    } catch (IOException e) {
+      LOG.error(StringUtils.stringifyException(e));
+    }
+    return null;
+  }
+
+  void refreshIfNeeded() {
+    long now = now();
+    if (now - lastRefreshTime > REFRESH_INTERVAL_MS) {
+      long refreshInterval = now - lastRefreshTime;
+      lastRefreshTime = now;
+      long lastCumCpuTimeMs = cumulativeCpuTimeMs;
+      reset();
+      String sysInfoStr = getSystemInfoInfoFromShell();
+      if (sysInfoStr != null) {
+        final int sysInfoSplitCount = 7;
+        String[] sysInfo = sysInfoStr.substring(0, sysInfoStr.indexOf("\r\n"))
+            .split(",");
+        if (sysInfo.length == sysInfoSplitCount) {
+          try {
+            vmemSize = Long.parseLong(sysInfo[0]);
+            memSize = Long.parseLong(sysInfo[1]);
+            vmemAvailable = Long.parseLong(sysInfo[2]);
+            memAvailable = Long.parseLong(sysInfo[3]);
+            numProcessors = Integer.parseInt(sysInfo[4]);
+            cpuFrequencyKhz = Long.parseLong(sysInfo[5]);
+            cumulativeCpuTimeMs = Long.parseLong(sysInfo[6]);
+            if (lastCumCpuTimeMs != -1) {
+              cpuUsage = (cumulativeCpuTimeMs - lastCumCpuTimeMs)
+                  / (refreshInterval * 1.0f);
+            }
+          } catch (NumberFormatException nfe) {
+            LOG.warn("Error parsing sysInfo", nfe);
+          }
+        } else {
+          LOG.warn("Expected split length of sysInfo to be "
+              + sysInfoSplitCount + ". Got " + sysInfo.length);
+        }
+      }
+    }
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getVirtualMemorySize() {
+    refreshIfNeeded();
+    return vmemSize;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getPhysicalMemorySize() {
+    refreshIfNeeded();
+    return memSize;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getAvailableVirtualMemorySize() {
+    refreshIfNeeded();
+    return vmemAvailable;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getAvailablePhysicalMemorySize() {
+    refreshIfNeeded();
+    return memAvailable;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public int getNumProcessors() {
+    refreshIfNeeded();
+    return numProcessors;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public int getNumCores() {
+    return getNumProcessors();
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getCpuFrequency() {
+    refreshIfNeeded();
+    return cpuFrequencyKhz;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getCumulativeCpuTime() {
+    refreshIfNeeded();
+    return cumulativeCpuTimeMs;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public float getCpuUsage() {
+    refreshIfNeeded();
+    return cpuUsage;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac604837/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoLinux.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoLinux.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoLinux.java
new file mode 100644
index 0000000..73edc77
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoLinux.java
@@ -0,0 +1,323 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.util;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * A JUnit test to test {@link SysInfoLinux}
+ * Create the fake /proc/ information and verify the parsing and calculation
+ */
+public class TestSysInfoLinux {
+  /**
+   * LinuxResourceCalculatorPlugin with a fake timer
+   */
+  static class FakeLinuxResourceCalculatorPlugin extends
+      SysInfoLinux {
+
+    long currentTime = 0;
+    public FakeLinuxResourceCalculatorPlugin(String procfsMemFile,
+                                             String procfsCpuFile,
+                                             String procfsStatFile,
+                                             long jiffyLengthInMillis) {
+      super(procfsMemFile, procfsCpuFile, procfsStatFile, jiffyLengthInMillis);
+    }
+    @Override
+    long getCurrentTime() {
+      return currentTime;
+    }
+    public void advanceTime(long adv) {
+      currentTime += adv * this.getJiffyLengthInMillis();
+    }
+  }
+  private static final FakeLinuxResourceCalculatorPlugin plugin;
+  private static String TEST_ROOT_DIR = new Path(System.getProperty(
+         "test.build.data", "/tmp")).toString().replace(' ', '+');
+  private static final String FAKE_MEMFILE;
+  private static final String FAKE_CPUFILE;
+  private static final String FAKE_STATFILE;
+  private static final long FAKE_JIFFY_LENGTH = 10L;
+  static {
+    int randomNum = (new Random()).nextInt(1000000000);
+    FAKE_MEMFILE = TEST_ROOT_DIR + File.separator + "MEMINFO_" + randomNum;
+    FAKE_CPUFILE = TEST_ROOT_DIR + File.separator + "CPUINFO_" + randomNum;
+    FAKE_STATFILE = TEST_ROOT_DIR + File.separator + "STATINFO_" + randomNum;
+    plugin = new FakeLinuxResourceCalculatorPlugin(FAKE_MEMFILE, FAKE_CPUFILE,
+                                                   FAKE_STATFILE,
+                                                   FAKE_JIFFY_LENGTH);
+  }
+  static final String MEMINFO_FORMAT =
+    "MemTotal:      %d kB\n" +
+    "MemFree:         %d kB\n" +
+    "Buffers:        138244 kB\n" +
+    "Cached:         947780 kB\n" +
+    "SwapCached:     142880 kB\n" +
+    "Active:        3229888 kB\n" +
+    "Inactive:       %d kB\n" +
+    "SwapTotal:     %d kB\n" +
+    "SwapFree:      %d kB\n" +
+    "Dirty:          122012 kB\n" +
+    "Writeback:           0 kB\n" +
+    "AnonPages:     2710792 kB\n" +
+    "Mapped:          24740 kB\n" +
+    "Slab:           132528 kB\n" +
+    "SReclaimable:   105096 kB\n" +
+    "SUnreclaim:      27432 kB\n" +
+    "PageTables:      11448 kB\n" +
+    "NFS_Unstable:        0 kB\n" +
+    "Bounce:              0 kB\n" +
+    "CommitLimit:   4125904 kB\n" +
+    "Committed_AS:  4143556 kB\n" +
+    "VmallocTotal: 34359738367 kB\n" +
+    "VmallocUsed:      1632 kB\n" +
+    "VmallocChunk: 34359736375 kB\n" +
+    "HugePages_Total:     0\n" +
+    "HugePages_Free:      0\n" +
+    "HugePages_Rsvd:      0\n" +
+    "Hugepagesize:     2048 kB";
+
+  static final String CPUINFO_FORMAT =
+    "processor : %s\n" +
+    "vendor_id : AuthenticAMD\n" +
+    "cpu family  : 15\n" +
+    "model   : 33\n" +
+    "model name  : Dual Core AMD Opteron(tm) Processor 280\n" +
+    "stepping  : 2\n" +
+    "cpu MHz   : %f\n" +
+    "cache size  : 1024 KB\n" +
+    "physical id : %s\n" +
+    "siblings  : 2\n" +
+    "core id   : %s\n" +
+    "cpu cores : 2\n" +
+    "fpu   : yes\n" +
+    "fpu_exception : yes\n" +
+    "cpuid level : 1\n" +
+    "wp    : yes\n" +
+    "flags   : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov " +
+    "pat pse36 clflush mmx fxsr sse sse2 ht syscall nx mmxext fxsr_opt lm " +
+    "3dnowext 3dnow pni lahf_lm cmp_legacy\n" +
+    "bogomips  : 4792.41\n" +
+    "TLB size  : 1024 4K pages\n" +
+    "clflush size  : 64\n" +
+    "cache_alignment : 64\n" +
+    "address sizes : 40 bits physical, 48 bits virtual\n" +
+    "power management: ts fid vid ttp";
+
+  static final String STAT_FILE_FORMAT =
+    "cpu  %d %d %d 1646495089 831319 48713 164346 0\n" +
+    "cpu0 15096055 30805 3823005 411456015 206027 13 14269 0\n" +
+    "cpu1 14760561 89890 6432036 408707910 456857 48074 130857 0\n" +
+    "cpu2 12761169 20842 3758639 413976772 98028 411 10288 0\n" +
+    "cpu3 12355207 47322 5789691 412354390 70406 213 8931 0\n" +
+    "intr 114648668 20010764 2 0 945665 2 0 0 0 0 0 0 0 4 0 0 0 0 0 0\n" +
+    "ctxt 242017731764\n" +
+    "btime 1257808753\n" +
+    "processes 26414943\n" +
+    "procs_running 1\n" +
+    "procs_blocked 0\n";
+
+  /**
+   * Test parsing /proc/stat and /proc/cpuinfo
+   * @throws IOException
+   */
+  @Test
+  public void parsingProcStatAndCpuFile() throws IOException {
+    // Write fake /proc/cpuinfo file.
+    long numProcessors = 8;
+    long cpuFrequencyKHz = 2392781;
+    String fileContent = "";
+    for (int i = 0; i < numProcessors; i++) {
+      fileContent +=
+          String.format(CPUINFO_FORMAT, i, cpuFrequencyKHz / 1000D, 0, 0)
+              + "\n";
+    }
+    File tempFile = new File(FAKE_CPUFILE);
+    tempFile.deleteOnExit();
+    FileWriter fWriter = new FileWriter(FAKE_CPUFILE);
+    fWriter.write(fileContent);
+    fWriter.close();
+    assertEquals(plugin.getNumProcessors(), numProcessors);
+    assertEquals(plugin.getCpuFrequency(), cpuFrequencyKHz);
+
+    // Write fake /proc/stat file.
+    long uTime = 54972994;
+    long nTime = 188860;
+    long sTime = 19803373;
+    tempFile = new File(FAKE_STATFILE);
+    tempFile.deleteOnExit();
+    updateStatFile(uTime, nTime, sTime);
+    assertEquals(plugin.getCumulativeCpuTime(),
+                 FAKE_JIFFY_LENGTH * (uTime + nTime + sTime));
+    assertEquals(plugin.getCpuUsage(), (float)(CpuTimeTracker.UNAVAILABLE),0.0);
+
+    // Advance the time and sample again to test the CPU usage calculation
+    uTime += 100L;
+    plugin.advanceTime(200L);
+    updateStatFile(uTime, nTime, sTime);
+    assertEquals(plugin.getCumulativeCpuTime(),
+                 FAKE_JIFFY_LENGTH * (uTime + nTime + sTime));
+    assertEquals(plugin.getCpuUsage(), 6.25F, 0.0);
+
+    // Advance the time and sample again. This time, we call getCpuUsage() only.
+    uTime += 600L;
+    plugin.advanceTime(300L);
+    updateStatFile(uTime, nTime, sTime);
+    assertEquals(plugin.getCpuUsage(), 25F, 0.0);
+
+    // Advance very short period of time (one jiffy length).
+    // In this case, CPU usage should not be updated.
+    uTime += 1L;
+    plugin.advanceTime(1L);
+    updateStatFile(uTime, nTime, sTime);
+    assertEquals(plugin.getCumulativeCpuTime(),
+                 FAKE_JIFFY_LENGTH * (uTime + nTime + sTime));
+    assertEquals(plugin.getCpuUsage(), 25F, 0.0); // CPU usage is not updated.
+  }
+
+  /**
+   * Write information to fake /proc/stat file
+   */
+  private void updateStatFile(long uTime, long nTime, long sTime)
+    throws IOException {
+    FileWriter fWriter = new FileWriter(FAKE_STATFILE);
+    fWriter.write(String.format(STAT_FILE_FORMAT, uTime, nTime, sTime));
+    fWriter.close();
+  }
+
+  /**
+   * Test parsing /proc/meminfo
+   * @throws IOException
+   */
+  @Test
+  public void parsingProcMemFile() throws IOException {
+    long memTotal = 4058864L;
+    long memFree = 99632L;
+    long inactive = 567732L;
+    long swapTotal = 2096472L;
+    long swapFree = 1818480L;
+    File tempFile = new File(FAKE_MEMFILE);
+    tempFile.deleteOnExit();
+    FileWriter fWriter = new FileWriter(FAKE_MEMFILE);
+    fWriter.write(String.format(MEMINFO_FORMAT,
+      memTotal, memFree, inactive, swapTotal, swapFree));
+
+    fWriter.close();
+    assertEquals(plugin.getAvailablePhysicalMemorySize(),
+                 1024L * (memFree + inactive));
+    assertEquals(plugin.getAvailableVirtualMemorySize(),
+                 1024L * (memFree + inactive + swapFree));
+    assertEquals(plugin.getPhysicalMemorySize(), 1024L * memTotal);
+    assertEquals(plugin.getVirtualMemorySize(), 1024L * (memTotal + swapTotal));
+  }
+
+  @Test
+  public void testCoreCounts() throws IOException {
+
+    String fileContent = "";
+    // single core, hyper threading
+    long numProcessors = 2;
+    long cpuFrequencyKHz = 2392781;
+    for (int i = 0; i < numProcessors; i++) {
+      fileContent =
+          fileContent.concat(String.format(CPUINFO_FORMAT, i,
+            cpuFrequencyKHz / 1000D, 0, 0));
+      fileContent = fileContent.concat("\n");
+    }
+    writeFakeCPUInfoFile(fileContent);
+    plugin.setReadCpuInfoFile(false);
+    assertEquals(numProcessors, plugin.getNumProcessors());
+    assertEquals(1, plugin.getNumCores());
+
+    // single socket quad core, no hyper threading
+    fileContent = "";
+    numProcessors = 4;
+    for (int i = 0; i < numProcessors; i++) {
+      fileContent =
+          fileContent.concat(String.format(CPUINFO_FORMAT, i,
+            cpuFrequencyKHz / 1000D, 0, i));
+      fileContent = fileContent.concat("\n");
+    }
+    writeFakeCPUInfoFile(fileContent);
+    plugin.setReadCpuInfoFile(false);
+    assertEquals(numProcessors, plugin.getNumProcessors());
+    assertEquals(4, plugin.getNumCores());
+
+    // dual socket single core, hyper threading
+    fileContent = "";
+    numProcessors = 4;
+    for (int i = 0; i < numProcessors; i++) {
+      fileContent =
+          fileContent.concat(String.format(CPUINFO_FORMAT, i,
+            cpuFrequencyKHz / 1000D, i / 2, 0));
+      fileContent = fileContent.concat("\n");
+    }
+    writeFakeCPUInfoFile(fileContent);
+    plugin.setReadCpuInfoFile(false);
+    assertEquals(numProcessors, plugin.getNumProcessors());
+    assertEquals(2, plugin.getNumCores());
+
+    // dual socket, dual core, no hyper threading
+    fileContent = "";
+    numProcessors = 4;
+    for (int i = 0; i < numProcessors; i++) {
+      fileContent =
+          fileContent.concat(String.format(CPUINFO_FORMAT, i,
+            cpuFrequencyKHz / 1000D, i / 2, i % 2));
+      fileContent = fileContent.concat("\n");
+    }
+    writeFakeCPUInfoFile(fileContent);
+    plugin.setReadCpuInfoFile(false);
+    assertEquals(numProcessors, plugin.getNumProcessors());
+    assertEquals(4, plugin.getNumCores());
+
+    // dual socket, dual core, hyper threading
+    fileContent = "";
+    numProcessors = 8;
+    for (int i = 0; i < numProcessors; i++) {
+      fileContent =
+          fileContent.concat(String.format(CPUINFO_FORMAT, i,
+            cpuFrequencyKHz / 1000D, i / 4, (i % 4) / 2));
+      fileContent = fileContent.concat("\n");
+    }
+    writeFakeCPUInfoFile(fileContent);
+    plugin.setReadCpuInfoFile(false);
+    assertEquals(numProcessors, plugin.getNumProcessors());
+    assertEquals(4, plugin.getNumCores());
+  }
+
+  private void writeFakeCPUInfoFile(String content) throws IOException {
+    File tempFile = new File(FAKE_CPUFILE);
+    FileWriter fWriter = new FileWriter(FAKE_CPUFILE);
+    tempFile.deleteOnExit();
+    try {
+      fWriter.write(content);
+    } finally {
+      IOUtils.closeQuietly(fWriter);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac604837/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoWindows.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoWindows.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoWindows.java
new file mode 100644
index 0000000..7924c02
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoWindows.java
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.util;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class TestSysInfoWindows {
+
+
+  static class SysInfoWindowsMock extends SysInfoWindows {
+    private long time = SysInfoWindows.REFRESH_INTERVAL_MS + 1;
+    private String infoStr = null;
+    void setSysinfoString(String infoStr) {
+      this.infoStr = infoStr;
+    }
+    void advance(long dur) {
+      time += dur;
+    }
+    @Override
+    String getSystemInfoInfoFromShell() {
+      return infoStr;
+    }
+    @Override
+    long now() {
+      return time;
+    }
+  }
+
+  @Test(timeout = 10000)
+  public void parseSystemInfoString() {
+    SysInfoWindowsMock tester = new SysInfoWindowsMock();
+    tester.setSysinfoString(
+        "17177038848,8589467648,15232745472,6400417792,1,2805000,6261812\r\n");
+    // info str derived from windows shell command has \r\n termination
+    assertEquals(17177038848L, tester.getVirtualMemorySize());
+    assertEquals(8589467648L, tester.getPhysicalMemorySize());
+    assertEquals(15232745472L, tester.getAvailableVirtualMemorySize());
+    assertEquals(6400417792L, tester.getAvailablePhysicalMemorySize());
+    assertEquals(1, tester.getNumProcessors());
+    assertEquals(1, tester.getNumCores());
+    assertEquals(2805000L, tester.getCpuFrequency());
+    assertEquals(6261812L, tester.getCumulativeCpuTime());
+    // undef on first call
+    assertEquals(-1.0, tester.getCpuUsage(), 0.0);
+  }
+
+  @Test(timeout = 10000)
+  public void refreshAndCpuUsage() throws InterruptedException {
+    SysInfoWindowsMock tester = new SysInfoWindowsMock();
+    tester.setSysinfoString(
+        "17177038848,8589467648,15232745472,6400417792,1,2805000,6261812\r\n");
+    // info str derived from windows shell command has \r\n termination
+    tester.getAvailablePhysicalMemorySize();
+    // verify information has been refreshed
+    assertEquals(6400417792L, tester.getAvailablePhysicalMemorySize());
+    assertEquals(-1.0, tester.getCpuUsage(), 0.0);
+
+    tester.setSysinfoString(
+        "17177038848,8589467648,15232745472,5400417792,1,2805000,6263012\r\n");
+    tester.getAvailablePhysicalMemorySize();
+    // verify information has not been refreshed
+    assertEquals(6400417792L, tester.getAvailablePhysicalMemorySize());
+    assertEquals(-1.0, tester.getCpuUsage(), 0.0);
+
+    // advance clock
+    tester.advance(SysInfoWindows.REFRESH_INTERVAL_MS + 1);
+
+    // verify information has been refreshed
+    assertEquals(5400417792L, tester.getAvailablePhysicalMemorySize());
+    assertEquals((6263012 - 6261812) / (SysInfoWindows.REFRESH_INTERVAL_MS + 1f),
+        tester.getCpuUsage(), 0.0);
+  }
+
+  @Test(timeout = 10000)
+  public void errorInGetSystemInfo() {
+    SysInfoWindowsMock tester = new SysInfoWindowsMock();
+    // info str derived from windows shell command has \r\n termination
+    tester.setSysinfoString(null);
+    // call a method to refresh values
+    tester.getAvailablePhysicalMemorySize();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac604837/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/CpuTimeTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/CpuTimeTracker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/CpuTimeTracker.java
deleted file mode 100644
index b09a4b6..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/CpuTimeTracker.java
+++ /dev/null
@@ -1,100 +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.util;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-import java.math.BigInteger;
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class CpuTimeTracker {
-  public static final int UNAVAILABLE =
-      ResourceCalculatorProcessTree.UNAVAILABLE;
-  final long MINIMUM_UPDATE_INTERVAL;
-
-  // CPU used time since system is on (ms)
-  BigInteger cumulativeCpuTime = BigInteger.ZERO;
-
-  // CPU used time read last time (ms)
-  BigInteger lastCumulativeCpuTime = BigInteger.ZERO;
-
-  // Unix timestamp while reading the CPU time (ms)
-  long sampleTime;
-  long lastSampleTime;
-  float cpuUsage;
-  BigInteger jiffyLengthInMillis;
-
-  public CpuTimeTracker(long jiffyLengthInMillis) {
-    this.jiffyLengthInMillis = BigInteger.valueOf(jiffyLengthInMillis);
-    this.cpuUsage = UNAVAILABLE;
-    this.sampleTime = UNAVAILABLE;
-    this.lastSampleTime = UNAVAILABLE;
-    MINIMUM_UPDATE_INTERVAL =  10 * jiffyLengthInMillis;
-  }
-
-  /**
-   * Return percentage of cpu time spent over the time since last update.
-   * CPU time spent is based on elapsed jiffies multiplied by amount of
-   * time for 1 core. Thus, if you use 2 cores completely you would have spent
-   * twice the actual time between updates and this will return 200%.
-   *
-   * @return Return percentage of cpu usage since last update, {@link
-   * CpuTimeTracker#UNAVAILABLE} if there haven't been 2 updates more than
-   * {@link CpuTimeTracker#MINIMUM_UPDATE_INTERVAL} apart
-   */
-  public float getCpuTrackerUsagePercent() {
-    if (lastSampleTime == UNAVAILABLE ||
-        lastSampleTime > sampleTime) {
-      // lastSampleTime > sampleTime may happen when the system time is changed
-      lastSampleTime = sampleTime;
-      lastCumulativeCpuTime = cumulativeCpuTime;
-      return cpuUsage;
-    }
-    // When lastSampleTime is sufficiently old, update cpuUsage.
-    // Also take a sample of the current time and cumulative CPU time for the
-    // use of the next calculation.
-    if (sampleTime > lastSampleTime + MINIMUM_UPDATE_INTERVAL) {
-      cpuUsage =
-          ((cumulativeCpuTime.subtract(lastCumulativeCpuTime)).floatValue())
-          * 100F / ((float) (sampleTime - lastSampleTime));
-      lastSampleTime = sampleTime;
-      lastCumulativeCpuTime = cumulativeCpuTime;
-    }
-    return cpuUsage;
-  }
-
-  public void updateElapsedJiffies(BigInteger elapedJiffies, long sampleTime) {
-    this.cumulativeCpuTime = elapedJiffies.multiply(jiffyLengthInMillis);
-    this.sampleTime = sampleTime;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("SampleTime " + this.sampleTime);
-    sb.append(" CummulativeCpuTime " + this.cumulativeCpuTime);
-    sb.append(" LastSampleTime " + this.lastSampleTime);
-    sb.append(" LastCummulativeCpuTime " + this.lastCumulativeCpuTime);
-    sb.append(" CpuUsage " + this.cpuUsage);
-    sb.append(" JiffyLengthMillisec " + this.jiffyLengthInMillis);
-    return sb.toString();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac604837/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/LinuxResourceCalculatorPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/LinuxResourceCalculatorPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/LinuxResourceCalculatorPlugin.java
index bf4cfa4..f458f16 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/LinuxResourceCalculatorPlugin.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/LinuxResourceCalculatorPlugin.java
@@ -15,25 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.yarn.util;
 
-import java.io.BufferedReader;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.InputStreamReader;
-import java.io.IOException;
-import java.math.BigInteger;
-import java.nio.charset.Charset;
-import java.util.HashSet;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.util.SysInfoLinux;
 
 /**
  * Plugin to calculate resource information on Linux systems.
@@ -41,383 +27,9 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class LinuxResourceCalculatorPlugin extends ResourceCalculatorPlugin {
-  private static final Log LOG =
-      LogFactory.getLog(LinuxResourceCalculatorPlugin.class);
-
-  /**
-   * proc's meminfo virtual file has keys-values in the format
-   * "key:[ \t]*value[ \t]kB".
-   */
-  private static final String PROCFS_MEMFILE = "/proc/meminfo";
-  private static final Pattern PROCFS_MEMFILE_FORMAT =
-      Pattern.compile("^([a-zA-Z]*):[ \t]*([0-9]*)[ \t]kB");
-
-  // We need the values for the following keys in meminfo
-  private static final String MEMTOTAL_STRING = "MemTotal";
-  private static final String SWAPTOTAL_STRING = "SwapTotal";
-  private static final String MEMFREE_STRING = "MemFree";
-  private static final String SWAPFREE_STRING = "SwapFree";
-  private static final String INACTIVE_STRING = "Inactive";
-
-  /**
-   * Patterns for parsing /proc/cpuinfo.
-   */
-  private static final String PROCFS_CPUINFO = "/proc/cpuinfo";
-  private static final Pattern PROCESSOR_FORMAT =
-      Pattern.compile("^processor[ \t]:[ \t]*([0-9]*)");
-  private static final Pattern FREQUENCY_FORMAT =
-      Pattern.compile("^cpu MHz[ \t]*:[ \t]*([0-9.]*)");
-  private static final Pattern PHYSICAL_ID_FORMAT =
-      Pattern.compile("^physical id[ \t]*:[ \t]*([0-9]*)");
-  private static final Pattern CORE_ID_FORMAT =
-      Pattern.compile("^core id[ \t]*:[ \t]*([0-9]*)");
-
-  /**
-   * Pattern for parsing /proc/stat.
-   */
-  private static final String PROCFS_STAT = "/proc/stat";
-  private static final Pattern CPU_TIME_FORMAT =
-      Pattern.compile("^cpu[ \t]*([0-9]*)" +
-    		            "[ \t]*([0-9]*)[ \t]*([0-9]*)[ \t].*");
-  private CpuTimeTracker cpuTimeTracker;
-
-  private String procfsMemFile;
-  private String procfsCpuFile;
-  private String procfsStatFile;
-  private long jiffyLengthInMillis;
-
-  private long ramSize = 0;
-  private long swapSize = 0;
-  private long ramSizeFree = 0;  // free ram space on the machine (kB)
-  private long swapSizeFree = 0; // free swap space on the machine (kB)
-  private long inactiveSize = 0; // inactive cache memory (kB)
-  /* number of logical processors on the system. */
-  private int numProcessors = 0;
-  /* number of physical cores on the system. */
-  private int numCores = 0;
-  private long cpuFrequency = 0L; // CPU frequency on the system (kHz)
-
-  private boolean readMemInfoFile = false;
-  private boolean readCpuInfoFile = false;
-
-  /**
-   * Get current time.
-   * @return Unix time stamp in millisecond
-   */
-  long getCurrentTime() {
-    return System.currentTimeMillis();
-  }
 
   public LinuxResourceCalculatorPlugin() {
-    this(PROCFS_MEMFILE, PROCFS_CPUINFO, PROCFS_STAT,
-        ProcfsBasedProcessTree.JIFFY_LENGTH_IN_MILLIS);
-  }
-
-  /**
-   * Constructor which allows assigning the /proc/ directories. This will be
-   * used only in unit tests.
-   * @param procfsMemFile fake file for /proc/meminfo
-   * @param procfsCpuFile fake file for /proc/cpuinfo
-   * @param procfsStatFile fake file for /proc/stat
-   * @param jiffyLengthInMillis fake jiffy length value
-   */
-  public LinuxResourceCalculatorPlugin(String procfsMemFile,
-                                       String procfsCpuFile,
-                                       String procfsStatFile,
-                                       long jiffyLengthInMillis) {
-    this.procfsMemFile = procfsMemFile;
-    this.procfsCpuFile = procfsCpuFile;
-    this.procfsStatFile = procfsStatFile;
-    this.jiffyLengthInMillis = jiffyLengthInMillis;
-    this.cpuTimeTracker = new CpuTimeTracker(jiffyLengthInMillis);
-  }
-
-  /**
-   * Read /proc/meminfo, parse and compute memory information only once.
-   */
-  private void readProcMemInfoFile() {
-    readProcMemInfoFile(false);
-  }
-
-  /**
-   * Read /proc/meminfo, parse and compute memory information.
-   * @param readAgain if false, read only on the first time
-   */
-  private void readProcMemInfoFile(boolean readAgain) {
-
-    if (readMemInfoFile && !readAgain) {
-      return;
-    }
-
-    // Read "/proc/memInfo" file
-    BufferedReader in;
-    InputStreamReader fReader;
-    try {
-      fReader = new InputStreamReader(
-          new FileInputStream(procfsMemFile), Charset.forName("UTF-8"));
-      in = new BufferedReader(fReader);
-    } catch (FileNotFoundException f) {
-      // shouldn't happen....
-      LOG.warn("Couldn't read " + procfsMemFile
-          + "; can't determine memory settings");
-      return;
-    }
-
-    Matcher mat;
-
-    try {
-      String str = in.readLine();
-      while (str != null) {
-        mat = PROCFS_MEMFILE_FORMAT.matcher(str);
-        if (mat.find()) {
-          if (mat.group(1).equals(MEMTOTAL_STRING)) {
-            ramSize = Long.parseLong(mat.group(2));
-          } else if (mat.group(1).equals(SWAPTOTAL_STRING)) {
-            swapSize = Long.parseLong(mat.group(2));
-          } else if (mat.group(1).equals(MEMFREE_STRING)) {
-            ramSizeFree = Long.parseLong(mat.group(2));
-          } else if (mat.group(1).equals(SWAPFREE_STRING)) {
-            swapSizeFree = Long.parseLong(mat.group(2));
-          } else if (mat.group(1).equals(INACTIVE_STRING)) {
-            inactiveSize = Long.parseLong(mat.group(2));
-          }
-        }
-        str = in.readLine();
-      }
-    } catch (IOException io) {
-      LOG.warn("Error reading the stream " + io);
-    } finally {
-      // Close the streams
-      try {
-        fReader.close();
-        try {
-          in.close();
-        } catch (IOException i) {
-          LOG.warn("Error closing the stream " + in);
-        }
-      } catch (IOException i) {
-        LOG.warn("Error closing the stream " + fReader);
-      }
-    }
-
-    readMemInfoFile = true;
-  }
-
-  /**
-   * Read /proc/cpuinfo, parse and calculate CPU information.
-   */
-  private void readProcCpuInfoFile() {
-    // This directory needs to be read only once
-    if (readCpuInfoFile) {
-      return;
-    }
-    HashSet<String> coreIdSet = new HashSet<>();
-    // Read "/proc/cpuinfo" file
-    BufferedReader in;
-    InputStreamReader fReader;
-    try {
-      fReader = new InputStreamReader(
-          new FileInputStream(procfsCpuFile), Charset.forName("UTF-8"));
-      in = new BufferedReader(fReader);
-    } catch (FileNotFoundException f) {
-      // shouldn't happen....
-      LOG.warn("Couldn't read " + procfsCpuFile + "; can't determine cpu info");
-      return;
-    }
-    Matcher mat;
-    try {
-      numProcessors = 0;
-      numCores = 1;
-      String currentPhysicalId = "";
-      String str = in.readLine();
-      while (str != null) {
-        mat = PROCESSOR_FORMAT.matcher(str);
-        if (mat.find()) {
-          numProcessors++;
-        }
-        mat = FREQUENCY_FORMAT.matcher(str);
-        if (mat.find()) {
-          cpuFrequency = (long)(Double.parseDouble(mat.group(1)) * 1000); // kHz
-        }
-        mat = PHYSICAL_ID_FORMAT.matcher(str);
-        if (mat.find()) {
-          currentPhysicalId = str;
-        }
-        mat = CORE_ID_FORMAT.matcher(str);
-        if (mat.find()) {
-          coreIdSet.add(currentPhysicalId + " " + str);
-          numCores = coreIdSet.size();
-        }
-        str = in.readLine();
-      }
-    } catch (IOException io) {
-      LOG.warn("Error reading the stream " + io);
-    } finally {
-      // Close the streams
-      try {
-        fReader.close();
-        try {
-          in.close();
-        } catch (IOException i) {
-          LOG.warn("Error closing the stream " + in);
-        }
-      } catch (IOException i) {
-        LOG.warn("Error closing the stream " + fReader);
-      }
-    }
-    readCpuInfoFile = true;
-  }
-
-  /**
-   * Read /proc/stat file, parse and calculate cumulative CPU.
-   */
-  private void readProcStatFile() {
-    // Read "/proc/stat" file
-    BufferedReader in;
-    InputStreamReader fReader;
-    try {
-      fReader = new InputStreamReader(
-          new FileInputStream(procfsStatFile), Charset.forName("UTF-8"));
-      in = new BufferedReader(fReader);
-    } catch (FileNotFoundException f) {
-      // shouldn't happen....
-      return;
-    }
-
-    Matcher mat;
-    try {
-      String str = in.readLine();
-      while (str != null) {
-        mat = CPU_TIME_FORMAT.matcher(str);
-        if (mat.find()) {
-          long uTime = Long.parseLong(mat.group(1));
-          long nTime = Long.parseLong(mat.group(2));
-          long sTime = Long.parseLong(mat.group(3));
-          cpuTimeTracker.updateElapsedJiffies(
-              BigInteger.valueOf(uTime + nTime + sTime),
-              getCurrentTime());
-          break;
-        }
-        str = in.readLine();
-      }
-    } catch (IOException io) {
-      LOG.warn("Error reading the stream " + io);
-    } finally {
-      // Close the streams
-      try {
-        fReader.close();
-        try {
-          in.close();
-        } catch (IOException i) {
-          LOG.warn("Error closing the stream " + in);
-        }
-      } catch (IOException i) {
-        LOG.warn("Error closing the stream " + fReader);
-      }
-    }
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public long getPhysicalMemorySize() {
-    readProcMemInfoFile();
-    return ramSize * 1024;
+    super(new SysInfoLinux());
   }
 
-  /** {@inheritDoc} */
-  @Override
-  public long getVirtualMemorySize() {
-    readProcMemInfoFile();
-    return (ramSize + swapSize) * 1024;
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public long getAvailablePhysicalMemorySize() {
-    readProcMemInfoFile(true);
-    return (ramSizeFree + inactiveSize) * 1024;
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public long getAvailableVirtualMemorySize() {
-    readProcMemInfoFile(true);
-    return (ramSizeFree + swapSizeFree + inactiveSize) * 1024;
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public int getNumProcessors() {
-    readProcCpuInfoFile();
-    return numProcessors;
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public int getNumCores() {
-    readProcCpuInfoFile();
-    return numCores;
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public long getCpuFrequency() {
-    readProcCpuInfoFile();
-    return cpuFrequency;
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public long getCumulativeCpuTime() {
-    readProcStatFile();
-    return cpuTimeTracker.cumulativeCpuTime.longValue();
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public float getCpuUsage() {
-    readProcStatFile();
-    float overallCpuUsage = cpuTimeTracker.getCpuTrackerUsagePercent();
-    if (overallCpuUsage != CpuTimeTracker.UNAVAILABLE) {
-      overallCpuUsage = overallCpuUsage / getNumProcessors();
-    }
-    return overallCpuUsage;
-  }
-
-  /**
-   * Test the {@link LinuxResourceCalculatorPlugin}.
-   *
-   * @param args - arguments to this calculator test
-   */
-  public static void main(String[] args) {
-    LinuxResourceCalculatorPlugin plugin = new LinuxResourceCalculatorPlugin();
-    System.out.println("Physical memory Size (bytes) : "
-        + plugin.getPhysicalMemorySize());
-    System.out.println("Total Virtual memory Size (bytes) : "
-        + plugin.getVirtualMemorySize());
-    System.out.println("Available Physical memory Size (bytes) : "
-        + plugin.getAvailablePhysicalMemorySize());
-    System.out.println("Total Available Virtual memory Size (bytes) : "
-        + plugin.getAvailableVirtualMemorySize());
-    System.out.println("Number of Processors : " + plugin.getNumProcessors());
-    System.out.println("CPU frequency (kHz) : " + plugin.getCpuFrequency());
-    System.out.println("Cumulative CPU time (ms) : " +
-            plugin.getCumulativeCpuTime());
-    try {
-      // Sleep so we can compute the CPU usage
-      Thread.sleep(500L);
-    } catch (InterruptedException e) {
-      // do nothing
-    }
-    System.out.println("CPU usage % : " + plugin.getCpuUsage());
-  }
-
-  @VisibleForTesting
-  void setReadCpuInfoFile(boolean readCpuInfoFileValue) {
-    this.readCpuInfoFile = readCpuInfoFileValue;
-  }
-
-  public long getJiffyLengthInMillis() {
-    return this.jiffyLengthInMillis;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac604837/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
index df9d28a..2345c62 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
@@ -40,9 +40,9 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.CpuTimeTracker;
 import org.apache.hadoop.util.Shell;
-import org.apache.hadoop.util.Shell.ShellCommandExecutor;
-import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.SysInfoLinux;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
 /**
@@ -64,8 +64,9 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
 
   public static final String PROCFS_STAT_FILE = "stat";
   public static final String PROCFS_CMDLINE_FILE = "cmdline";
-  public static final long PAGE_SIZE;
-  public static final long JIFFY_LENGTH_IN_MILLIS; // in millisecond
+  public static final long PAGE_SIZE = SysInfoLinux.PAGE_SIZE;
+  public static final long JIFFY_LENGTH_IN_MILLIS =
+      SysInfoLinux.JIFFY_LENGTH_IN_MILLIS; // in millisecond
   private final CpuTimeTracker cpuTimeTracker;
   private Clock clock;
 
@@ -108,31 +109,6 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
   protected Map<String, ProcessTreeSmapMemInfo> processSMAPTree =
       new HashMap<String, ProcessTreeSmapMemInfo>();
 
-  static {
-    long jiffiesPerSecond = -1;
-    long pageSize = -1;
-    try {
-      if(Shell.LINUX) {
-        ShellCommandExecutor shellExecutorClk = new ShellCommandExecutor(
-            new String[] { "getconf", "CLK_TCK" });
-        shellExecutorClk.execute();
-        jiffiesPerSecond = Long.parseLong(shellExecutorClk.getOutput().replace("\n", ""));
-
-        ShellCommandExecutor shellExecutorPage = new ShellCommandExecutor(
-            new String[] { "getconf", "PAGESIZE" });
-        shellExecutorPage.execute();
-        pageSize = Long.parseLong(shellExecutorPage.getOutput().replace("\n", ""));
-
-      }
-    } catch (IOException e) {
-      LOG.error(StringUtils.stringifyException(e));
-    } finally {
-      JIFFY_LENGTH_IN_MILLIS = jiffiesPerSecond != -1 ?
-                     Math.round(1000D / jiffiesPerSecond) : -1;
-                     PAGE_SIZE = pageSize;
-    }
-  }
-
   // to enable testing, using this variable which can be configured
   // to a test directory.
   private String procfsDir;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac604837/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
index 40bd44e..5e5f1b4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.yarn.util;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -23,29 +22,42 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.util.SysInfo;
 
 /**
  * Plugin to calculate resource information on the system.
- *
  */
 @InterfaceAudience.LimitedPrivate({"YARN", "MAPREDUCE"})
 @InterfaceStability.Unstable
-public abstract class ResourceCalculatorPlugin extends Configured {
+public class ResourceCalculatorPlugin extends Configured {
+
+  private final SysInfo sys;
+
+  protected ResourceCalculatorPlugin() {
+    this(SysInfo.newInstance());
+  }
+
+  public ResourceCalculatorPlugin(SysInfo sys) {
+    this.sys = sys;
+  }
 
   /**
    * Obtain the total size of the virtual memory present in the system.
    *
    * @return virtual memory size in bytes.
    */
-  public abstract long getVirtualMemorySize();
+  public long getVirtualMemorySize() {
+    return sys.getVirtualMemorySize();
+  }
 
   /**
    * Obtain the total size of the physical memory present in the system.
    *
    * @return physical memory size bytes.
    */
-  public abstract long getPhysicalMemorySize();
+  public long getPhysicalMemorySize() {
+    return sys.getPhysicalMemorySize();
+  }
 
   /**
    * Obtain the total size of the available virtual memory present
@@ -53,7 +65,9 @@ public abstract class ResourceCalculatorPlugin extends Configured {
    *
    * @return available virtual memory size in bytes.
    */
-  public abstract long getAvailableVirtualMemorySize();
+  public long getAvailableVirtualMemorySize() {
+    return sys.getAvailableVirtualMemorySize();
+  }
 
   /**
    * Obtain the total size of the available physical memory present
@@ -61,42 +75,54 @@ public abstract class ResourceCalculatorPlugin extends Configured {
    *
    * @return available physical memory size bytes.
    */
-  public abstract long getAvailablePhysicalMemorySize();
+  public long getAvailablePhysicalMemorySize() {
+    return sys.getAvailablePhysicalMemorySize();
+  }
 
   /**
    * Obtain the total number of logical processors present on the system.
    *
    * @return number of logical processors
    */
-  public abstract int getNumProcessors();
+  public int getNumProcessors() {
+    return sys.getNumProcessors();
+  }
 
   /**
    * Obtain total number of physical cores present on the system.
    *
    * @return number of physical cores
    */
-  public abstract int getNumCores();
+  public int getNumCores() {
+    return sys.getNumCores();
+  }
 
   /**
    * Obtain the CPU frequency of on the system.
    *
    * @return CPU frequency in kHz
    */
-  public abstract long getCpuFrequency();
+  public long getCpuFrequency() {
+    return sys.getCpuFrequency();
+  }
 
   /**
    * Obtain the cumulative CPU time since the system is on.
    *
    * @return cumulative CPU time in milliseconds
    */
-  public abstract long getCumulativeCpuTime();
+  public long getCumulativeCpuTime() {
+    return sys.getCumulativeCpuTime();
+  }
 
   /**
    * Obtain the CPU usage % of the machine. Return -1 if it is unavailable
    *
    * @return CPU usage in %
    */
-  public abstract float getCpuUsage();
+  public float getCpuUsage() {
+    return sys.getCpuUsage();
+  }
 
   /**
    * Create the ResourceCalculatorPlugin from the class name and configure it. If
@@ -114,21 +140,11 @@ public abstract class ResourceCalculatorPlugin extends Configured {
     if (clazz != null) {
       return ReflectionUtils.newInstance(clazz, conf);
     }
-
-    // No class given, try a os specific class
     try {
-      if (Shell.LINUX) {
-        return new LinuxResourceCalculatorPlugin();
-      }
-      if (Shell.WINDOWS) {
-        return new WindowsResourceCalculatorPlugin();
-      }
-    } catch (SecurityException se) {
-      // Failed to get Operating System name.
+      return new ResourceCalculatorPlugin();
+    } catch (SecurityException e) {
       return null;
     }
-
-    // Not supported on this system.
-    return null;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac604837/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java
index 7d9c7d3..ebe8df1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java
@@ -229,7 +229,7 @@ public class WindowsBasedProcessTree extends ResourceCalculatorProcessTree {
 
   @Override
   public float getCpuUsagePercent() {
-    return CpuTimeTracker.UNAVAILABLE;
+    return UNAVAILABLE;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac604837/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsResourceCalculatorPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsResourceCalculatorPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsResourceCalculatorPlugin.java
index cdbf525..f817b7a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsResourceCalculatorPlugin.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsResourceCalculatorPlugin.java
@@ -15,162 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.yarn.util;
 
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.util.Shell;
-import org.apache.hadoop.util.Shell.ShellCommandExecutor;
-import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.util.SysInfoWindows;
 
-@Private
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
 public class WindowsResourceCalculatorPlugin extends ResourceCalculatorPlugin {
-  
-  static final Log LOG = LogFactory
-      .getLog(WindowsResourceCalculatorPlugin.class);
-  
-  long vmemSize;
-  long memSize;
-  long vmemAvailable;
-  long memAvailable;
-  int numProcessors;
-  long cpuFrequencyKhz;
-  long cumulativeCpuTimeMs;
-  float cpuUsage;
-  
-  long lastRefreshTime;
-  private final int refreshIntervalMs = 1000;
-  
-  WindowsBasedProcessTree pTree = null;
-  
-  public WindowsResourceCalculatorPlugin() {
-    lastRefreshTime = 0;
-    reset();
-  }
-  
-  void reset() {
-    vmemSize = -1;
-    memSize = -1;
-    vmemAvailable = -1;
-    memAvailable = -1;
-    numProcessors = -1;
-    cpuFrequencyKhz = -1;
-    cumulativeCpuTimeMs = -1;
-    cpuUsage = -1;
-  }
-
-  String getSystemInfoInfoFromShell() {
-    ShellCommandExecutor shellExecutor = new ShellCommandExecutor(
-        new String[] { Shell.WINUTILS, "systeminfo" });
-    try {
-      shellExecutor.execute();
-      return shellExecutor.getOutput();
-    } catch (IOException e) {
-      LOG.error(StringUtils.stringifyException(e));
-    }
-    return null;
-  }
-  
-  void refreshIfNeeded() {
-    long now = System.currentTimeMillis();
-    if (now - lastRefreshTime > refreshIntervalMs) {
-      long refreshInterval = now - lastRefreshTime;
-      lastRefreshTime = now;
-      long lastCumCpuTimeMs = cumulativeCpuTimeMs;
-      reset();
-      String sysInfoStr = getSystemInfoInfoFromShell();
-      if (sysInfoStr != null) {
-        final int sysInfoSplitCount = 7;
-        String[] sysInfo = sysInfoStr.substring(0, sysInfoStr.indexOf("\r\n"))
-            .split(",");
-        if (sysInfo.length == sysInfoSplitCount) {
-          try {
-            vmemSize = Long.parseLong(sysInfo[0]);
-            memSize = Long.parseLong(sysInfo[1]);
-            vmemAvailable = Long.parseLong(sysInfo[2]);
-            memAvailable = Long.parseLong(sysInfo[3]);
-            numProcessors = Integer.parseInt(sysInfo[4]);
-            cpuFrequencyKhz = Long.parseLong(sysInfo[5]);
-            cumulativeCpuTimeMs = Long.parseLong(sysInfo[6]);
-            if (lastCumCpuTimeMs != -1) {
-              cpuUsage = (cumulativeCpuTimeMs - lastCumCpuTimeMs)
-                  / (refreshInterval * 1.0f);
-            }
-
-          } catch (NumberFormatException nfe) {
-            LOG.warn("Error parsing sysInfo." + nfe);
-          }
-        } else {
-          LOG.warn("Expected split length of sysInfo to be "
-              + sysInfoSplitCount + ". Got " + sysInfo.length);
-        }
-      }
-    }
-  }
-  
-  /** {@inheritDoc} */
-  @Override
-  public long getVirtualMemorySize() {
-    refreshIfNeeded();
-    return vmemSize;
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public long getPhysicalMemorySize() {
-    refreshIfNeeded();
-    return memSize;
-  }
 
-  /** {@inheritDoc} */
-  @Override
-  public long getAvailableVirtualMemorySize() {
-    refreshIfNeeded();
-    return vmemAvailable;
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public long getAvailablePhysicalMemorySize() {
-    refreshIfNeeded();
-    return memAvailable;
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public int getNumProcessors() {
-    refreshIfNeeded();
-    return numProcessors;
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public int getNumCores() {
-    return getNumProcessors();
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public long getCpuFrequency() {
-    refreshIfNeeded();
-    return cpuFrequencyKhz;
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public long getCumulativeCpuTime() {
-    refreshIfNeeded();
-    return cumulativeCpuTimeMs;
+  public WindowsResourceCalculatorPlugin() {
+    super(new SysInfoWindows());
   }
 
-  /** {@inheritDoc} */
-  @Override
-  public float getCpuUsage() {
-    refreshIfNeeded();
-    return cpuUsage;
-  }
 }


[26/50] hadoop git commit: YARN-3800. Reduce storage footprint for ReservationAllocation. Contributed by Anubhav Dhoot.

Posted by ar...@apache.org.
YARN-3800. Reduce storage footprint for ReservationAllocation. Contributed by Anubhav Dhoot.


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

Branch: refs/heads/HDFS-7240
Commit: 0e602fa3a1529134214452fba10a90307d9c2072
Parents: f4ca530
Author: carlo curino <Carlo Curino>
Authored: Thu Jul 9 16:47:35 2015 -0700
Committer: carlo curino <Carlo Curino>
Committed: Thu Jul 9 16:51:59 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../reservation/GreedyReservationAgent.java     | 27 ++++++-----
 .../reservation/InMemoryPlan.java               |  9 ++--
 .../InMemoryReservationAllocation.java          | 24 +++++----
 .../RLESparseResourceAllocation.java            | 43 ++---------------
 .../reservation/ReservationAllocation.java      |  3 +-
 .../reservation/ReservationSystemUtil.java      | 51 ++++++++++++++++++++
 .../reservation/ReservationSystemTestUtil.java  | 11 +++--
 .../reservation/TestCapacityOverTimePolicy.java | 16 +++---
 .../reservation/TestGreedyReservationAgent.java |  2 +-
 .../reservation/TestInMemoryPlan.java           | 37 ++++++++++----
 .../TestInMemoryReservationAllocation.java      | 29 ++++++-----
 .../TestRLESparseResourceAllocation.java        | 33 ++++++-------
 .../TestSimpleCapacityReplanner.java            | 11 +++--
 14 files changed, 176 insertions(+), 123 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e602fa3/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 3c232eb..89b5e9f 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -323,6 +323,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3827. Migrate YARN native build to new CMake framework (Alan Burlison
     via Colin P. McCabe)
 
+    YARN-3800. Reduce storage footprint for ReservationAllocation. (Anubhav Dhoot
+    via curino)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e602fa3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/GreedyReservationAgent.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/reservation/GreedyReservationAgent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/GreedyReservationAgent.java
index 5a61b94..214df1c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/GreedyReservationAgent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/GreedyReservationAgent.java
@@ -97,8 +97,8 @@ public class GreedyReservationAgent implements ReservationAgent {
     long curDeadline = deadline;
     long oldDeadline = -1;
 
-    Map<ReservationInterval, ReservationRequest> allocations =
-        new HashMap<ReservationInterval, ReservationRequest>();
+    Map<ReservationInterval, Resource> allocations =
+        new HashMap<ReservationInterval, Resource>();
     RLESparseResourceAllocation tempAssigned =
         new RLESparseResourceAllocation(plan.getResourceCalculator(),
             plan.getMinimumAllocation());
@@ -108,6 +108,8 @@ public class GreedyReservationAgent implements ReservationAgent {
     ReservationRequestInterpreter type = contract.getReservationRequests()
         .getInterpreter();
 
+    boolean hasGang = false;
+
     // Iterate the stages in backward from deadline
     for (ListIterator<ReservationRequest> li = 
         stages.listIterator(stages.size()); li.hasPrevious();) {
@@ -117,8 +119,10 @@ public class GreedyReservationAgent implements ReservationAgent {
       // validate the RR respect basic constraints
       validateInput(plan, currentReservationStage, totalCapacity);
 
+      hasGang |= currentReservationStage.getConcurrency() > 1;
+
       // run allocation for a single stage
-      Map<ReservationInterval, ReservationRequest> curAlloc =
+      Map<ReservationInterval, Resource> curAlloc =
           placeSingleStage(plan, tempAssigned, currentReservationStage,
               earliestStart, curDeadline, oldReservation, totalCapacity);
 
@@ -178,8 +182,7 @@ public class GreedyReservationAgent implements ReservationAgent {
 
     // create reservation with above allocations if not null/empty
 
-    ReservationRequest ZERO_RES =
-        ReservationRequest.newInstance(Resource.newInstance(0, 0), 0);
+    Resource ZERO_RES = Resource.newInstance(0, 0);
 
     long firstStartTime = findEarliestTime(allocations.keySet());
     
@@ -200,7 +203,7 @@ public class GreedyReservationAgent implements ReservationAgent {
         new InMemoryReservationAllocation(reservationId, contract, user,
             plan.getQueueName(), firstStartTime,
             findLatestTime(allocations.keySet()), allocations,
-            plan.getResourceCalculator(), plan.getMinimumAllocation());
+            plan.getResourceCalculator(), plan.getMinimumAllocation(), hasGang);
     if (oldReservation != null) {
       return plan.updateReservation(capReservation);
     } else {
@@ -242,13 +245,13 @@ public class GreedyReservationAgent implements ReservationAgent {
    * previous instant in time until the time-window is exhausted or we placed
    * all the user request.
    */
-  private Map<ReservationInterval, ReservationRequest> placeSingleStage(
+  private Map<ReservationInterval, Resource> placeSingleStage(
       Plan plan, RLESparseResourceAllocation tempAssigned,
       ReservationRequest rr, long earliestStart, long curDeadline,
       ReservationAllocation oldResAllocation, final Resource totalCapacity) {
 
-    Map<ReservationInterval, ReservationRequest> allocationRequests =
-        new HashMap<ReservationInterval, ReservationRequest>();
+    Map<ReservationInterval, Resource> allocationRequests =
+        new HashMap<ReservationInterval, Resource>();
 
     // compute the gang as a resource and get the duration
     Resource gang = Resources.multiply(rr.getCapability(), rr.getConcurrency());
@@ -322,7 +325,7 @@ public class GreedyReservationAgent implements ReservationAgent {
 
         ReservationInterval reservationInt =
             new ReservationInterval(curDeadline - dur, curDeadline);
-        ReservationRequest reservationRes =
+        ReservationRequest reservationRequest =
             ReservationRequest.newInstance(rr.getCapability(),
                 rr.getConcurrency() * maxGang, rr.getConcurrency(),
                 rr.getDuration());
@@ -331,6 +334,8 @@ public class GreedyReservationAgent implements ReservationAgent {
         // placing other ReservationRequest within the same
         // ReservationDefinition,
         // and we must avoid double-counting the available resources
+        final Resource reservationRes = ReservationSystemUtil.toResource(
+            reservationRequest);
         tempAssigned.addInterval(reservationInt, reservationRes);
         allocationRequests.put(reservationInt, reservationRes);
 
@@ -350,7 +355,7 @@ public class GreedyReservationAgent implements ReservationAgent {
       // If we are here is becasue we did not manage to satisfy this request.
       // So we need to remove unwanted side-effect from tempAssigned (needed
       // for ANY).
-      for (Map.Entry<ReservationInterval, ReservationRequest> tempAllocation :
+      for (Map.Entry<ReservationInterval, Resource> tempAllocation :
         allocationRequests.entrySet()) {
         tempAssigned.removeInterval(tempAllocation.getKey(),
             tempAllocation.getValue());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e602fa3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.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/reservation/InMemoryPlan.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
index ce2e7d7..50d66cf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
@@ -31,7 +31,6 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.hadoop.yarn.api.records.ReservationId;
-import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
@@ -110,7 +109,7 @@ class InMemoryPlan implements Plan {
 
   private void incrementAllocation(ReservationAllocation reservation) {
     assert (readWriteLock.isWriteLockedByCurrentThread());
-    Map<ReservationInterval, ReservationRequest> allocationRequests =
+    Map<ReservationInterval, Resource> allocationRequests =
         reservation.getAllocationRequests();
     // check if we have encountered the user earlier and if not add an entry
     String user = reservation.getUser();
@@ -119,7 +118,7 @@ class InMemoryPlan implements Plan {
       resAlloc = new RLESparseResourceAllocation(resCalc, minAlloc);
       userResourceAlloc.put(user, resAlloc);
     }
-    for (Map.Entry<ReservationInterval, ReservationRequest> r : allocationRequests
+    for (Map.Entry<ReservationInterval, Resource> r : allocationRequests
         .entrySet()) {
       resAlloc.addInterval(r.getKey(), r.getValue());
       rleSparseVector.addInterval(r.getKey(), r.getValue());
@@ -128,11 +127,11 @@ class InMemoryPlan implements Plan {
 
   private void decrementAllocation(ReservationAllocation reservation) {
     assert (readWriteLock.isWriteLockedByCurrentThread());
-    Map<ReservationInterval, ReservationRequest> allocationRequests =
+    Map<ReservationInterval, Resource> allocationRequests =
         reservation.getAllocationRequests();
     String user = reservation.getUser();
     RLESparseResourceAllocation resAlloc = userResourceAlloc.get(user);
-    for (Map.Entry<ReservationInterval, ReservationRequest> r : allocationRequests
+    for (Map.Entry<ReservationInterval, Resource> r : allocationRequests
         .entrySet()) {
       resAlloc.removeInterval(r.getKey(), r.getValue());
       rleSparseVector.removeInterval(r.getKey(), r.getValue());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e602fa3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.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/reservation/InMemoryReservationAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
index fc8407b..a4dd23b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
@@ -22,7 +22,6 @@ import java.util.Map;
 
 import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationId;
-import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -40,7 +39,7 @@ class InMemoryReservationAllocation implements ReservationAllocation {
   private final ReservationDefinition contract;
   private final long startTime;
   private final long endTime;
-  private final Map<ReservationInterval, ReservationRequest> allocationRequests;
+  private final Map<ReservationInterval, Resource> allocationRequests;
   private boolean hasGang = false;
   private long acceptedAt = -1;
 
@@ -49,22 +48,29 @@ class InMemoryReservationAllocation implements ReservationAllocation {
   InMemoryReservationAllocation(ReservationId reservationID,
       ReservationDefinition contract, String user, String planName,
       long startTime, long endTime,
-      Map<ReservationInterval, ReservationRequest> allocationRequests,
+      Map<ReservationInterval, Resource> allocations,
       ResourceCalculator calculator, Resource minAlloc) {
+    this(reservationID, contract, user, planName, startTime, endTime,
+        allocations, calculator, minAlloc, false);
+  }
+
+  InMemoryReservationAllocation(ReservationId reservationID,
+      ReservationDefinition contract, String user, String planName,
+      long startTime, long endTime,
+      Map<ReservationInterval, Resource> allocations,
+      ResourceCalculator calculator, Resource minAlloc, boolean hasGang) {
     this.contract = contract;
     this.startTime = startTime;
     this.endTime = endTime;
     this.reservationID = reservationID;
     this.user = user;
-    this.allocationRequests = allocationRequests;
+    this.allocationRequests = allocations;
     this.planName = planName;
+    this.hasGang = hasGang;
     resourcesOverTime = new RLESparseResourceAllocation(calculator, minAlloc);
-    for (Map.Entry<ReservationInterval, ReservationRequest> r : allocationRequests
+    for (Map.Entry<ReservationInterval, Resource> r : allocations
         .entrySet()) {
       resourcesOverTime.addInterval(r.getKey(), r.getValue());
-      if (r.getValue().getConcurrency() > 1) {
-        hasGang = true;
-      }
     }
   }
 
@@ -89,7 +95,7 @@ class InMemoryReservationAllocation implements ReservationAllocation {
   }
 
   @Override
-  public Map<ReservationInterval, ReservationRequest> getAllocationRequests() {
+  public Map<ReservationInterval, Resource> getAllocationRequests() {
     return Collections.unmodifiableMap(allocationRequests);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e602fa3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.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/reservation/RLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
index 3f6f405..2957cc6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 import java.io.IOException;
 import java.io.StringWriter;
 import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.NavigableMap;
@@ -31,9 +30,7 @@ import java.util.TreeMap;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
-import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
@@ -80,14 +77,11 @@ public class RLESparseResourceAllocation {
    * 
    * @param reservationInterval the interval for which the resource is to be
    *          added
-   * @param capacity the resource to be added
+   * @param totCap the resource to be added
    * @return true if addition is successful, false otherwise
    */
   public boolean addInterval(ReservationInterval reservationInterval,
-      ReservationRequest capacity) {
-    Resource totCap =
-        Resources.multiply(capacity.getCapability(),
-            (float) capacity.getNumContainers());
+      Resource totCap) {
     if (totCap.equals(ZERO_RESOURCE)) {
       return true;
     }
@@ -143,44 +137,15 @@ public class RLESparseResourceAllocation {
   }
 
   /**
-   * Add multiple resources for the specified interval
-   * 
-   * @param reservationInterval the interval for which the resource is to be
-   *          added
-   * @param ReservationRequests the resources to be added
-   * @param clusterResource the total resources in the cluster
-   * @return true if addition is successful, false otherwise
-   */
-  public boolean addCompositeInterval(ReservationInterval reservationInterval,
-      List<ReservationRequest> ReservationRequests, Resource clusterResource) {
-    ReservationRequest aggregateReservationRequest =
-        Records.newRecord(ReservationRequest.class);
-    Resource capacity = Resource.newInstance(0, 0);
-    for (ReservationRequest ReservationRequest : ReservationRequests) {
-      Resources.addTo(capacity, Resources.multiply(
-          ReservationRequest.getCapability(),
-          ReservationRequest.getNumContainers()));
-    }
-    aggregateReservationRequest.setNumContainers((int) Math.ceil(Resources
-        .divide(resourceCalculator, clusterResource, capacity, minAlloc)));
-    aggregateReservationRequest.setCapability(minAlloc);
-
-    return addInterval(reservationInterval, aggregateReservationRequest);
-  }
-
-  /**
    * Removes a resource for the specified interval
    * 
    * @param reservationInterval the interval for which the resource is to be
    *          removed
-   * @param capacity the resource to be removed
+   * @param totCap the resource to be removed
    * @return true if removal is successful, false otherwise
    */
   public boolean removeInterval(ReservationInterval reservationInterval,
-      ReservationRequest capacity) {
-    Resource totCap =
-        Resources.multiply(capacity.getCapability(),
-            (float) capacity.getNumContainers());
+      Resource totCap) {
     if (totCap.equals(ZERO_RESOURCE)) {
       return true;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e602fa3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.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/reservation/ReservationAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
index 89c0e55..0d3c692 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
@@ -22,7 +22,6 @@ import java.util.Map;
 
 import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationId;
-import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.Resource;
 
 /**
@@ -71,7 +70,7 @@ public interface ReservationAllocation extends
    * @return the allocationRequests the map of resources requested against the
    *         time interval for which they were
    */
-  public Map<ReservationInterval, ReservationRequest> getAllocationRequests();
+  public Map<ReservationInterval, Resource> getAllocationRequests();
 
   /**
    * Return a string identifying the plan to which the reservation belongs

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e602fa3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemUtil.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/reservation/ReservationSystemUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemUtil.java
new file mode 100644
index 0000000..8affae4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemUtil.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import org.apache.hadoop.yarn.api.records.ReservationRequest;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+import java.util.HashMap;
+import java.util.Map;
+
+final class ReservationSystemUtil {
+
+  private ReservationSystemUtil() {
+    // not called
+  }
+
+  public static Resource toResource(ReservationRequest request) {
+    Resource resource = Resources.multiply(request.getCapability(),
+        (float) request.getNumContainers());
+    return resource;
+  }
+
+  public static Map<ReservationInterval, Resource> toResources(
+      Map<ReservationInterval, ReservationRequest> allocations) {
+    Map<ReservationInterval, Resource> resources =
+        new HashMap<ReservationInterval, Resource>();
+    for (Map.Entry<ReservationInterval, ReservationRequest> entry :
+        allocations.entrySet()) {
+      resources.put(entry.getKey(),
+          toResource(entry.getValue()));
+    }
+    return resources;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e602fa3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
index bfaf06b..be1d69a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
@@ -378,14 +378,15 @@ public class ReservationSystemTestUtil {
     return rr;
   }
 
-  public static Map<ReservationInterval, ReservationRequest> generateAllocation(
+  public static Map<ReservationInterval, Resource> generateAllocation(
       long startTime, long step, int[] alloc) {
-    Map<ReservationInterval, ReservationRequest> req =
-        new TreeMap<ReservationInterval, ReservationRequest>();
+    Map<ReservationInterval, Resource> req =
+        new TreeMap<ReservationInterval, Resource>();
     for (int i = 0; i < alloc.length; i++) {
       req.put(new ReservationInterval(startTime + i * step, startTime + (i + 1)
-          * step), ReservationRequest.newInstance(
-          Resource.newInstance(1024, 1), alloc[i]));
+          * step), ReservationSystemUtil.toResource(ReservationRequest
+          .newInstance(
+          Resource.newInstance(1024, 1), alloc[i])));
     }
     return req;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e602fa3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityOverTimePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityOverTimePolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityOverTimePolicy.java
index 61561e9..19f876d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityOverTimePolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityOverTimePolicy.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.util.Map;
@@ -198,12 +197,14 @@ public class TestCapacityOverTimePolicy {
   @Test(expected = PlanningQuotaException.class)
   public void testFailAvg() throws IOException, PlanningException {
     // generate an allocation which violates the 25% average single-shot
-    Map<ReservationInterval, ReservationRequest> req =
-        new TreeMap<ReservationInterval, ReservationRequest>();
+    Map<ReservationInterval, Resource> req =
+        new TreeMap<ReservationInterval, Resource>();
     long win = timeWindow / 2 + 100;
     int cont = (int) Math.ceil(0.5 * totCont);
     req.put(new ReservationInterval(initTime, initTime + win),
-        ReservationRequest.newInstance(Resource.newInstance(1024, 1), cont));
+        ReservationSystemUtil.toResource(
+            ReservationRequest.newInstance(Resource.newInstance(1024, 1),
+                cont)));
 
     assertTrue(plan.toString(),
         plan.addReservation(new InMemoryReservationAllocation(
@@ -214,12 +215,13 @@ public class TestCapacityOverTimePolicy {
   @Test
   public void testFailAvgBySum() throws IOException, PlanningException {
     // generate an allocation which violates the 25% average by sum
-    Map<ReservationInterval, ReservationRequest> req =
-        new TreeMap<ReservationInterval, ReservationRequest>();
+    Map<ReservationInterval, Resource> req =
+        new TreeMap<ReservationInterval, Resource>();
     long win = 86400000 / 4 + 1;
     int cont = (int) Math.ceil(0.5 * totCont);
     req.put(new ReservationInterval(initTime, initTime + win),
-        ReservationRequest.newInstance(Resource.newInstance(1024, 1), cont));
+        ReservationSystemUtil.toResource(ReservationRequest.newInstance(Resource
+            .newInstance(1024, 1), cont)));
     assertTrue(plan.toString(),
         plan.addReservation(new InMemoryReservationAllocation(
             ReservationSystemTestUtil.getNewReservationId(), null, "u1",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e602fa3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestGreedyReservationAgent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestGreedyReservationAgent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestGreedyReservationAgent.java
index b8cf6c5..de94dcd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestGreedyReservationAgent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestGreedyReservationAgent.java
@@ -516,7 +516,7 @@ public class TestGreedyReservationAgent {
                 .generateAllocation(0, step, f), res, minAlloc)));
 
     int[] f2 = { 5, 5, 5, 5, 5, 5, 5 };
-    Map<ReservationInterval, ReservationRequest> alloc = 
+    Map<ReservationInterval, Resource> alloc =
         ReservationSystemTestUtil.generateAllocation(5000, step, f2);
     assertTrue(plan.toString(),
         plan.addReservation(new InMemoryReservationAllocation(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e602fa3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
index 91c1962..722fb29 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
@@ -100,9 +100,11 @@ public class TestInMemoryPlan {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length,
             alloc.length, allocations.values());
+    Map<ReservationInterval, Resource> allocs =
+        ReservationSystemUtil.toResources(allocations);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
-            start, start + alloc.length, allocations, resCalc, minAlloc);
+            start, start + alloc.length, allocs, resCalc, minAlloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation);
@@ -132,9 +134,11 @@ public class TestInMemoryPlan {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length,
             alloc.length, allocations.values());
+    Map<ReservationInterval, Resource> allocs = ReservationSystemUtil.toResources
+        (allocations);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
-            start, start + alloc.length, allocations, resCalc, minAlloc);
+            start, start + alloc.length, allocs, resCalc, minAlloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation);
@@ -158,9 +162,11 @@ public class TestInMemoryPlan {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length,
             alloc.length, allocations.values());
+    Map<ReservationInterval, Resource> allocs = ReservationSystemUtil.toResources
+        (allocations);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
-            start, start + alloc.length, allocations, resCalc, minAlloc);
+            start, start + alloc.length, allocs, resCalc, minAlloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation);
@@ -202,9 +208,11 @@ public class TestInMemoryPlan {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length,
             alloc.length, allocations.values());
+    Map<ReservationInterval, Resource> allocs = ReservationSystemUtil.toResources
+        (allocations);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
-            start, start + alloc.length, allocations, resCalc, minAlloc);
+            start, start + alloc.length, allocs, resCalc, minAlloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation);
@@ -226,9 +234,12 @@ public class TestInMemoryPlan {
     rDef =
         createSimpleReservationDefinition(start, start + updatedAlloc.length,
             updatedAlloc.length, allocations.values());
+    Map<ReservationInterval, Resource> updatedAllocs =
+        ReservationSystemUtil.toResources(allocations);
     rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
-            start, start + updatedAlloc.length, allocations, resCalc, minAlloc);
+            start, start + updatedAlloc.length, updatedAllocs, resCalc,
+            minAlloc);
     try {
       plan.updateReservation(rAllocation);
     } catch (PlanningException e) {
@@ -260,9 +271,11 @@ public class TestInMemoryPlan {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length,
             alloc.length, allocations.values());
+    Map<ReservationInterval, Resource> allocs =
+        ReservationSystemUtil.toResources(allocations);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
-            start, start + alloc.length, allocations, resCalc, minAlloc);
+            start, start + alloc.length, allocs, resCalc, minAlloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.updateReservation(rAllocation);
@@ -290,9 +303,11 @@ public class TestInMemoryPlan {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length,
             alloc.length, allocations.values());
+    Map<ReservationInterval, Resource> allocs =
+        ReservationSystemUtil.toResources(allocations);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
-            start, start + alloc.length, allocations, resCalc, minAlloc);
+            start, start + alloc.length, allocs, resCalc, minAlloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation);
@@ -359,9 +374,11 @@ public class TestInMemoryPlan {
     ReservationDefinition rDef1 =
         createSimpleReservationDefinition(start, start + alloc1.length,
             alloc1.length, allocations1.values());
+    Map<ReservationInterval, Resource> allocs1 =
+        ReservationSystemUtil.toResources(allocations1);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID1, rDef1, user,
-            planName, start, start + alloc1.length, allocations1, resCalc,
+            planName, start, start + alloc1.length, allocs1, resCalc,
             minAlloc);
     Assert.assertNull(plan.getReservationById(reservationID1));
     try {
@@ -388,9 +405,11 @@ public class TestInMemoryPlan {
     ReservationDefinition rDef2 =
         createSimpleReservationDefinition(start, start + alloc2.length,
             alloc2.length, allocations2.values());
+    Map<ReservationInterval, Resource> allocs2 =
+        ReservationSystemUtil.toResources(allocations2);
     rAllocation =
         new InMemoryReservationAllocation(reservationID2, rDef2, user,
-            planName, start, start + alloc2.length, allocations2, resCalc,
+            planName, start, start + alloc2.length, allocs2, resCalc,
             minAlloc);
     Assert.assertNull(plan.getReservationById(reservationID2));
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e602fa3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryReservationAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryReservationAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryReservationAllocation.java
index 76f39dc..55224a9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryReservationAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryReservationAllocation.java
@@ -69,7 +69,7 @@ public class TestInMemoryReservationAllocation {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length + 1,
             alloc.length);
-    Map<ReservationInterval, ReservationRequest> allocations =
+    Map<ReservationInterval, Resource> allocations =
         generateAllocation(start, alloc, false, false);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
@@ -91,7 +91,7 @@ public class TestInMemoryReservationAllocation {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length + 1,
             alloc.length);
-    Map<ReservationInterval, ReservationRequest> allocations =
+    Map<ReservationInterval, Resource> allocations =
         generateAllocation(start, alloc, true, false);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
@@ -114,7 +114,7 @@ public class TestInMemoryReservationAllocation {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length + 1,
             alloc.length);
-    Map<ReservationInterval, ReservationRequest> allocations =
+    Map<ReservationInterval, Resource> allocations =
         generateAllocation(start, alloc, true, false);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
@@ -137,8 +137,8 @@ public class TestInMemoryReservationAllocation {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length + 1,
             alloc.length);
-    Map<ReservationInterval, ReservationRequest> allocations =
-        new HashMap<ReservationInterval, ReservationRequest>();
+    Map<ReservationInterval, Resource> allocations =
+        new HashMap<ReservationInterval, Resource>();
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
             start, start + alloc.length + 1, allocations, resCalc, minAlloc);
@@ -156,11 +156,13 @@ public class TestInMemoryReservationAllocation {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length + 1,
             alloc.length);
-    Map<ReservationInterval, ReservationRequest> allocations =
-        generateAllocation(start, alloc, false, true);
+    boolean isGang = true;
+    Map<ReservationInterval, Resource> allocations =
+        generateAllocation(start, alloc, false, isGang);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
-            start, start + alloc.length + 1, allocations, resCalc, minAlloc);
+            start, start + alloc.length + 1, allocations, resCalc, minAlloc,
+            isGang);
     doAssertions(rAllocation, reservationID, rDef, allocations, start, alloc);
     Assert.assertTrue(rAllocation.containsGangs());
     for (int i = 0; i < alloc.length; i++) {
@@ -171,7 +173,7 @@ public class TestInMemoryReservationAllocation {
 
   private void doAssertions(ReservationAllocation rAllocation,
       ReservationId reservationID, ReservationDefinition rDef,
-      Map<ReservationInterval, ReservationRequest> allocations, int start,
+      Map<ReservationInterval, Resource> allocations, int start,
       int[] alloc) {
     Assert.assertEquals(reservationID, rAllocation.getReservationId());
     Assert.assertEquals(rDef, rAllocation.getReservationDefinition());
@@ -198,10 +200,10 @@ public class TestInMemoryReservationAllocation {
     return rDef;
   }
 
-  private Map<ReservationInterval, ReservationRequest> generateAllocation(
+  private Map<ReservationInterval, Resource> generateAllocation(
       int startTime, int[] alloc, boolean isStep, boolean isGang) {
-    Map<ReservationInterval, ReservationRequest> req =
-        new HashMap<ReservationInterval, ReservationRequest>();
+    Map<ReservationInterval, Resource> req =
+        new HashMap<ReservationInterval, Resource>();
     int numContainers = 0;
     for (int i = 0; i < alloc.length; i++) {
       if (isStep) {
@@ -215,7 +217,8 @@ public class TestInMemoryReservationAllocation {
       if (isGang) {
         rr.setConcurrency(numContainers);
       }
-      req.put(new ReservationInterval(startTime + i, startTime + i + 1), rr);
+      req.put(new ReservationInterval(startTime + i, startTime + i + 1),
+          ReservationSystemUtil.toResource(rr));
     }
     return req;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e602fa3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
index c7301c7..d0f4dc6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
@@ -46,9 +46,9 @@ public class TestRLESparseResourceAllocation {
         new RLESparseResourceAllocation(resCalc, minAlloc);
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    Set<Entry<ReservationInterval, ReservationRequest>> inputs =
+    Set<Entry<ReservationInterval, Resource>> inputs =
         generateAllocation(start, alloc, false).entrySet();
-    for (Entry<ReservationInterval, ReservationRequest> ip : inputs) {
+    for (Entry<ReservationInterval, Resource> ip : inputs) {
       rleSparseVector.addInterval(ip.getKey(), ip.getValue());
     }
     LOG.info(rleSparseVector.toString());
@@ -63,7 +63,7 @@ public class TestRLESparseResourceAllocation {
     }
     Assert.assertEquals(Resource.newInstance(0, 0),
         rleSparseVector.getCapacityAtTime(start + alloc.length + 2));
-    for (Entry<ReservationInterval, ReservationRequest> ip : inputs) {
+    for (Entry<ReservationInterval, Resource> ip : inputs) {
       rleSparseVector.removeInterval(ip.getKey(), ip.getValue());
     }
     LOG.info(rleSparseVector.toString());
@@ -83,9 +83,9 @@ public class TestRLESparseResourceAllocation {
         new RLESparseResourceAllocation(resCalc, minAlloc);
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    Set<Entry<ReservationInterval, ReservationRequest>> inputs =
+    Set<Entry<ReservationInterval, Resource>> inputs =
         generateAllocation(start, alloc, true).entrySet();
-    for (Entry<ReservationInterval, ReservationRequest> ip : inputs) {
+    for (Entry<ReservationInterval, Resource> ip : inputs) {
       rleSparseVector.addInterval(ip.getKey(), ip.getValue());
     }
     LOG.info(rleSparseVector.toString());
@@ -101,8 +101,8 @@ public class TestRLESparseResourceAllocation {
     }
     Assert.assertEquals(Resource.newInstance(0, 0),
         rleSparseVector.getCapacityAtTime(start + alloc.length + 2));
-    for (Entry<ReservationInterval, ReservationRequest> ip : inputs) {
-      rleSparseVector.removeInterval(ip.getKey(), ip.getValue());
+    for (Entry<ReservationInterval, Resource> ip : inputs) {
+      rleSparseVector.removeInterval(ip.getKey(),ip.getValue());
     }
     LOG.info(rleSparseVector.toString());
     for (int i = 0; i < alloc.length; i++) {
@@ -121,9 +121,9 @@ public class TestRLESparseResourceAllocation {
         new RLESparseResourceAllocation(resCalc, minAlloc);
     int[] alloc = { 0, 5, 10, 10, 5, 0 };
     int start = 100;
-    Set<Entry<ReservationInterval, ReservationRequest>> inputs =
+    Set<Entry<ReservationInterval, Resource>> inputs =
         generateAllocation(start, alloc, true).entrySet();
-    for (Entry<ReservationInterval, ReservationRequest> ip : inputs) {
+    for (Entry<ReservationInterval, Resource> ip : inputs) {
       rleSparseVector.addInterval(ip.getKey(), ip.getValue());
     }
     LOG.info(rleSparseVector.toString());
@@ -139,7 +139,7 @@ public class TestRLESparseResourceAllocation {
     }
     Assert.assertEquals(Resource.newInstance(0, 0),
         rleSparseVector.getCapacityAtTime(start + alloc.length + 2));
-    for (Entry<ReservationInterval, ReservationRequest> ip : inputs) {
+    for (Entry<ReservationInterval, Resource> ip : inputs) {
       rleSparseVector.removeInterval(ip.getKey(), ip.getValue());
     }
     LOG.info(rleSparseVector.toString());
@@ -157,17 +157,17 @@ public class TestRLESparseResourceAllocation {
     RLESparseResourceAllocation rleSparseVector =
         new RLESparseResourceAllocation(resCalc, minAlloc);
     rleSparseVector.addInterval(new ReservationInterval(0, Long.MAX_VALUE),
-        ReservationRequest.newInstance(Resource.newInstance(0, 0), (0)));
+        Resource.newInstance(0, 0));
     LOG.info(rleSparseVector.toString());
     Assert.assertEquals(Resource.newInstance(0, 0),
         rleSparseVector.getCapacityAtTime(new Random().nextLong()));
     Assert.assertTrue(rleSparseVector.isEmpty());
   }
 
-  private Map<ReservationInterval, ReservationRequest> generateAllocation(
+  private Map<ReservationInterval, Resource> generateAllocation(
       int startTime, int[] alloc, boolean isStep) {
-    Map<ReservationInterval, ReservationRequest> req =
-        new HashMap<ReservationInterval, ReservationRequest>();
+    Map<ReservationInterval, Resource> req =
+        new HashMap<ReservationInterval, Resource>();
     int numContainers = 0;
     for (int i = 0; i < alloc.length; i++) {
       if (isStep) {
@@ -176,9 +176,8 @@ public class TestRLESparseResourceAllocation {
         numContainers = alloc[i];
       }
       req.put(new ReservationInterval(startTime + i, startTime + i + 1),
-
-      ReservationRequest.newInstance(Resource.newInstance(1024, 1),
-          (numContainers)));
+          ReservationSystemUtil.toResource(ReservationRequest.newInstance(
+              Resource.newInstance(1024, 1), (numContainers))));
     }
     return req;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e602fa3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestSimpleCapacityReplanner.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestSimpleCapacityReplanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestSimpleCapacityReplanner.java
index 1ca9f2e..d4a97ba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestSimpleCapacityReplanner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestSimpleCapacityReplanner.java
@@ -146,14 +146,15 @@ public class TestSimpleCapacityReplanner {
     }
   }
 
-  private Map<ReservationInterval, ReservationRequest> generateAllocation(
+  private Map<ReservationInterval, Resource> generateAllocation(
       int startTime, int[] alloc) {
-    Map<ReservationInterval, ReservationRequest> req =
-        new TreeMap<ReservationInterval, ReservationRequest>();
+    Map<ReservationInterval, Resource> req =
+        new TreeMap<ReservationInterval, Resource>();
     for (int i = 0; i < alloc.length; i++) {
       req.put(new ReservationInterval(startTime + i, startTime + i + 1),
-          ReservationRequest.newInstance(Resource.newInstance(1024, 1),
-              alloc[i]));
+          ReservationSystemUtil.toResource(
+              ReservationRequest.newInstance(Resource.newInstance(1024, 1),
+                  alloc[i])));
     }
     return req;
   }


[16/50] hadoop git commit: HADOOP-11878. FileContext#fixRelativePart should check for not null for a more informative exception. (Brahma Reddy Battula via kasha)

Posted by ar...@apache.org.
HADOOP-11878. FileContext#fixRelativePart should check for not null for a more informative exception. (Brahma Reddy Battula via kasha)


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

Branch: refs/heads/HDFS-7240
Commit: 2e3d83f97b85c015f16f1c162b8ef0e7fee1ddf2
Parents: 625d7ed
Author: Karthik Kambatla <ka...@apache.org>
Authored: Wed Jul 8 11:34:34 2015 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Wed Jul 8 11:34:34 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                   | 3 +++
 .../src/main/java/org/apache/hadoop/fs/FileContext.java           | 3 +++
 2 files changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e3d83f9/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 6cc6b71..eb18e6c 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -722,6 +722,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12194. Support for incremental generation in the protoc plugin.
     (wang)
 
+    HADOOP-11878. FileContext#fixRelativePart should check for not null for a
+    more informative exception. (Brahma Reddy Battula via kasha)
+
   BUG FIXES
 
     HADOOP-11802: DomainSocketWatcher thread terminates sometimes after there

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e3d83f9/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
index 122ddf6..0f21a61 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
@@ -59,6 +59,8 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.ShutdownHookManager;
 
+import com.google.common.base.Preconditions;
+
 /**
  * The FileContext class provides an interface for users of the Hadoop
  * file system. It exposes a number of file system operations, e.g. create,
@@ -262,6 +264,7 @@ public class FileContext {
    * has been deliberately declared private.
    */
   Path fixRelativePart(Path p) {
+    Preconditions.checkNotNull(p, "path cannot be null");
     if (p.isUriPathAbsolute()) {
       return p;
     } else {


[38/50] hadoop git commit: MAPREDUCE-6421. Fix findbugs warning in RMContainerAllocator.reduceNodeLabelExpression. Contributed by Brahma Reddy Battula.

Posted by ar...@apache.org.
MAPREDUCE-6421. Fix findbugs warning in RMContainerAllocator.reduceNodeLabelExpression. Contributed by Brahma Reddy Battula.


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

Branch: refs/heads/HDFS-7240
Commit: e04faf8e462a24a3ddfd8f22050e74eda4aa19ff
Parents: 5ed1fea
Author: Akira Ajisaka <aa...@apache.org>
Authored: Mon Jul 13 14:28:14 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Mon Jul 13 14:29:14 2015 +0900

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                      | 4 ++++
 hadoop-mapreduce-project/dev-support/findbugs-exclude.xml | 1 +
 2 files changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e04faf8e/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 81b202b..95eec1c 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -533,6 +533,10 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6038. A boolean may be set error in the Word Count v2.0 in
     MapReduce Tutorial. (Tsuyoshi Ozawa via cdouglas)
 
+    MAPREDUCE-6421. Fix findbugs warning in
+    RMContainerAllocator.reduceNodeLabelExpression.
+    (Brahma Reddy Battula via aajisaka)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e04faf8e/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml b/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml
index dd4892b..f1ef2b8 100644
--- a/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml
+++ b/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml
@@ -479,6 +479,7 @@
       <Field name="reduceResourceRequest" />
       <Field name="maxReduceRampupLimit" />
       <Field name="reduceSlowStart" />
+      <Field name="reduceNodeLabelExpression" />
      </Or>
      <Bug pattern="IS2_INCONSISTENT_SYNC" />
    </Match>


[09/50] hadoop git commit: YARN-3690. [JDK8] 'mvn site' fails. Contributed by Brahma Reddy Battula.

Posted by ar...@apache.org.
YARN-3690. [JDK8] 'mvn site' fails. Contributed by Brahma Reddy Battula.


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

Branch: refs/heads/HDFS-7240
Commit: d6325745e26d6acb0bbdb0972ce036297d66d5b0
Parents: c9dd2ca
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Jul 8 15:41:33 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Jul 8 15:42:31 2015 +0900

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                  | 2 ++
 .../src/main/java/org/apache/hadoop/yarn/util/package-info.java  | 2 --
 .../org/apache/hadoop/yarn/client/api/impl/package-info.java     | 4 ----
 .../java/org/apache/hadoop/yarn/client/api/package-info.java     | 4 ----
 .../main/java/org/apache/hadoop/yarn/factories/package-info.java | 2 --
 .../org/apache/hadoop/yarn/factory/providers/package-info.java   | 2 --
 6 files changed, 2 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6325745/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 8b5cc0c..d1960e6 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -619,6 +619,8 @@ Release 2.7.2 - UNRELEASED
     YARN-3508. Prevent processing preemption events on the main RM dispatcher. 
     (Varun Saxena via wangda)
 
+    YARN-3690. [JDK8] 'mvn site' fails. (Brahma Reddy Battula via aajisaka)
+
 Release 2.7.1 - 2015-07-06 
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6325745/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/package-info.java
index fd7cfb2..e4518ae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/package-info.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/package-info.java
@@ -15,6 +15,4 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-@InterfaceAudience.Private
 package org.apache.hadoop.yarn.util;
-import org.apache.hadoop.classification.InterfaceAudience;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6325745/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/package-info.java
index 9e2d911..7d747d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/package-info.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/package-info.java
@@ -15,9 +15,5 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
 package org.apache.hadoop.yarn.client.api.impl;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6325745/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/package-info.java
index a37e3d9..9130828 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/package-info.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/package-info.java
@@ -15,9 +15,5 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
 package org.apache.hadoop.yarn.client.api;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6325745/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/package-info.java
index 249f335..a52771b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/package-info.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factories/package-info.java
@@ -15,7 +15,5 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-@InterfaceAudience.LimitedPrivate({ "MapReduce", "YARN" })
 package org.apache.hadoop.yarn.factories;
-import org.apache.hadoop.classification.InterfaceAudience;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6325745/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/package-info.java
index 26579ae..dd0c85c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/package-info.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/package-info.java
@@ -15,7 +15,5 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-@InterfaceAudience.LimitedPrivate({ "MapReduce", "YARN" })
 package org.apache.hadoop.yarn.factory.providers;
-import org.apache.hadoop.classification.InterfaceAudience;
 


[15/50] hadoop git commit: HADOOP-12194. Support for incremental generation in the protoc plugin.

Posted by ar...@apache.org.
HADOOP-12194. Support for incremental generation in the protoc plugin.


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

Branch: refs/heads/HDFS-7240
Commit: 625d7ed9eb65f0df204b506ce92c11803fbce273
Parents: fc6182d
Author: Andrew Wang <wa...@apache.org>
Authored: Wed Jul 8 11:09:43 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Wed Jul 8 11:09:43 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 hadoop-maven-plugins/pom.xml                    |   8 +
 .../hadoop/maven/plugin/protoc/ProtocMojo.java  | 188 +++++++++++++++++--
 3 files changed, 185 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/625d7ed9/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 8ab109d..6cc6b71 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -719,6 +719,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12172. FsShell mkdir -p makes an unnecessary check for the existence
     of the parent. (cnauroth)
 
+    HADOOP-12194. Support for incremental generation in the protoc plugin.
+    (wang)
+
   BUG FIXES
 
     HADOOP-11802: DomainSocketWatcher thread terminates sometimes after there

http://git-wip-us.apache.org/repos/asf/hadoop/blob/625d7ed9/hadoop-maven-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-maven-plugins/pom.xml b/hadoop-maven-plugins/pom.xml
index b48b9ac..b39c22b 100644
--- a/hadoop-maven-plugins/pom.xml
+++ b/hadoop-maven-plugins/pom.xml
@@ -47,6 +47,14 @@
       <version>${maven.plugin-tools.version}</version>
       <scope>provided</scope>
     </dependency>
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-core-asl</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-mapper-asl</artifactId>
+    </dependency>
   </dependencies>
   <build>
     <plugins>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/625d7ed9/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocMojo.java
----------------------------------------------------------------------
diff --git a/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocMojo.java b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocMojo.java
index 465b713..b9be33e 100644
--- a/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocMojo.java
+++ b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocMojo.java
@@ -22,11 +22,21 @@ import org.apache.maven.plugins.annotations.LifecyclePhase;
 import org.apache.maven.plugins.annotations.Mojo;
 import org.apache.maven.plugins.annotations.Parameter;
 import org.apache.maven.project.MavenProject;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.type.TypeReference;
 
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
 import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
-
+import java.util.Map;
+import java.util.zip.CRC32;
 
 @Mojo(name="protoc", defaultPhase = LifecyclePhase.GENERATE_SOURCES)
 public class ProtocMojo extends AbstractMojo {
@@ -49,6 +59,118 @@ public class ProtocMojo extends AbstractMojo {
   @Parameter(required=true)
   private String protocVersion;
 
+  @Parameter(defaultValue =
+      "${project.build.directory}/hadoop-maven-plugins-protoc-checksums.json")
+  private String checksumPath;
+
+  /**
+   * Compares include and source file checksums against previously computed
+   * checksums stored in a json file in the build directory.
+   */
+  public class ChecksumComparator {
+
+    private final Map<String, Long> storedChecksums;
+    private final Map<String, Long> computedChecksums;
+
+    private final File checksumFile;
+
+    ChecksumComparator(String checksumPath) throws IOException {
+      checksumFile = new File(checksumPath);
+      // Read in the checksums
+      if (checksumFile.exists()) {
+        ObjectMapper mapper = new ObjectMapper();
+        storedChecksums = mapper
+            .readValue(checksumFile, new TypeReference<Map<String, Long>>() {
+            });
+      } else {
+        storedChecksums = new HashMap<>(0);
+      }
+      computedChecksums = new HashMap<>();
+    }
+
+    public boolean hasChanged(File file) throws IOException {
+      if (!file.exists()) {
+        throw new FileNotFoundException(
+            "Specified protoc include or source does not exist: " + file);
+      }
+      if (file.isDirectory()) {
+        return hasDirectoryChanged(file);
+      } else if (file.isFile()) {
+        return hasFileChanged(file);
+      } else {
+        throw new IOException("Not a file or directory: " + file);
+      }
+    }
+
+    private boolean hasDirectoryChanged(File directory) throws IOException {
+      File[] listing = directory.listFiles();
+      boolean changed = false;
+      // Do not exit early, since we need to compute and save checksums
+      // for each file within the directory.
+      for (File f : listing) {
+        if (f.isDirectory()) {
+          if (hasDirectoryChanged(f)) {
+            changed = true;
+          }
+        } else if (f.isFile()) {
+          if (hasFileChanged(f)) {
+            changed = true;
+          }
+        } else {
+          getLog().debug("Skipping entry that is not a file or directory: "
+              + f);
+        }
+      }
+      return changed;
+    }
+
+    private boolean hasFileChanged(File file) throws IOException {
+      long computedCsum = computeChecksum(file);
+
+      // Return if the generated csum matches the stored csum
+      Long storedCsum = storedChecksums.get(file.getCanonicalPath());
+      if (storedCsum == null || storedCsum.longValue() != computedCsum) {
+        // It has changed.
+        return true;
+      }
+      return false;
+    }
+
+    private long computeChecksum(File file) throws IOException {
+      // If we've already computed the csum, reuse the computed value
+      final String canonicalPath = file.getCanonicalPath();
+      if (computedChecksums.containsKey(canonicalPath)) {
+        return computedChecksums.get(canonicalPath);
+      }
+      // Compute the csum for the file
+      CRC32 crc = new CRC32();
+      byte[] buffer = new byte[1024*64];
+      try (BufferedInputStream in =
+          new BufferedInputStream(new FileInputStream(file))) {
+        while (true) {
+          int read = in.read(buffer);
+          if (read <= 0) {
+            break;
+          }
+          crc.update(buffer, 0, read);
+        }
+      }
+      // Save it in the generated map and return
+      final long computedCsum = crc.getValue();
+      computedChecksums.put(canonicalPath, computedCsum);
+      return crc.getValue();
+    }
+
+    public void writeChecksums() throws IOException {
+      ObjectMapper mapper = new ObjectMapper();
+      try (BufferedOutputStream out = new BufferedOutputStream(
+          new FileOutputStream(checksumFile))) {
+        mapper.writeValue(out, computedChecksums);
+        getLog().info("Wrote protoc checksums to file " + checksumFile);
+      }
+    }
+  }
+
   public void execute() throws MojoExecutionException {
     try {
       List<String> command = new ArrayList<String>();
@@ -58,7 +180,7 @@ public class ProtocMojo extends AbstractMojo {
       List<String> out = new ArrayList<String>();
       if (exec.run(command, out) == 127) {
         getLog().error("protoc, not found at: " + protocCommand);
-        throw new MojoExecutionException("protoc failure");        
+        throw new MojoExecutionException("protoc failure");
       } else {
         if (out.isEmpty()) {
           getLog().error("stdout: " + out);
@@ -67,36 +189,74 @@ public class ProtocMojo extends AbstractMojo {
         } else {
           if (!out.get(0).endsWith(protocVersion)) {
             throw new MojoExecutionException(
-                "protoc version is '" + out.get(0) + "', expected version is '" 
-                    + protocVersion + "'");            
+                "protoc version is '" + out.get(0) + "', expected version is '"
+                    + protocVersion + "'");
           }
         }
       }
       if (!output.mkdirs()) {
         if (!output.exists()) {
-          throw new MojoExecutionException("Could not create directory: " + 
-            output);
+          throw new MojoExecutionException(
+              "Could not create directory: " + output);
         }
       }
+
+      // Whether the import or source protoc files have changed.
+      ChecksumComparator comparator = new ChecksumComparator(checksumPath);
+      boolean importsChanged = false;
+
       command = new ArrayList<String>();
       command.add(protocCommand);
       command.add("--java_out=" + output.getCanonicalPath());
       if (imports != null) {
         for (File i : imports) {
+          if (comparator.hasChanged(i)) {
+            importsChanged = true;
+          }
           command.add("-I" + i.getCanonicalPath());
         }
       }
+      // Filter to generate classes for just the changed source files.
+      List<File> changedSources = new ArrayList<>();
+      boolean sourcesChanged = false;
       for (File f : FileSetUtils.convertFileSetToFiles(source)) {
-        command.add(f.getCanonicalPath());
+        // Need to recompile if the source has changed, or if any import has
+        // changed.
+        if (comparator.hasChanged(f) || importsChanged) {
+          sourcesChanged = true;
+          changedSources.add(f);
+          command.add(f.getCanonicalPath());
+        }
       }
-      exec = new Exec(this);
-      out = new ArrayList<String>();
-      if (exec.run(command, out) != 0) {
-        getLog().error("protoc compiler error");
-        for (String s : out) {
-          getLog().error(s);
+
+      if (!sourcesChanged && !importsChanged) {
+        getLog().info("No changes detected in protoc files, skipping "
+            + "generation.");
+      } else {
+        if (getLog().isDebugEnabled()) {
+          StringBuilder b = new StringBuilder();
+          b.append("Generating classes for the following protoc files: [");
+          String prefix = "";
+          for (File f : changedSources) {
+            b.append(prefix);
+            b.append(f.toString());
+            prefix = ", ";
+          }
+          b.append("]");
+          getLog().debug(b.toString());
         }
-        throw new MojoExecutionException("protoc failure");
+
+        exec = new Exec(this);
+        out = new ArrayList<String>();
+        if (exec.run(command, out) != 0) {
+          getLog().error("protoc compiler error");
+          for (String s : out) {
+            getLog().error(s);
+          }
+          throw new MojoExecutionException("protoc failure");
+        }
+        // Write the new checksum file on success.
+        comparator.writeChecksums();
       }
     } catch (Throwable ex) {
       throw new MojoExecutionException(ex.toString(), ex);


[42/50] hadoop git commit: HDFS-8541. Mover should exit with NO_MOVE_PROGRESS if there is no move progress. Contributed by Surendra Singh Lilhore

Posted by ar...@apache.org.
HDFS-8541. Mover should exit with NO_MOVE_PROGRESS if there is no move progress.  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/9ef03a4c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9ef03a4c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9ef03a4c

Branch: refs/heads/HDFS-7240
Commit: 9ef03a4c5bb5573eadc7d04e371c4af2dc6bae37
Parents: f7c8311
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jul 13 15:12:26 2015 -0700
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Mon Jul 13 15:12:26 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../hadoop/hdfs/server/balancer/Dispatcher.java | 18 +++++++++++++
 .../apache/hadoop/hdfs/server/mover/Mover.java  | 27 +++++++++++++++-----
 .../hadoop/hdfs/server/mover/TestMover.java     |  2 +-
 4 files changed, 43 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ef03a4c/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 1491990..e843dcc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -716,6 +716,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8751. Remove setBlocks API from INodeFile and misc code cleanup. (Zhe
     Zhang via jing9)
 
+    HDFS-8541. Mover should exit with NO_MOVE_PROGRESS if there is no move
+    progress.  (Surendra Singh Lilhore via szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ef03a4c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
index 4a8f40f..298b86d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
@@ -317,6 +317,7 @@ public class Dispatcher {
         sendRequest(out, eb, accessToken);
         receiveResponse(in);
         nnc.getBytesMoved().addAndGet(block.getNumBytes());
+        target.getDDatanode().setHasSuccess();
         LOG.info("Successfully moved " + this);
       } catch (IOException e) {
         LOG.warn("Failed to move " + this + ": " + e.getMessage());
@@ -500,6 +501,7 @@ public class Dispatcher {
     /** blocks being moved but not confirmed yet */
     private final List<PendingMove> pendings;
     private volatile boolean hasFailure = false;
+    private volatile boolean hasSuccess = false;
     private final int maxConcurrentMoves;
 
     @Override
@@ -573,6 +575,10 @@ public class Dispatcher {
     void setHasFailure() {
       this.hasFailure = true;
     }
+
+    void setHasSuccess() {
+      this.hasSuccess = true;
+    }
   }
 
   /** A node that can be the sources of a block move */
@@ -965,6 +971,18 @@ public class Dispatcher {
   }
 
   /**
+   * @return true if some moves are success.
+   */
+  public static boolean checkForSuccess(
+      Iterable<? extends StorageGroup> targets) {
+    boolean hasSuccess = false;
+    for (StorageGroup t : targets) {
+      hasSuccess |= t.getDDatanode().hasSuccess;
+    }
+    return hasSuccess;
+  }
+
+  /**
    * Decide if the block is a good candidate to be moved from source to target.
    * A block is a good candidate if
    * 1. the block is not in the process of being moved/has not been moved;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ef03a4c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
index 344b9fc..afacebb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
@@ -269,10 +269,14 @@ public class Mover {
       // wait for pending move to finish and retry the failed migration
       boolean hasFailed = Dispatcher.waitForMoveCompletion(storages.targets
           .values());
-      if (hasFailed) {
+      boolean hasSuccess = Dispatcher.checkForSuccess(storages.targets
+          .values());
+      if (hasFailed && !hasSuccess) {
         if (retryCount.get() == retryMaxAttempts) {
-          throw new IOException("Failed to move some block's after "
+          result.setRetryFailed();
+          LOG.error("Failed to move some block's after "
               + retryMaxAttempts + " retries.");
+          return result;
         } else {
           retryCount.incrementAndGet();
         }
@@ -713,10 +717,12 @@ public class Mover {
 
     private boolean hasRemaining;
     private boolean noBlockMoved;
+    private boolean retryFailed;
 
     Result() {
       hasRemaining = false;
       noBlockMoved = true;
+      retryFailed = false;
     }
 
     boolean isHasRemaining() {
@@ -735,16 +741,25 @@ public class Mover {
       this.noBlockMoved = noBlockMoved;
     }
 
+    void setRetryFailed() {
+      this.retryFailed = true;
+    }
+
     /**
-     * @return SUCCESS if all moves are success and there is no remaining move.
+     * @return NO_MOVE_PROGRESS if no progress in move after some retry. Return
+     *         SUCCESS if all moves are success and there is no remaining move.
      *         Return NO_MOVE_BLOCK if there moves available but all the moves
      *         cannot be scheduled. Otherwise, return IN_PROGRESS since there
      *         must be some remaining moves.
      */
     ExitStatus getExitStatus() {
-      return !isHasRemaining() ? ExitStatus.SUCCESS
-          : isNoBlockMoved() ? ExitStatus.NO_MOVE_BLOCK
-              : ExitStatus.IN_PROGRESS;
+      if (retryFailed) {
+        return ExitStatus.NO_MOVE_PROGRESS;
+      } else {
+        return !isHasRemaining() ? ExitStatus.SUCCESS
+            : isNoBlockMoved() ? ExitStatus.NO_MOVE_BLOCK
+                : ExitStatus.IN_PROGRESS;
+      }
     }
 
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ef03a4c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
index 899b5c0..d3d814c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
@@ -404,7 +404,7 @@ public class TestMover {
       int rc = ToolRunner.run(conf, new Mover.Cli(),
           new String[] {"-p", file.toString()});
       Assert.assertEquals("Movement should fail after some retry",
-          ExitStatus.IO_EXCEPTION.getExitCode(), rc);
+          ExitStatus.NO_MOVE_PROGRESS.getExitCode(), rc);
     } finally {
       cluster.shutdown();
     }


[47/50] hadoop git commit: HDFS-7608: add CHANGES.txt

Posted by ar...@apache.org.
HDFS-7608: add CHANGES.txt


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

Branch: refs/heads/HDFS-7240
Commit: b7fb6ec4513de7d342c541eb3d9e14642286e2cf
Parents: 1d74cce
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Tue Jul 14 10:58:57 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Tue Jul 14 10:58:57 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7fb6ec4/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 e843dcc..86b1ea1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1038,6 +1038,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-2956. calling fetchdt without a --renewer argument throws NPE
     (vinayakumarb)
 
+    HDFS-7608: hdfs dfsclient newConnectedPeer has no write timeout (Xiaoyu Yao
+    via Colin P. McCabe)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[45/50] hadoop git commit: HADOOP-12191. Bzip2Factory is not thread safe. Contributed by Brahma Reddy Battula.

Posted by ar...@apache.org.
HADOOP-12191. Bzip2Factory is not thread safe. Contributed by Brahma Reddy Battula.


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

Branch: refs/heads/HDFS-7240
Commit: 4084eaf94347042d9b8cb1e86ba831036621dfdd
Parents: ac94ba3
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Tue Jul 14 19:13:45 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Tue Jul 14 19:14:42 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                   | 3 +++
 .../java/org/apache/hadoop/io/compress/bzip2/Bzip2Factory.java    | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4084eaf9/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 3121430..a9bd7de 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -973,6 +973,9 @@ Release 2.7.2 - UNRELEASED
     HADOOP-12186. ActiveStandbyElector shouldn't call monitorLockNodeAsync
     multiple times (zhihai xu via vinayakumarb)
 
+    HADOOP-12191. Bzip2Factory is not thread safe. (Brahma Reddy Battula
+    via ozawa)
+
 Release 2.7.1 - 2015-07-06 
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4084eaf9/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Factory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Factory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Factory.java
index 32fe910..45f1edd 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Factory.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Factory.java
@@ -44,7 +44,7 @@ public class Bzip2Factory {
    * @return <code>true</code> if native-bzip2 is loaded & initialized 
    *         and can be loaded for this job, else <code>false</code>
    */
-  public static boolean isNativeBzip2Loaded(Configuration conf) {
+  public static synchronized boolean isNativeBzip2Loaded(Configuration conf) {
     String libname = conf.get("io.compression.codec.bzip2.library", 
                               "system-native");
     if (!bzip2LibraryName.equals(libname)) {


[07/50] hadoop git commit: HDFS-8620. Clean up the checkstyle warinings about ClientProtocol. Contributed by Takanobu Asanuma.

Posted by ar...@apache.org.
HDFS-8620. Clean up the checkstyle warinings about ClientProtocol. Contributed by Takanobu Asanuma.


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

Branch: refs/heads/HDFS-7240
Commit: c0b8e4e5b5083631ed22d8d36c8992df7d34303c
Parents: b68701b
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Jul 7 14:01:19 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Jul 7 14:01:19 2015 -0700

----------------------------------------------------------------------
 .../hdfs/client/HdfsClientConfigKeys.java       |   3 +-
 .../hadoop/hdfs/protocol/ClientProtocol.java    | 606 +++++++++----------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 3 files changed, 288 insertions(+), 324 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c0b8e4e5/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 e6d579b..600c7ca 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
@@ -39,7 +39,8 @@ public interface HdfsClientConfigKeys {
   String  DFS_NAMENODE_HTTPS_ADDRESS_KEY = "dfs.namenode.https-address";
   String DFS_HA_NAMENODES_KEY_PREFIX = "dfs.ha.namenodes";
   int DFS_NAMENODE_RPC_PORT_DEFAULT = 8020;
-  String DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY = "dfs.namenode.kerberos.principal";
+  String DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY =
+      "dfs.namenode.kerberos.principal";
 
   /** dfs.client.retry configuration properties */
   interface Retry {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c0b8e4e5/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index ab41911..381be30 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -77,32 +77,33 @@ public interface ClientProtocol {
 
   /**
    * Until version 69, this class ClientProtocol served as both
-   * the client interface to the NN AND the RPC protocol used to 
+   * the client interface to the NN AND the RPC protocol used to
    * communicate with the NN.
-   * 
-   * This class is used by both the DFSClient and the 
+   *
+   * This class is used by both the DFSClient and the
    * NN server side to insulate from the protocol serialization.
-   * 
-   * If you are adding/changing this interface then you need to 
+   *
+   * If you are adding/changing this interface then you need to
    * change both this class and ALSO related protocol buffer
    * wire protocol definition in ClientNamenodeProtocol.proto.
-   * 
-   * For more details on protocol buffer wire protocol, please see 
+   *
+   * For more details on protocol buffer wire protocol, please see
    * .../org/apache/hadoop/hdfs/protocolPB/overview.html
-   * 
+   *
    * The log of historical changes can be retrieved from the svn).
    * 69: Eliminate overloaded method names.
-   * 
+   *
    * 69L is the last version id when this class was used for protocols
-   *  serialization. DO not update this version any further. 
+   *  serialization. DO not update this version any further.
    */
-  public static final long versionID = 69L;
-  
+  long versionID = 69L;
+
   ///////////////////////////////////////
   // File contents
   ///////////////////////////////////////
   /**
-   * Get locations of the blocks of the specified file within the specified range.
+   * Get locations of the blocks of the specified file
+   * within the specified range.
    * DataNode locations for each block are sorted by
    * the proximity to the client.
    * <p>
@@ -111,9 +112,9 @@ public interface ClientProtocol {
    * DataNode locations for each block are sorted by
    * the distance to the client's address.
    * <p>
-   * The client will then have to contact 
+   * The client will then have to contact
    * one of the indicated DataNodes to obtain the actual data.
-   * 
+   *
    * @param src file name
    * @param offset range start offset
    * @param length range length
@@ -126,11 +127,8 @@ public interface ClientProtocol {
    * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public LocatedBlocks getBlockLocations(String src,
-                                         long offset,
-                                         long length) 
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
+  LocatedBlocks getBlockLocations(String src, long offset, long length)
+      throws IOException;
 
   /**
    * Get server default values for a number of configuration params.
@@ -138,7 +136,7 @@ public interface ClientProtocol {
    * @throws IOException
    */
   @Idempotent
-  public FsServerDefaults getServerDefaults() throws IOException;
+  FsServerDefaults getServerDefaults() throws IOException;
 
   /**
    * Create a new file entry in the namespace.
@@ -148,36 +146,36 @@ public interface ClientProtocol {
    * The name-node does not have a notion of "current" directory for a client.
    * <p>
    * Once created, the file is visible and available for read to other clients.
-   * Although, other clients cannot {@link #delete(String, boolean)}, re-create or 
-   * {@link #rename(String, String)} it until the file is completed
+   * Although, other clients cannot {@link #delete(String, boolean)}, re-create
+   * or {@link #rename(String, String)} it until the file is completed
    * or explicitly as a result of lease expiration.
    * <p>
    * Blocks have a maximum size.  Clients that intend to create
-   * multi-block files must also use 
+   * multi-block files must also use
    * {@link #addBlock}
    *
    * @param src path of the file being created.
    * @param masked masked permission.
    * @param clientName name of the current client.
-   * @param flag indicates whether the file should be 
+   * @param flag indicates whether the file should be
    * overwritten if it already exists or create if it does not exist or append.
    * @param createParent create missing parent directory if true
    * @param replication block replication factor.
    * @param blockSize maximum block size.
    * @param supportedVersions CryptoProtocolVersions supported by the client
-   * 
+   *
    * @return the status of the created file, it could be null if the server
    *           doesn't support returning the file status
    * @throws AccessControlException If access is denied
    * @throws AlreadyBeingCreatedException if the path does not exist.
-   * @throws DSQuotaExceededException If file creation violates disk space 
+   * @throws DSQuotaExceededException If file creation violates disk space
    *           quota restriction
    * @throws FileAlreadyExistsException If file <code>src</code> already exists
    * @throws FileNotFoundException If parent of <code>src</code> does not exist
    *           and <code>createParent</code> is false
    * @throws ParentNotDirectoryException If parent of <code>src</code> is not a
    *           directory.
-   * @throws NSQuotaExceededException If file creation violates name space 
+   * @throws NSQuotaExceededException If file creation violates name space
    *           quota restriction
    * @throws SafeModeException create not allowed in safemode
    * @throws UnresolvedLinkException If <code>src</code> contains a symlink
@@ -190,33 +188,29 @@ public interface ClientProtocol {
    * <em>Note that create with {@link CreateFlag#OVERWRITE} is idempotent.</em>
    */
   @AtMostOnce
-  public HdfsFileStatus create(String src, FsPermission masked,
+  HdfsFileStatus create(String src, FsPermission masked,
       String clientName, EnumSetWritable<CreateFlag> flag,
-      boolean createParent, short replication, long blockSize, 
+      boolean createParent, short replication, long blockSize,
       CryptoProtocolVersion[] supportedVersions)
-      throws AccessControlException, AlreadyBeingCreatedException,
-      DSQuotaExceededException, FileAlreadyExistsException,
-      FileNotFoundException, NSQuotaExceededException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      SnapshotAccessControlException, IOException;
+      throws IOException;
 
   /**
-   * Append to the end of the file. 
+   * Append to the end of the file.
    * @param src path of the file being created.
    * @param clientName name of the current client.
    * @param flag indicates whether the data is appended to a new block.
    * @return wrapper with information about the last partial block and file
    *    status if any
-   * @throws AccessControlException if permission to append file is 
-   * denied by the system. As usually on the client side the exception will 
+   * @throws AccessControlException if permission to append file is
+   * denied by the system. As usually on the client side the exception will
    * be wrapped into {@link org.apache.hadoop.ipc.RemoteException}.
    * Allows appending to an existing file if the server is
    * configured with the parameter dfs.support.append set to true, otherwise
    * throws an IOException.
-   * 
+   *
    * @throws AccessControlException If permission to append to file is denied
    * @throws FileNotFoundException If file <code>src</code> is not found
-   * @throws DSQuotaExceededException If append violates disk space quota 
+   * @throws DSQuotaExceededException If append violates disk space quota
    *           restriction
    * @throws SafeModeException append not allowed in safemode
    * @throws UnresolvedLinkException If <code>src</code> contains a symlink
@@ -227,27 +221,25 @@ public interface ClientProtocol {
    * @throws UnsupportedOperationException if append is not supported
    */
   @AtMostOnce
-  public LastBlockWithStatus append(String src, String clientName,
-      EnumSetWritable<CreateFlag> flag) throws AccessControlException,
-      DSQuotaExceededException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+  LastBlockWithStatus append(String src, String clientName,
+      EnumSetWritable<CreateFlag> flag) throws IOException;
 
   /**
    * Set replication for an existing file.
    * <p>
    * The NameNode sets replication to the new value and returns.
-   * The actual block replication is not expected to be performed during  
-   * this method call. The blocks will be populated or removed in the 
+   * The actual block replication is not expected to be performed during
+   * this method call. The blocks will be populated or removed in the
    * background as the result of the routine block maintenance procedures.
-   * 
+   *
    * @param src file name
    * @param replication new replication
-   * 
+   *
    * @return true if successful;
    *         false if file does not exist or is a directory
    *
    * @throws AccessControlException If access is denied
-   * @throws DSQuotaExceededException If replication violates disk space 
+   * @throws DSQuotaExceededException If replication violates disk space
    *           quota restriction
    * @throws FileNotFoundException If file <code>src</code> is not found
    * @throws SafeModeException not allowed in safemode
@@ -256,21 +248,19 @@ public interface ClientProtocol {
    * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public boolean setReplication(String src, short replication)
-      throws AccessControlException, DSQuotaExceededException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      SnapshotAccessControlException, IOException;
+  boolean setReplication(String src, short replication)
+      throws IOException;
 
   /**
    * Get all the available block storage policies.
    * @return All the in-use block storage policies currently.
    */
   @Idempotent
-  public BlockStoragePolicy[] getStoragePolicies() throws IOException;
+  BlockStoragePolicy[] getStoragePolicies() throws IOException;
 
   /**
-   * Set the storage policy for a file/directory
-   * @param src Path of an existing file/directory. 
+   * Set the storage policy for a file/directory.
+   * @param src Path of an existing file/directory.
    * @param policyName The name of the storage policy
    * @throws SnapshotAccessControlException If access is denied
    * @throws UnresolvedLinkException if <code>src</code> contains a symlink
@@ -278,13 +268,12 @@ public interface ClientProtocol {
    * @throws QuotaExceededException If changes violate the quota restriction
    */
   @Idempotent
-  public void setStoragePolicy(String src, String policyName)
-      throws SnapshotAccessControlException, UnresolvedLinkException,
-      FileNotFoundException, QuotaExceededException, IOException;
+  void setStoragePolicy(String src, String policyName)
+      throws IOException;
 
   /**
    * Set permissions for an existing file/directory.
-   * 
+   *
    * @throws AccessControlException If access is denied
    * @throws FileNotFoundException If file <code>src</code> is not found
    * @throws SafeModeException not allowed in safemode
@@ -293,9 +282,8 @@ public interface ClientProtocol {
    * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public void setPermission(String src, FsPermission permission)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+  void setPermission(String src, FsPermission permission)
+      throws IOException;
 
   /**
    * Set Owner of a path (i.e. a file or a directory).
@@ -312,16 +300,15 @@ public interface ClientProtocol {
    * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public void setOwner(String src, String username, String groupname)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+  void setOwner(String src, String username, String groupname)
+      throws IOException;
 
   /**
    * The client can give up on a block by calling abandonBlock().
-   * The client can then either obtain a new block, or complete or abandon the 
+   * The client can then either obtain a new block, or complete or abandon the
    * file.
    * Any partial writes to the block will be discarded.
-   * 
+   *
    * @param b         Block to abandon
    * @param fileId    The id of the file where the block resides.  Older clients
    *                    will pass GRANDFATHER_INODE_ID here.
@@ -334,19 +321,18 @@ public interface ClientProtocol {
    * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public void abandonBlock(ExtendedBlock b, long fileId,
+  void abandonBlock(ExtendedBlock b, long fileId,
       String src, String holder)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
+      throws IOException;
 
   /**
-   * A client that wants to write an additional block to the 
+   * A client that wants to write an additional block to the
    * indicated filename (which must currently be open for writing)
-   * should call addBlock().  
+   * should call addBlock().
    *
    * addBlock() allocates a new block and datanodes the block data
    * should be replicated to.
-   * 
+   *
    * addBlock() also commits the previous block by reporting
    * to the name-node the actual generation stamp and the length
    * of the block that the client has transmitted to data-nodes.
@@ -372,16 +358,14 @@ public interface ClientProtocol {
    * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public LocatedBlock addBlock(String src, String clientName,
-      ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId, 
+  LocatedBlock addBlock(String src, String clientName,
+      ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId,
       String[] favoredNodes)
-      throws AccessControlException, FileNotFoundException,
-      NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
-      IOException;
+      throws IOException;
 
-  /** 
+  /**
    * Get a datanode for an existing pipeline.
-   * 
+   *
    * @param src the file being written
    * @param fileId the ID of the file being written
    * @param blk the block being written
@@ -389,9 +373,9 @@ public interface ClientProtocol {
    * @param excludes the excluded nodes
    * @param numAdditionalNodes number of additional datanodes
    * @param clientName the name of the client
-   * 
+   *
    * @return the located block.
-   * 
+   *
    * @throws AccessControlException If access is denied
    * @throws FileNotFoundException If file <code>src</code> is not found
    * @throws SafeModeException create not allowed in safemode
@@ -399,22 +383,21 @@ public interface ClientProtocol {
    * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public LocatedBlock getAdditionalDatanode(final String src,
+  LocatedBlock getAdditionalDatanode(final String src,
       final long fileId, final ExtendedBlock blk,
       final DatanodeInfo[] existings,
       final String[] existingStorageIDs,
       final DatanodeInfo[] excludes,
       final int numAdditionalNodes, final String clientName
-      ) throws AccessControlException, FileNotFoundException,
-          SafeModeException, UnresolvedLinkException, IOException;
+      ) throws IOException;
 
   /**
-   * The client is done writing data to the given filename, and would 
-   * like to complete it.  
+   * The client is done writing data to the given filename, and would
+   * like to complete it.
    *
    * The function returns whether the file has been closed successfully.
    * If the function returns false, the caller should try again.
-   * 
+   *
    * close() also commits the last block of file by reporting
    * to the name-node the actual generation stamp and the length
    * of the block that the client has transmitted to data-nodes.
@@ -434,14 +417,13 @@ public interface ClientProtocol {
    * @throws AccessControlException If access is denied
    * @throws FileNotFoundException If file <code>src</code> is not found
    * @throws SafeModeException create not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink 
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
    * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public boolean complete(String src, String clientName,
+  boolean complete(String src, String clientName,
                           ExtendedBlock last, long fileId)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException;
+      throws IOException;
 
   /**
    * The client wants to report corrupted blocks (blocks with specified
@@ -449,7 +431,7 @@ public interface ClientProtocol {
    * @param blocks Array of located blocks to report
    */
   @Idempotent
-  public void reportBadBlocks(LocatedBlock[] blocks) throws IOException;
+  void reportBadBlocks(LocatedBlock[] blocks) throws IOException;
 
   ///////////////////////////////////////
   // Namespace management
@@ -460,17 +442,17 @@ public interface ClientProtocol {
    * @param dst new name.
    * @return true if successful, or false if the old name does not exist
    * or if the new name already belongs to the namespace.
-   * 
+   *
    * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException an I/O error occurred 
+   * @throws IOException an I/O error occurred
    */
   @AtMostOnce
-  public boolean rename(String src, String dst) 
-      throws UnresolvedLinkException, SnapshotAccessControlException, IOException;
+  boolean rename(String src, String dst)
+      throws IOException;
 
   /**
-   * Moves blocks from srcs to trg and delete srcs
-   * 
+   * Moves blocks from srcs to trg and delete srcs.
+   *
    * @param trg existing file
    * @param srcs - list of existing files (same block size, same replication)
    * @throws IOException if some arguments are invalid
@@ -479,8 +461,8 @@ public interface ClientProtocol {
    * @throws SnapshotAccessControlException if path is in RO snapshot
    */
   @AtMostOnce
-  public void concat(String trg, String[] srcs) 
-      throws IOException, UnresolvedLinkException, SnapshotAccessControlException;
+  void concat(String trg, String[] srcs)
+      throws IOException;
 
   /**
    * Rename src to dst.
@@ -491,7 +473,7 @@ public interface ClientProtocol {
    * </ul>
    * <p>
    * Without OVERWRITE option, rename fails if the dst already exists.
-   * With OVERWRITE option, rename overwrites the dst, if it is a file 
+   * With OVERWRITE option, rename overwrites the dst, if it is a file
    * or an empty directory. Rename fails if dst is a non-empty directory.
    * <p>
    * This implementation of rename is atomic.
@@ -499,17 +481,17 @@ public interface ClientProtocol {
    * @param src existing file or directory name.
    * @param dst new name.
    * @param options Rename options
-   * 
+   *
    * @throws AccessControlException If access is denied
-   * @throws DSQuotaExceededException If rename violates disk space 
+   * @throws DSQuotaExceededException If rename violates disk space
    *           quota restriction
    * @throws FileAlreadyExistsException If <code>dst</code> already exists and
-   *           <code>options</options> has {@link Rename#OVERWRITE} option
+   *           <code>options</code> has {@link Rename#OVERWRITE} option
    *           false.
    * @throws FileNotFoundException If <code>src</code> does not exist
-   * @throws NSQuotaExceededException If rename violates namespace 
+   * @throws NSQuotaExceededException If rename violates namespace
    *           quota restriction
-   * @throws ParentNotDirectoryException If parent of <code>dst</code> 
+   * @throws ParentNotDirectoryException If parent of <code>dst</code>
    *           is not a directory
    * @throws SafeModeException rename not allowed in safemode
    * @throws UnresolvedLinkException If <code>src</code> or
@@ -518,11 +500,8 @@ public interface ClientProtocol {
    * @throws IOException If an I/O error occurred
    */
   @AtMostOnce
-  public void rename2(String src, String dst, Options.Rename... options)
-      throws AccessControlException, DSQuotaExceededException,
-      FileAlreadyExistsException, FileNotFoundException,
-      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+  void rename2(String src, String dst, Options.Rename... options)
+      throws IOException;
 
   /**
    * Truncate file src to new size.
@@ -550,21 +529,20 @@ public interface ClientProtocol {
    * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public boolean truncate(String src, long newLength, String clientName)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+  boolean truncate(String src, long newLength, String clientName)
+      throws IOException;
 
   /**
    * Delete the given file or directory from the file system.
    * <p>
-   * same as delete but provides a way to avoid accidentally 
-   * deleting non empty directories programmatically. 
+   * same as delete but provides a way to avoid accidentally
+   * deleting non empty directories programmatically.
    * @param src existing name
    * @param recursive if true deletes a non empty directory recursively,
    * else throws an exception.
-   * @return true only if the existing file or directory was actually removed 
+   * @return true only if the existing file or directory was actually removed
    * from the file system.
-   * 
+   *
    * @throws AccessControlException If access is denied
    * @throws FileNotFoundException If file <code>src</code> is not found
    * @throws SafeModeException create not allowed in safemode
@@ -573,10 +551,9 @@ public interface ClientProtocol {
    * @throws IOException If an I/O error occurred
    */
   @AtMostOnce
-  public boolean delete(String src, boolean recursive)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
-  
+  boolean delete(String src, boolean recursive)
+      throws IOException;
+
   /**
    * Create a directory (or hierarchy of directories) with the given
    * name and permission.
@@ -591,8 +568,9 @@ public interface ClientProtocol {
    * @throws FileAlreadyExistsException If <code>src</code> already exists
    * @throws FileNotFoundException If parent of <code>src</code> does not exist
    *           and <code>createParent</code> is false
-   * @throws NSQuotaExceededException If file creation violates quota restriction
-   * @throws ParentNotDirectoryException If parent of <code>src</code> 
+   * @throws NSQuotaExceededException If file creation violates quota
+   *           restriction
+   * @throws ParentNotDirectoryException If parent of <code>src</code>
    *           is not a directory
    * @throws SafeModeException create not allowed in safemode
    * @throws UnresolvedLinkException If <code>src</code> contains a symlink
@@ -603,14 +581,11 @@ public interface ClientProtocol {
    * @throws InvalidPathException If <code>src</code> is invalid
    */
   @Idempotent
-  public boolean mkdirs(String src, FsPermission masked, boolean createParent)
-      throws AccessControlException, FileAlreadyExistsException,
-      FileNotFoundException, NSQuotaExceededException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      SnapshotAccessControlException, IOException;
+  boolean mkdirs(String src, FsPermission masked, boolean createParent)
+      throws IOException;
 
   /**
-   * Get a partial listing of the indicated directory
+   * Get a partial listing of the indicated directory.
    *
    * @param src the directory name
    * @param startAfter the name to start listing after encoded in java UTF8
@@ -624,20 +599,17 @@ public interface ClientProtocol {
    * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public DirectoryListing getListing(String src,
-                                     byte[] startAfter,
-                                     boolean needLocation)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
-  
-  /**
-   * Get listing of all the snapshottable directories
-   * 
+  DirectoryListing getListing(String src, byte[] startAfter,
+      boolean needLocation) throws IOException;
+
+  /**
+   * Get listing of all the snapshottable directories.
+   *
    * @return Information about all the current snapshottable directory
    * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+  SnapshottableDirectoryStatus[] getSnapshottableDirListing()
       throws IOException;
 
   ///////////////////////////////////////
@@ -646,7 +618,7 @@ public interface ClientProtocol {
 
   /**
    * Client programs can cause stateful changes in the NameNode
-   * that affect other clients.  A client may obtain a file and 
+   * that affect other clients.  A client may obtain a file and
    * neither abandon nor complete it.  A client might hold a series
    * of locks that prevent other clients from proceeding.
    * Clearly, it would be bad if a client held a bunch of locks
@@ -664,29 +636,28 @@ public interface ClientProtocol {
    * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public void renewLease(String clientName) throws AccessControlException,
-      IOException;
+  void renewLease(String clientName) throws IOException;
 
   /**
    * Start lease recovery.
    * Lightweight NameNode operation to trigger lease recovery
-   * 
+   *
    * @param src path of the file to start lease recovery
    * @param clientName name of the current client
    * @return true if the file is already closed
    * @throws IOException
    */
   @Idempotent
-  public boolean recoverLease(String src, String clientName) throws IOException;
+  boolean recoverLease(String src, String clientName) throws IOException;
+
+  int GET_STATS_CAPACITY_IDX = 0;
+  int GET_STATS_USED_IDX = 1;
+  int GET_STATS_REMAINING_IDX = 2;
+  int GET_STATS_UNDER_REPLICATED_IDX = 3;
+  int GET_STATS_CORRUPT_BLOCKS_IDX = 4;
+  int GET_STATS_MISSING_BLOCKS_IDX = 5;
+  int GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX = 6;
 
-  public int GET_STATS_CAPACITY_IDX = 0;
-  public int GET_STATS_USED_IDX = 1;
-  public int GET_STATS_REMAINING_IDX = 2;
-  public int GET_STATS_UNDER_REPLICATED_IDX = 3;
-  public int GET_STATS_CORRUPT_BLOCKS_IDX = 4;
-  public int GET_STATS_MISSING_BLOCKS_IDX = 5;
-  public int GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX = 6;
-  
   /**
    * Get a set of statistics about the filesystem.
    * Right now, only seven values are returned.
@@ -700,11 +671,11 @@ public interface ClientProtocol {
    * <li> [6] contains number of blocks which have replication factor
    *          1 and have lost the only replica. </li>
    * </ul>
-   * Use public constants like {@link #GET_STATS_CAPACITY_IDX} in place of 
+   * Use public constants like {@link #GET_STATS_CAPACITY_IDX} in place of
    * actual numbers to index into the array.
    */
   @Idempotent
-  public long[] getStats() throws IOException;
+  long[] getStats() throws IOException;
 
   /**
    * Get a report on the system's current datanodes.
@@ -713,14 +684,14 @@ public interface ClientProtocol {
    * otherwise all datanodes if type is ALL.
    */
   @Idempotent
-  public DatanodeInfo[] getDatanodeReport(HdfsConstants.DatanodeReportType type)
+  DatanodeInfo[] getDatanodeReport(HdfsConstants.DatanodeReportType type)
       throws IOException;
 
   /**
    * Get a report on the current datanode storages.
    */
   @Idempotent
-  public DatanodeStorageReport[] getDatanodeStorageReport(
+  DatanodeStorageReport[] getDatanodeStorageReport(
       HdfsConstants.DatanodeReportType type) throws IOException;
 
   /**
@@ -728,11 +699,11 @@ public interface ClientProtocol {
    * @param filename The name of the file
    * @return The number of bytes in each block
    * @throws IOException
-   * @throws UnresolvedLinkException if the path contains a symlink. 
+   * @throws UnresolvedLinkException if the path contains a symlink.
    */
   @Idempotent
-  public long getPreferredBlockSize(String filename) 
-      throws IOException, UnresolvedLinkException;
+  long getPreferredBlockSize(String filename)
+      throws IOException;
 
   /**
    * Enter, leave or get safe mode.
@@ -740,16 +711,17 @@ public interface ClientProtocol {
    * Safe mode is a name node state when it
    * <ol><li>does not accept changes to name space (read-only), and</li>
    * <li>does not replicate or delete blocks.</li></ol>
-   * 
+   *
    * <p>
    * Safe mode is entered automatically at name node startup.
    * Safe mode can also be entered manually using
-   * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean) setSafeMode(SafeModeAction.SAFEMODE_ENTER,false)}.
+   * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean)
+   * setSafeMode(SafeModeAction.SAFEMODE_ENTER,false)}.
    * <p>
    * At startup the name node accepts data node reports collecting
    * information about block locations.
    * In order to leave safe mode it needs to collect a configurable
-   * percentage called threshold of blocks, which satisfy the minimal 
+   * percentage called threshold of blocks, which satisfy the minimal
    * replication condition.
    * The minimal replication condition is that each block must have at least
    * <tt>dfs.namenode.replication.min</tt> replicas.
@@ -760,37 +732,40 @@ public interface ClientProtocol {
    * Then the name node leaves safe mode.
    * <p>
    * If safe mode is turned on manually using
-   * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean) setSafeMode(SafeModeAction.SAFEMODE_ENTER,false)}
+   * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean)
+   * setSafeMode(SafeModeAction.SAFEMODE_ENTER,false)}
    * then the name node stays in safe mode until it is manually turned off
-   * using {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean) setSafeMode(SafeModeAction.SAFEMODE_LEAVE,false)}.
+   * using {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean)
+   * setSafeMode(SafeModeAction.SAFEMODE_LEAVE,false)}.
    * Current state of the name node can be verified using
-   * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean) setSafeMode(SafeModeAction.SAFEMODE_GET,false)}
+   * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean)
+   * setSafeMode(SafeModeAction.SAFEMODE_GET,false)}
    * <h4>Configuration parameters:</h4>
    * <tt>dfs.safemode.threshold.pct</tt> is the threshold parameter.<br>
    * <tt>dfs.safemode.extension</tt> is the safe mode extension parameter.<br>
    * <tt>dfs.namenode.replication.min</tt> is the minimal replication parameter.
-   * 
+   *
    * <h4>Special cases:</h4>
-   * The name node does not enter safe mode at startup if the threshold is 
+   * The name node does not enter safe mode at startup if the threshold is
    * set to 0 or if the name space is empty.<br>
-   * If the threshold is set to 1 then all blocks need to have at least 
+   * If the threshold is set to 1 then all blocks need to have at least
    * minimal replication.<br>
-   * If the threshold value is greater than 1 then the name node will not be 
+   * If the threshold value is greater than 1 then the name node will not be
    * able to turn off safe mode automatically.<br>
    * Safe mode can always be turned off manually.
-   * 
+   *
    * @param action  <ul> <li>0 leave safe mode;</li>
    *                <li>1 enter safe mode;</li>
    *                <li>2 get safe mode state.</li></ul>
    * @param isChecked If true then action will be done only in ActiveNN.
-   * 
-   * @return <ul><li>0 if the safe mode is OFF or</li> 
+   *
+   * @return <ul><li>0 if the safe mode is OFF or</li>
    *         <li>1 if the safe mode is ON.</li></ul>
-   *                   
+   *
    * @throws IOException
    */
   @Idempotent
-  public boolean setSafeMode(HdfsConstants.SafeModeAction action, boolean isChecked) 
+  boolean setSafeMode(HdfsConstants.SafeModeAction action, boolean isChecked)
       throws IOException;
 
   /**
@@ -808,47 +783,45 @@ public interface ClientProtocol {
    * @throws IOException if image creation failed.
    */
   @AtMostOnce
-  public boolean saveNamespace(long timeWindow, long txGap) throws IOException;
+  boolean saveNamespace(long timeWindow, long txGap) throws IOException;
 
-  
   /**
    * Roll the edit log.
    * Requires superuser privileges.
-   * 
+   *
    * @throws AccessControlException if the superuser privilege is violated
    * @throws IOException if log roll fails
    * @return the txid of the new segment
    */
   @Idempotent
-  public long rollEdits() throws AccessControlException, IOException;
+  long rollEdits() throws IOException;
 
   /**
    * Enable/Disable restore failed storage.
    * <p>
    * sets flag to enable restore of failed storage replicas
-   * 
+   *
    * @throws AccessControlException if the superuser privilege is violated.
    */
   @Idempotent
-  public boolean restoreFailedStorage(String arg) 
-      throws AccessControlException, IOException;
+  boolean restoreFailedStorage(String arg) throws IOException;
 
   /**
-   * Tells the namenode to reread the hosts and exclude files. 
+   * Tells the namenode to reread the hosts and exclude files.
    * @throws IOException
    */
   @Idempotent
-  public void refreshNodes() throws IOException;
+  void refreshNodes() throws IOException;
 
   /**
    * Finalize previous upgrade.
    * Remove file system state saved during the upgrade.
    * The upgrade will become irreversible.
-   * 
+   *
    * @throws IOException
    */
   @Idempotent
-  public void finalizeUpgrade() throws IOException;
+  void finalizeUpgrade() throws IOException;
 
   /**
    * Rolling upgrade operations.
@@ -857,7 +830,7 @@ public interface ClientProtocol {
    * progress, returns null.
    */
   @Idempotent
-  public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
+  RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
       throws IOException;
 
   /**
@@ -871,9 +844,9 @@ public interface ClientProtocol {
    * cookie returned from the previous call.
    */
   @Idempotent
-  public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
+  CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
       throws IOException;
-  
+
   /**
    * Dumps namenode data structures into specified file. If the file
    * already exists, then append.
@@ -881,7 +854,7 @@ public interface ClientProtocol {
    * @throws IOException
    */
   @Idempotent
-  public void metaSave(String filename) throws IOException;
+  void metaSave(String filename) throws IOException;
 
   /**
    * Tell all datanodes to use a new, non-persistent bandwidth value for
@@ -891,8 +864,8 @@ public interface ClientProtocol {
    * @throws IOException
    */
   @Idempotent
-  public void setBalancerBandwidth(long bandwidth) throws IOException;
-  
+  void setBalancerBandwidth(long bandwidth) throws IOException;
+
   /**
    * Get the file info for a specific file or directory.
    * @param src The string representation of the path to the file
@@ -901,29 +874,27 @@ public interface ClientProtocol {
    *         or null if file not found
    * @throws AccessControlException permission denied
    * @throws FileNotFoundException file <code>src</code> is not found
-   * @throws UnresolvedLinkException if the path contains a symlink. 
-   * @throws IOException If an I/O error occurred        
+   * @throws UnresolvedLinkException if the path contains a symlink.
+   * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException;
-  
+  HdfsFileStatus getFileInfo(String src) throws IOException;
+
   /**
-   * Get the close status of a file
+   * Get the close status of a file.
    * @param src The string representation of the path to the file
    *
    * @return return true if file is closed
    * @throws AccessControlException permission denied
    * @throws FileNotFoundException file <code>src</code> is not found
    * @throws UnresolvedLinkException if the path contains a symlink.
-   * @throws IOException If an I/O error occurred     
+   * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public boolean isFileClosed(String src) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException;
-  
+  boolean isFileClosed(String src) throws IOException;
+
   /**
-   * Get the file info for a specific file or directory. If the path 
+   * Get the file info for a specific file or directory. If the path
    * refers to a symlink then the FileStatus of the symlink is returned.
    * @param src The string representation of the path to the file
    *
@@ -932,58 +903,55 @@ public interface ClientProtocol {
    *
    * @throws AccessControlException permission denied
    * @throws UnresolvedLinkException if <code>src</code> contains a symlink
-   * @throws IOException If an I/O error occurred        
+   * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public HdfsFileStatus getFileLinkInfo(String src)
-      throws AccessControlException, UnresolvedLinkException, IOException;
-  
+  HdfsFileStatus getFileLinkInfo(String src) throws IOException;
+
   /**
    * Get {@link ContentSummary} rooted at the specified directory.
    * @param path The string representation of the path
    *
    * @throws AccessControlException permission denied
    * @throws FileNotFoundException file <code>path</code> is not found
-   * @throws UnresolvedLinkException if <code>path</code> contains a symlink. 
+   * @throws UnresolvedLinkException if <code>path</code> contains a symlink.
    * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public ContentSummary getContentSummary(String path)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
+  ContentSummary getContentSummary(String path) throws IOException;
 
   /**
    * Set the quota for a directory.
    * @param path  The string representation of the path to the directory
-   * @param namespaceQuota Limit on the number of names in the tree rooted 
+   * @param namespaceQuota Limit on the number of names in the tree rooted
    *                       at the directory
-   * @param storagespaceQuota Limit on storage space occupied all the files under
-   *                       this directory.
+   * @param storagespaceQuota Limit on storage space occupied all the files
+   *                       under this directory.
    * @param type StorageType that the space quota is intended to be set on.
-   *             It may be null when called by traditional space/namespace quota.
-   *             When type is is not null, the storagespaceQuota parameter is for
-   *             type specified and namespaceQuota must be
+   *             It may be null when called by traditional space/namespace
+   *             quota. When type is is not null, the storagespaceQuota
+   *             parameter is for type specified and namespaceQuota must be
    *             {@link HdfsConstants#QUOTA_DONT_SET}.
    *
    * <br><br>
-   *                       
-   * The quota can have three types of values : (1) 0 or more will set 
+   *
+   * The quota can have three types of values : (1) 0 or more will set
    * the quota to that value, (2) {@link HdfsConstants#QUOTA_DONT_SET}  implies
    * the quota will not be changed, and (3) {@link HdfsConstants#QUOTA_RESET}
    * implies the quota will be reset. Any other value is a runtime error.
-   * 
+   *
    * @throws AccessControlException permission denied
    * @throws FileNotFoundException file <code>path</code> is not found
-   * @throws QuotaExceededException if the directory size 
+   * @throws QuotaExceededException if the directory size
    *           is greater than the given quota
-   * @throws UnresolvedLinkException if the <code>path</code> contains a symlink. 
+   * @throws UnresolvedLinkException if the <code>path</code> contains
+   *           a symlink.
    * @throws SnapshotAccessControlException if path is in RO snapshot
    * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
-      StorageType type) throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+  void setQuota(String path, long namespaceQuota, long storagespaceQuota,
+      StorageType type) throws IOException;
 
   /**
    * Write all metadata for this file into persistent storage.
@@ -992,39 +960,35 @@ public interface ClientProtocol {
    * @param inodeId The inode ID, or GRANDFATHER_INODE_ID if the client is
    *                too old to support fsync with inode IDs.
    * @param client The string representation of the client
-   * @param lastBlockLength The length of the last block (under construction) 
-   *                        to be reported to NameNode 
+   * @param lastBlockLength The length of the last block (under construction)
+   *                        to be reported to NameNode
    * @throws AccessControlException permission denied
    * @throws FileNotFoundException file <code>src</code> is not found
-   * @throws UnresolvedLinkException if <code>src</code> contains a symlink. 
+   * @throws UnresolvedLinkException if <code>src</code> contains a symlink.
    * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public void fsync(String src, long inodeId, String client,
-                    long lastBlockLength)
-      throws AccessControlException, FileNotFoundException, 
-      UnresolvedLinkException, IOException;
+  void fsync(String src, long inodeId, String client, long lastBlockLength)
+      throws IOException;
 
   /**
    * Sets the modification and access time of the file to the specified time.
    * @param src The string representation of the path
    * @param mtime The number of milliseconds since Jan 1, 1970.
-   *              Setting mtime to -1 means that modification time should not be set
-   *              by this call.
+   *              Setting mtime to -1 means that modification time should not
+   *              be set by this call.
    * @param atime The number of milliseconds since Jan 1, 1970.
    *              Setting atime to -1 means that access time should not be set
    *              by this call.
-   *              
+   *
    * @throws AccessControlException permission denied
    * @throws FileNotFoundException file <code>src</code> is not found
-   * @throws UnresolvedLinkException if <code>src</code> contains a symlink. 
+   * @throws UnresolvedLinkException if <code>src</code> contains a symlink.
    * @throws SnapshotAccessControlException if path is in RO snapshot
    * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public void setTimes(String src, long mtime, long atime)
-      throws AccessControlException, FileNotFoundException, 
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+  void setTimes(String src, long mtime, long atime) throws IOException;
 
   /**
    * Create symlink to a file or directory.
@@ -1041,16 +1005,13 @@ public interface ClientProtocol {
    *           and <code>createParent</code> is false
    * @throws ParentNotDirectoryException If parent of <code>link</code> is not a
    *           directory.
-   * @throws UnresolvedLinkException if <code>link</target> contains a symlink. 
+   * @throws UnresolvedLinkException if <code>link</code> contains a symlink.
    * @throws SnapshotAccessControlException if path is in RO snapshot
    * @throws IOException If an I/O error occurred
    */
   @AtMostOnce
-  public void createSymlink(String target, String link, FsPermission dirPerm,
-      boolean createParent) throws AccessControlException,
-      FileAlreadyExistsException, FileNotFoundException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      SnapshotAccessControlException, IOException;
+  void createSymlink(String target, String link, FsPermission dirPerm,
+      boolean createParent) throws IOException;
 
   /**
    * Return the target of the given symlink. If there is an intermediate
@@ -1065,28 +1026,27 @@ public interface ClientProtocol {
    *           or an I/O error occurred
    */
   @Idempotent
-  public String getLinkTarget(String path) throws AccessControlException,
-      FileNotFoundException, IOException; 
-  
+  String getLinkTarget(String path) throws IOException;
+
   /**
-   * Get a new generation stamp together with an access token for 
+   * Get a new generation stamp together with an access token for
    * a block under construction
-   * 
+   *
    * This method is called only when a client needs to recover a failed
    * pipeline or set up a pipeline for appending to a block.
-   * 
+   *
    * @param block a block
    * @param clientName the name of the client
    * @return a located block with a new generation stamp and an access token
    * @throws IOException if any error occurs
    */
   @Idempotent
-  public LocatedBlock updateBlockForPipeline(ExtendedBlock block,
+  LocatedBlock updateBlockForPipeline(ExtendedBlock block,
       String clientName) throws IOException;
 
   /**
-   * Update a pipeline for a block under construction
-   * 
+   * Update a pipeline for a block under construction.
+   *
    * @param clientName the name of the client
    * @param oldBlock the old block
    * @param newBlock the new block containing new generation stamp and length
@@ -1094,104 +1054,104 @@ public interface ClientProtocol {
    * @throws IOException if any error occurs
    */
   @AtMostOnce
-  public void updatePipeline(String clientName, ExtendedBlock oldBlock, 
+  void updatePipeline(String clientName, ExtendedBlock oldBlock,
       ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
       throws IOException;
 
   /**
    * Get a valid Delegation Token.
-   * 
+   *
    * @param renewer the designated renewer for the token
    * @return Token<DelegationTokenIdentifier>
    * @throws IOException
    */
   @Idempotent
-  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer) 
+  Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
       throws IOException;
 
   /**
    * Renew an existing delegation token.
-   * 
+   *
    * @param token delegation token obtained earlier
    * @return the new expiration time
    * @throws IOException
    */
   @Idempotent
-  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
+  long renewDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException;
-  
+
   /**
    * Cancel an existing delegation token.
-   * 
+   *
    * @param token delegation token
    * @throws IOException
    */
   @Idempotent
-  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+  void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException;
-  
+
   /**
    * @return encryption key so a client can encrypt data sent via the
    *         DataTransferProtocol to/from DataNodes.
    * @throws IOException
    */
   @Idempotent
-  public DataEncryptionKey getDataEncryptionKey() throws IOException;
-  
+  DataEncryptionKey getDataEncryptionKey() throws IOException;
+
   /**
-   * Create a snapshot
+   * Create a snapshot.
    * @param snapshotRoot the path that is being snapshotted
    * @param snapshotName name of the snapshot created
    * @return the snapshot path.
    * @throws IOException
    */
   @AtMostOnce
-  public String createSnapshot(String snapshotRoot, String snapshotName)
+  String createSnapshot(String snapshotRoot, String snapshotName)
       throws IOException;
 
   /**
-   * Delete a specific snapshot of a snapshottable directory
+   * Delete a specific snapshot of a snapshottable directory.
    * @param snapshotRoot  The snapshottable directory
    * @param snapshotName Name of the snapshot for the snapshottable directory
    * @throws IOException
    */
   @AtMostOnce
-  public void deleteSnapshot(String snapshotRoot, String snapshotName)
+  void deleteSnapshot(String snapshotRoot, String snapshotName)
       throws IOException;
-  
+
   /**
-   * Rename a snapshot
-   * @param snapshotRoot the directory path where the snapshot was taken 
+   * Rename a snapshot.
+   * @param snapshotRoot the directory path where the snapshot was taken
    * @param snapshotOldName old name of the snapshot
    * @param snapshotNewName new name of the snapshot
    * @throws IOException
    */
   @AtMostOnce
-  public void renameSnapshot(String snapshotRoot, String snapshotOldName,
+  void renameSnapshot(String snapshotRoot, String snapshotOldName,
       String snapshotNewName) throws IOException;
-  
+
   /**
    * Allow snapshot on a directory.
    * @param snapshotRoot the directory to be snapped
    * @throws IOException on error
    */
   @Idempotent
-  public void allowSnapshot(String snapshotRoot)
+  void allowSnapshot(String snapshotRoot)
       throws IOException;
-    
+
   /**
    * Disallow snapshot on a directory.
    * @param snapshotRoot the directory to disallow snapshot
    * @throws IOException on error
    */
   @Idempotent
-  public void disallowSnapshot(String snapshotRoot)
+  void disallowSnapshot(String snapshotRoot)
       throws IOException;
-  
+
   /**
    * Get the difference between two snapshots, or between a snapshot and the
    * current tree of a directory.
-   * 
+   *
    * @param snapshotRoot
    *          full path of the directory where snapshots are taken
    * @param fromSnapshot
@@ -1204,93 +1164,93 @@ public interface ClientProtocol {
    * @throws IOException on error
    */
   @Idempotent
-  public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
+  SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
       String fromSnapshot, String toSnapshot) throws IOException;
 
   /**
    * Add a CacheDirective to the CacheManager.
-   * 
+   *
    * @param directive A CacheDirectiveInfo to be added
    * @param flags {@link CacheFlag}s to use for this operation.
    * @return A CacheDirectiveInfo associated with the added directive
    * @throws IOException if the directive could not be added
    */
   @AtMostOnce
-  public long addCacheDirective(CacheDirectiveInfo directive,
+  long addCacheDirective(CacheDirectiveInfo directive,
       EnumSet<CacheFlag> flags) throws IOException;
 
   /**
    * Modify a CacheDirective in the CacheManager.
-   * 
+   *
    * @param flags {@link CacheFlag}s to use for this operation.
    * @throws IOException if the directive could not be modified
    */
   @AtMostOnce
-  public void modifyCacheDirective(CacheDirectiveInfo directive,
+  void modifyCacheDirective(CacheDirectiveInfo directive,
       EnumSet<CacheFlag> flags) throws IOException;
 
   /**
    * Remove a CacheDirectiveInfo from the CacheManager.
-   * 
+   *
    * @param id of a CacheDirectiveInfo
    * @throws IOException if the cache directive could not be removed
    */
   @AtMostOnce
-  public void removeCacheDirective(long id) throws IOException;
+  void removeCacheDirective(long id) throws IOException;
 
   /**
    * List the set of cached paths of a cache pool. Incrementally fetches results
    * from the server.
-   * 
+   *
    * @param prevId The last listed entry ID, or -1 if this is the first call to
    *               listCacheDirectives.
-   * @param filter Parameters to use to filter the list results, 
+   * @param filter Parameters to use to filter the list results,
    *               or null to display all directives visible to us.
    * @return A batch of CacheDirectiveEntry objects.
    */
   @Idempotent
-  public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(
+  BatchedEntries<CacheDirectiveEntry> listCacheDirectives(
       long prevId, CacheDirectiveInfo filter) throws IOException;
 
   /**
    * Add a new cache pool.
-   * 
+   *
    * @param info Description of the new cache pool
    * @throws IOException If the request could not be completed.
    */
   @AtMostOnce
-  public void addCachePool(CachePoolInfo info) throws IOException;
+  void addCachePool(CachePoolInfo info) throws IOException;
 
   /**
    * Modify an existing cache pool.
    *
    * @param req
    *          The request to modify a cache pool.
-   * @throws IOException 
+   * @throws IOException
    *          If the request could not be completed.
    */
   @AtMostOnce
-  public void modifyCachePool(CachePoolInfo req) throws IOException;
-  
+  void modifyCachePool(CachePoolInfo req) throws IOException;
+
   /**
    * Remove a cache pool.
-   * 
+   *
    * @param pool name of the cache pool to remove.
    * @throws IOException if the cache pool did not exist, or could not be
    *           removed.
    */
   @AtMostOnce
-  public void removeCachePool(String pool) throws IOException;
+  void removeCachePool(String pool) throws IOException;
 
   /**
    * List the set of cache pools. Incrementally fetches results from the server.
-   * 
-   * @param prevPool name of the last pool listed, or the empty string if this is
-   *          the first invocation of listCachePools
+   *
+   * @param prevPool name of the last pool listed, or the empty string if this
+   *          is the first invocation of listCachePools
    * @return A batch of CachePoolEntry objects.
    */
   @Idempotent
-  public BatchedEntries<CachePoolEntry> listCachePools(String prevPool)
+  BatchedEntries<CachePoolEntry> listCachePools(String prevPool)
       throws IOException;
 
   /**
@@ -1300,7 +1260,7 @@ public interface ClientProtocol {
    * changes.  (Modifications are merged into the current ACL.)
    */
   @Idempotent
-  public void modifyAclEntries(String src, List<AclEntry> aclSpec)
+  void modifyAclEntries(String src, List<AclEntry> aclSpec)
       throws IOException;
 
   /**
@@ -1308,14 +1268,14 @@ public interface ClientProtocol {
    * retained.
    */
   @Idempotent
-  public void removeAclEntries(String src, List<AclEntry> aclSpec)
+  void removeAclEntries(String src, List<AclEntry> aclSpec)
       throws IOException;
 
   /**
    * Removes all default ACL entries from files and directories.
    */
   @Idempotent
-  public void removeDefaultAcl(String src) throws IOException;
+  void removeDefaultAcl(String src) throws IOException;
 
   /**
    * Removes all but the base ACL entries of files and directories.  The entries
@@ -1323,33 +1283,33 @@ public interface ClientProtocol {
    * bits.
    */
   @Idempotent
-  public void removeAcl(String src) throws IOException;
+  void removeAcl(String src) throws IOException;
 
   /**
    * Fully replaces ACL of files and directories, discarding all existing
    * entries.
    */
   @Idempotent
-  public void setAcl(String src, List<AclEntry> aclSpec) throws IOException;
+  void setAcl(String src, List<AclEntry> aclSpec) throws IOException;
 
   /**
    * Gets the ACLs of files and directories.
    */
   @Idempotent
-  public AclStatus getAclStatus(String src) throws IOException;
-  
+  AclStatus getAclStatus(String src) throws IOException;
+
   /**
-   * Create an encryption zone
+   * Create an encryption zone.
    */
   @AtMostOnce
-  public void createEncryptionZone(String src, String keyName)
+  void createEncryptionZone(String src, String keyName)
     throws IOException;
 
   /**
    * Get the encryption zone for a path.
    */
   @Idempotent
-  public EncryptionZone getEZForPath(String src)
+  EncryptionZone getEZForPath(String src)
     throws IOException;
 
   /**
@@ -1360,7 +1320,7 @@ public interface ClientProtocol {
    * @return Batch of encryption zones.
    */
   @Idempotent
-  public BatchedEntries<EncryptionZone> listEncryptionZones(
+  BatchedEntries<EncryptionZone> listEncryptionZones(
       long prevId) throws IOException;
 
   /**
@@ -1376,9 +1336,9 @@ public interface ClientProtocol {
    * @throws IOException
    */
   @AtMostOnce
-  public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag) 
+  void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
       throws IOException;
-  
+
   /**
    * Get xattrs of a file or directory. Values in xAttrs parameter are ignored.
    * If xAttrs is null or empty, this is the same as getting all xattrs of the
@@ -1389,11 +1349,11 @@ public interface ClientProtocol {
    *
    * @param src file or directory
    * @param xAttrs xAttrs to get
-   * @return List<XAttr> <code>XAttr</code> list 
+   * @return List<XAttr> <code>XAttr</code> list
    * @throws IOException
    */
   @Idempotent
-  public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs) 
+  List<XAttr> getXAttrs(String src, List<XAttr> xAttrs)
       throws IOException;
 
   /**
@@ -1408,9 +1368,9 @@ public interface ClientProtocol {
    * @throws IOException
    */
   @Idempotent
-  public List<XAttr> listXAttrs(String src)
+  List<XAttr> listXAttrs(String src)
       throws IOException;
-  
+
   /**
    * Remove xattr of a file or directory.Value in xAttr parameter is ignored.
    * The name must be prefixed with the namespace followed by ".". For example,
@@ -1423,7 +1383,7 @@ public interface ClientProtocol {
    * @throws IOException
    */
   @AtMostOnce
-  public void removeXAttr(String src, XAttr xAttr) throws IOException;
+  void removeXAttr(String src, XAttr xAttr) throws IOException;
 
   /**
    * Checks if the user can access a path.  The mode specifies which access
@@ -1441,7 +1401,7 @@ public interface ClientProtocol {
    * @throws IOException see specific implementation
    */
   @Idempotent
-  public void checkAccess(String path, FsAction mode) throws IOException;
+  void checkAccess(String path, FsAction mode) throws IOException;
 
   /**
    * Get the highest txid the NameNode knows has been written to the edit
@@ -1449,12 +1409,12 @@ public interface ClientProtocol {
    * the starting point for the inotify event stream.
    */
   @Idempotent
-  public long getCurrentEditLogTxid() throws IOException;
+  long getCurrentEditLogTxid() throws IOException;
 
   /**
    * Get an ordered list of batches of events corresponding to the edit log
    * transactions for txids equal to or greater than txid.
    */
   @Idempotent
-  public EventBatchList getEditsFromTxid(long txid) throws IOException;
+  EventBatchList getEditsFromTxid(long txid) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c0b8e4e5/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 1e1e6bb..db77d0b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -704,6 +704,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8711. setSpaceQuota command should print the available storage type
     when input storage type is wrong. (Brahma Reddy Battula via xyao)
 
+    HDFS-8620. Clean up the checkstyle warinings about ClientProtocol.
+    (Takanobu Asanuma via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than


[23/50] hadoop git commit: HADOOP-12180. Move ResourceCalculatorPlugin from YARN to Common. (Chris Douglas via kasha)

Posted by ar...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac604837/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestLinuxResourceCalculatorPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestLinuxResourceCalculatorPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestLinuxResourceCalculatorPlugin.java
deleted file mode 100644
index a59d503..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestLinuxResourceCalculatorPlugin.java
+++ /dev/null
@@ -1,324 +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.util;
-
-import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Random;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.fs.Path;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * A JUnit test to test {@link LinuxResourceCalculatorPlugin}
- * Create the fake /proc/ information and verify the parsing and calculation
- */
-public class TestLinuxResourceCalculatorPlugin {
-  /**
-   * LinuxResourceCalculatorPlugin with a fake timer
-   */
-  static class FakeLinuxResourceCalculatorPlugin extends
-      LinuxResourceCalculatorPlugin {
-    
-	  long currentTime = 0;
-	  public FakeLinuxResourceCalculatorPlugin(String procfsMemFile,
-			                                       String procfsCpuFile,
-			                                       String procfsStatFile,
-			                                       long jiffyLengthInMillis) {
-	    super(procfsMemFile, procfsCpuFile, procfsStatFile, jiffyLengthInMillis);
-	  }
-	  @Override
-	  long getCurrentTime() {
-	    return currentTime;
-	  }
-	  public void advanceTime(long adv) {
-	    currentTime += adv * this.getJiffyLengthInMillis();
-	  }
-  }
-  private static final FakeLinuxResourceCalculatorPlugin plugin;
-  private static String TEST_ROOT_DIR = new Path(System.getProperty(
-         "test.build.data", "/tmp")).toString().replace(' ', '+');
-  private static final String FAKE_MEMFILE;
-  private static final String FAKE_CPUFILE;
-  private static final String FAKE_STATFILE;
-  private static final long FAKE_JIFFY_LENGTH = 10L;
-  static {
-    int randomNum = (new Random()).nextInt(1000000000);
-    FAKE_MEMFILE = TEST_ROOT_DIR + File.separator + "MEMINFO_" + randomNum;
-    FAKE_CPUFILE = TEST_ROOT_DIR + File.separator + "CPUINFO_" + randomNum;
-    FAKE_STATFILE = TEST_ROOT_DIR + File.separator + "STATINFO_" + randomNum;
-    plugin = new FakeLinuxResourceCalculatorPlugin(FAKE_MEMFILE, FAKE_CPUFILE,
-                                                   FAKE_STATFILE,
-                                                   FAKE_JIFFY_LENGTH);
-  }
-  static final String MEMINFO_FORMAT = 
-	  "MemTotal:      %d kB\n" +
-	  "MemFree:         %d kB\n" +
-	  "Buffers:        138244 kB\n" +
-	  "Cached:         947780 kB\n" +
-	  "SwapCached:     142880 kB\n" +
-	  "Active:        3229888 kB\n" +
-	  "Inactive:       %d kB\n" +
-	  "SwapTotal:     %d kB\n" +
-	  "SwapFree:      %d kB\n" +
-	  "Dirty:          122012 kB\n" +
-	  "Writeback:           0 kB\n" +
-	  "AnonPages:     2710792 kB\n" +
-	  "Mapped:          24740 kB\n" +
-	  "Slab:           132528 kB\n" +
-	  "SReclaimable:   105096 kB\n" +
-	  "SUnreclaim:      27432 kB\n" +
-	  "PageTables:      11448 kB\n" +
-	  "NFS_Unstable:        0 kB\n" +
-	  "Bounce:              0 kB\n" +
-	  "CommitLimit:   4125904 kB\n" +
-	  "Committed_AS:  4143556 kB\n" +
-	  "VmallocTotal: 34359738367 kB\n" +
-	  "VmallocUsed:      1632 kB\n" +
-	  "VmallocChunk: 34359736375 kB\n" +
-	  "HugePages_Total:     0\n" +
-	  "HugePages_Free:      0\n" +
-	  "HugePages_Rsvd:      0\n" +
-	  "Hugepagesize:     2048 kB";
-  
-  static final String CPUINFO_FORMAT =
-    "processor : %s\n" +
-    "vendor_id : AuthenticAMD\n" +
-    "cpu family  : 15\n" +
-    "model   : 33\n" +
-    "model name  : Dual Core AMD Opteron(tm) Processor 280\n" +
-    "stepping  : 2\n" +
-    "cpu MHz   : %f\n" +
-    "cache size  : 1024 KB\n" +
-    "physical id : %s\n" +
-    "siblings  : 2\n" +
-    "core id   : %s\n" +
-    "cpu cores : 2\n" +
-    "fpu   : yes\n" +
-    "fpu_exception : yes\n" +
-    "cpuid level : 1\n" +
-    "wp    : yes\n" +
-    "flags   : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov " +
-    "pat pse36 clflush mmx fxsr sse sse2 ht syscall nx mmxext fxsr_opt lm " +
-    "3dnowext 3dnow pni lahf_lm cmp_legacy\n" +
-    "bogomips  : 4792.41\n" +
-    "TLB size  : 1024 4K pages\n" +
-    "clflush size  : 64\n" +
-    "cache_alignment : 64\n" +
-    "address sizes : 40 bits physical, 48 bits virtual\n" +
-    "power management: ts fid vid ttp";
-  
-  static final String STAT_FILE_FORMAT = 
-    "cpu  %d %d %d 1646495089 831319 48713 164346 0\n" +
-    "cpu0 15096055 30805 3823005 411456015 206027 13 14269 0\n" +
-    "cpu1 14760561 89890 6432036 408707910 456857 48074 130857 0\n" +
-    "cpu2 12761169 20842 3758639 413976772 98028 411 10288 0\n" +
-    "cpu3 12355207 47322 5789691 412354390 70406 213 8931 0\n" +
-    "intr 114648668 20010764 2 0 945665 2 0 0 0 0 0 0 0 4 0 0 0 0 0 0\n" +
-    "ctxt 242017731764\n" +
-    "btime 1257808753\n" +
-    "processes 26414943\n" +
-    "procs_running 1\n" +
-    "procs_blocked 0\n";
-  
-  /**
-   * Test parsing /proc/stat and /proc/cpuinfo
-   * @throws IOException
-   */
-  @Test
-  public void parsingProcStatAndCpuFile() throws IOException {
-    // Write fake /proc/cpuinfo file.
-    long numProcessors = 8;
-    long cpuFrequencyKHz = 2392781;
-    String fileContent = "";
-    for (int i = 0; i < numProcessors; i++) {
-      fileContent +=
-          String.format(CPUINFO_FORMAT, i, cpuFrequencyKHz / 1000D, 0, 0)
-              + "\n";
-    }
-    File tempFile = new File(FAKE_CPUFILE);
-    tempFile.deleteOnExit();
-    FileWriter fWriter = new FileWriter(FAKE_CPUFILE);
-    fWriter.write(fileContent);
-    fWriter.close();
-    assertEquals(plugin.getNumProcessors(), numProcessors);
-    assertEquals(plugin.getCpuFrequency(), cpuFrequencyKHz);
-    
-    // Write fake /proc/stat file.
-    long uTime = 54972994;
-    long nTime = 188860;
-    long sTime = 19803373;
-    tempFile = new File(FAKE_STATFILE);
-    tempFile.deleteOnExit();
-    updateStatFile(uTime, nTime, sTime);
-    assertEquals(plugin.getCumulativeCpuTime(),
-                 FAKE_JIFFY_LENGTH * (uTime + nTime + sTime));
-    assertEquals(plugin.getCpuUsage(), (float)(CpuTimeTracker.UNAVAILABLE),0.0);
-
-    // Advance the time and sample again to test the CPU usage calculation
-    uTime += 100L;
-    plugin.advanceTime(200L);
-    updateStatFile(uTime, nTime, sTime);
-    assertEquals(plugin.getCumulativeCpuTime(),
-                 FAKE_JIFFY_LENGTH * (uTime + nTime + sTime));
-    assertEquals(plugin.getCpuUsage(), 6.25F, 0.0);
-    
-    // Advance the time and sample again. This time, we call getCpuUsage() only.
-    uTime += 600L;
-    plugin.advanceTime(300L);
-    updateStatFile(uTime, nTime, sTime);
-    assertEquals(plugin.getCpuUsage(), 25F, 0.0);
-    
-    // Advance very short period of time (one jiffy length).
-    // In this case, CPU usage should not be updated.
-    uTime += 1L;
-    plugin.advanceTime(1L);
-    updateStatFile(uTime, nTime, sTime);
-    assertEquals(plugin.getCumulativeCpuTime(),
-                 FAKE_JIFFY_LENGTH * (uTime + nTime + sTime));
-    assertEquals(plugin.getCpuUsage(), 25F, 0.0); // CPU usage is not updated.
-  }
-  
-  /**
-   * Write information to fake /proc/stat file
-   */
-  private void updateStatFile(long uTime, long nTime, long sTime)
-    throws IOException {
-    FileWriter fWriter = new FileWriter(FAKE_STATFILE);
-    fWriter.write(String.format(STAT_FILE_FORMAT, uTime, nTime, sTime));
-    fWriter.close();
-  }
-  
-  /**
-   * Test parsing /proc/meminfo
-   * @throws IOException
-   */
-  @Test
-  public void parsingProcMemFile() throws IOException {
-    long memTotal = 4058864L;
-    long memFree = 99632L;
-    long inactive = 567732L;
-    long swapTotal = 2096472L;
-    long swapFree = 1818480L;
-    File tempFile = new File(FAKE_MEMFILE);
-    tempFile.deleteOnExit();
-    FileWriter fWriter = new FileWriter(FAKE_MEMFILE);
-    fWriter.write(String.format(MEMINFO_FORMAT,
-      memTotal, memFree, inactive, swapTotal, swapFree));
-    
-    fWriter.close();
-    assertEquals(plugin.getAvailablePhysicalMemorySize(),
-                 1024L * (memFree + inactive));
-    assertEquals(plugin.getAvailableVirtualMemorySize(),
-                 1024L * (memFree + inactive + swapFree));
-    assertEquals(plugin.getPhysicalMemorySize(), 1024L * memTotal);
-    assertEquals(plugin.getVirtualMemorySize(), 1024L * (memTotal + swapTotal));
-  }
-
-  @Test
-  public void testCoreCounts() throws IOException {
-
-    String fileContent = "";
-    // single core, hyper threading
-    long numProcessors = 2;
-    long cpuFrequencyKHz = 2392781;
-    for (int i = 0; i < numProcessors; i++) {
-      fileContent =
-          fileContent.concat(String.format(CPUINFO_FORMAT, i,
-            cpuFrequencyKHz / 1000D, 0, 0));
-      fileContent = fileContent.concat("\n");
-    }
-    writeFakeCPUInfoFile(fileContent);
-    plugin.setReadCpuInfoFile(false);
-    assertEquals(numProcessors, plugin.getNumProcessors());
-    assertEquals(1, plugin.getNumCores());
-
-    // single socket quad core, no hyper threading
-    fileContent = "";
-    numProcessors = 4;
-    for (int i = 0; i < numProcessors; i++) {
-      fileContent =
-          fileContent.concat(String.format(CPUINFO_FORMAT, i,
-            cpuFrequencyKHz / 1000D, 0, i));
-      fileContent = fileContent.concat("\n");
-    }
-    writeFakeCPUInfoFile(fileContent);
-    plugin.setReadCpuInfoFile(false);
-    assertEquals(numProcessors, plugin.getNumProcessors());
-    assertEquals(4, plugin.getNumCores());
-
-    // dual socket single core, hyper threading
-    fileContent = "";
-    numProcessors = 4;
-    for (int i = 0; i < numProcessors; i++) {
-      fileContent =
-          fileContent.concat(String.format(CPUINFO_FORMAT, i,
-            cpuFrequencyKHz / 1000D, i / 2, 0));
-      fileContent = fileContent.concat("\n");
-    }
-    writeFakeCPUInfoFile(fileContent);
-    plugin.setReadCpuInfoFile(false);
-    assertEquals(numProcessors, plugin.getNumProcessors());
-    assertEquals(2, plugin.getNumCores());
-
-    // dual socket, dual core, no hyper threading
-    fileContent = "";
-    numProcessors = 4;
-    for (int i = 0; i < numProcessors; i++) {
-      fileContent =
-          fileContent.concat(String.format(CPUINFO_FORMAT, i,
-            cpuFrequencyKHz / 1000D, i / 2, i % 2));
-      fileContent = fileContent.concat("\n");
-    }
-    writeFakeCPUInfoFile(fileContent);
-    plugin.setReadCpuInfoFile(false);
-    assertEquals(numProcessors, plugin.getNumProcessors());
-    assertEquals(4, plugin.getNumCores());
-
-    // dual socket, dual core, hyper threading
-    fileContent = "";
-    numProcessors = 8;
-    for (int i = 0; i < numProcessors; i++) {
-      fileContent =
-          fileContent.concat(String.format(CPUINFO_FORMAT, i,
-            cpuFrequencyKHz / 1000D, i / 4, (i % 4) / 2));
-      fileContent = fileContent.concat("\n");
-    }
-    writeFakeCPUInfoFile(fileContent);
-    plugin.setReadCpuInfoFile(false);
-    assertEquals(numProcessors, plugin.getNumProcessors());
-    assertEquals(4, plugin.getNumCores());
-  }
-
-  private void writeFakeCPUInfoFile(String content) throws IOException {
-    File tempFile = new File(FAKE_CPUFILE);
-    FileWriter fWriter = new FileWriter(FAKE_CPUFILE);
-    tempFile.deleteOnExit();
-    try {
-      fWriter.write(content);
-    } finally {
-      IOUtils.closeQuietly(fWriter);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac604837/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestResourceCalculatorProcessTree.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestResourceCalculatorProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestResourceCalculatorProcessTree.java
index 777ea9f..7a3e0e7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestResourceCalculatorProcessTree.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestResourceCalculatorProcessTree.java
@@ -65,7 +65,7 @@ public class TestResourceCalculatorProcessTree {
 
     @Override
     public float getCpuUsagePercent() {
-      return CpuTimeTracker.UNAVAILABLE;
+      return UNAVAILABLE;
     }
 
     public boolean checkPidPgrpidForMatch() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac604837/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestWindowsResourceCalculatorPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestWindowsResourceCalculatorPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestWindowsResourceCalculatorPlugin.java
deleted file mode 100644
index a9e20bc..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestWindowsResourceCalculatorPlugin.java
+++ /dev/null
@@ -1,86 +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.util;
-
-import org.junit.Test;
-import static org.junit.Assert.assertTrue;
-
-public class TestWindowsResourceCalculatorPlugin {
-  
-  
-  class WindowsResourceCalculatorPluginTester extends WindowsResourceCalculatorPlugin {
-    private String infoStr = null;
-    @Override
-    String getSystemInfoInfoFromShell() {
-      return infoStr;
-    }    
-  }
-
-  @Test (timeout = 30000)
-  public void parseSystemInfoString() {
-    WindowsResourceCalculatorPluginTester tester = new WindowsResourceCalculatorPluginTester();
-    // info str derived from windows shell command has \r\n termination
-    tester.infoStr = "17177038848,8589467648,15232745472,6400417792,1,2805000,6261812\r\n";
-    // call a method to refresh values
-    tester.getAvailablePhysicalMemorySize();
-    // verify information has been refreshed
-    assertTrue(tester.vmemSize == 17177038848L);
-    assertTrue(tester.memSize == 8589467648L);
-    assertTrue(tester.vmemAvailable == 15232745472L);
-    assertTrue(tester.memAvailable == 6400417792L);
-    assertTrue(tester.numProcessors == 1);
-    assertTrue(tester.cpuFrequencyKhz == 2805000L);
-    assertTrue(tester.cumulativeCpuTimeMs == 6261812L);
-    assertTrue(tester.cpuUsage == -1);
-  }
-
-  @Test (timeout = 20000)
-  public void refreshAndCpuUsage() throws InterruptedException {
-    WindowsResourceCalculatorPluginTester tester = new WindowsResourceCalculatorPluginTester();
-    // info str derived from windows shell command has \r\n termination
-    tester.infoStr = "17177038848,8589467648,15232745472,6400417792,1,2805000,6261812\r\n";
-    tester.getAvailablePhysicalMemorySize();
-    // verify information has been refreshed
-    assertTrue(tester.memAvailable == 6400417792L);
-    assertTrue(tester.cpuUsage == -1);
-    
-    tester.infoStr = "17177038848,8589467648,15232745472,5400417792,1,2805000,6261812\r\n";
-    tester.getAvailablePhysicalMemorySize();
-    // verify information has not been refreshed
-    assertTrue(tester.memAvailable == 6400417792L);
-    assertTrue(tester.cpuUsage == -1);
-    
-    Thread.sleep(1500);
-    tester.infoStr = "17177038848,8589467648,15232745472,5400417792,1,2805000,6286812\r\n";
-    tester.getAvailablePhysicalMemorySize();
-    // verify information has been refreshed
-    assertTrue(tester.memAvailable == 5400417792L);
-    assertTrue(tester.cpuUsage >= 0.1);
-  }
-
-  @Test (timeout = 20000)
-  public void errorInGetSystemInfo() {
-    WindowsResourceCalculatorPluginTester tester = new WindowsResourceCalculatorPluginTester();
-    // info str derived from windows shell command has \r\n termination
-    tester.infoStr = null;
-    // call a method to refresh values
-    tester.getAvailablePhysicalMemorySize();    
-  }
-
-}


[35/50] hadoop git commit: YARN-3917. getResourceCalculatorPlugin for the default should intercept all exceptions. (gera)

Posted by ar...@apache.org.
YARN-3917. getResourceCalculatorPlugin for the default should intercept all exceptions. (gera)


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

Branch: refs/heads/HDFS-7240
Commit: d7319dee37ea93f1a1ba4153ea63ea8010ba2441
Parents: 1df39c1
Author: Gera Shegalov <ge...@apache.org>
Authored: Fri Jul 10 17:43:53 2015 -0700
Committer: Gera Shegalov <ge...@apache.org>
Committed: Sat Jul 11 22:29:36 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                             | 3 +++
 .../apache/hadoop/yarn/util/ResourceCalculatorPlugin.java   | 9 +++++++--
 2 files changed, 10 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d7319dee/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 1365747..9881771 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -616,6 +616,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3849. Too much of preemption activity causing continuos killing of
     containers across queues. (Sunil G via wangda)
 
+    YARN-3917. getResourceCalculatorPlugin for the default should intercept all
+    exceptions. (gera)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d7319dee/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
index 21724a9..3af4aee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.yarn.util;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -30,6 +32,8 @@ import org.apache.hadoop.util.SysInfo;
 @InterfaceAudience.LimitedPrivate({"YARN", "MAPREDUCE"})
 @InterfaceStability.Unstable
 public class ResourceCalculatorPlugin extends Configured {
+  private static final Log LOG =
+      LogFactory.getLog(ResourceCalculatorPlugin.class);
 
   private final SysInfo sys;
 
@@ -158,9 +162,10 @@ public class ResourceCalculatorPlugin extends Configured {
     }
     try {
       return new ResourceCalculatorPlugin();
-    } catch (SecurityException e) {
-      return null;
+    } catch (Throwable t) {
+      LOG.warn(t + ": Failed to instantiate default resource calculator.", t);
     }
+    return null;
   }
 
 }


[03/50] hadoop git commit: MAPREDUCE-6038. A boolean may be set error in the Word Count v2.0 in MapReduce Tutorial. Contributed by Tsuyoshi Ozawa

Posted by ar...@apache.org.
MAPREDUCE-6038. A boolean may be set error in the Word Count v2.0 in MapReduce Tutorial. Contributed by Tsuyoshi Ozawa


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

Branch: refs/heads/HDFS-7240
Commit: 7e2fe8c9f28ec6fff32741ebf1bdbf47729d9eaf
Parents: bc99aaf
Author: Chris Douglas <cd...@apache.org>
Authored: Tue Jul 7 10:26:27 2015 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Tue Jul 7 10:27:04 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                              | 3 +++
 .../src/site/markdown/MapReduceTutorial.md                        | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e2fe8c9/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 4c60174..874ecea 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -530,6 +530,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6418. MRApp should not shutdown LogManager during shutdown
     (Chang Li via jlowe)
 
+    MAPREDUCE-6038. A boolean may be set error in the Word Count v2.0 in
+    MapReduce Tutorial. (Tsuyoshi Ozawa via cdouglas)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e2fe8c9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/MapReduceTutorial.md
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/MapReduceTutorial.md b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/MapReduceTutorial.md
index cd087d5..e2aaaf6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/MapReduceTutorial.md
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/MapReduceTutorial.md
@@ -978,7 +978,7 @@ public class WordCount2 {
         InterruptedException {
       conf = context.getConfiguration();
       caseSensitive = conf.getBoolean("wordcount.case.sensitive", true);
-      if (conf.getBoolean("wordcount.skip.patterns", true)) {
+      if (conf.getBoolean("wordcount.skip.patterns", false)) {
         URI[] patternsURIs = Job.getInstance(conf).getCacheFiles();
         for (URI patternsURI : patternsURIs) {
           Path patternsPath = new Path(patternsURI.getPath());


[36/50] hadoop git commit: YARN-3069. Document missing properties in yarn-default.xml. Contributed by Ray Chiang.

Posted by ar...@apache.org.
YARN-3069. Document missing properties in yarn-default.xml. Contributed by Ray Chiang.


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

Branch: refs/heads/HDFS-7240
Commit: d6675606dc5f141c9af4f76a37128f8de4cfedad
Parents: d7319de
Author: Akira Ajisaka <aa...@apache.org>
Authored: Mon Jul 13 12:57:01 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Mon Jul 13 12:57:01 2015 +0900

----------------------------------------------------------------------
 .../src/site/markdown/DeprecatedProperties.md   |   1 +
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../src/main/resources/yarn-default.xml         | 466 ++++++++++++++++++-
 .../yarn/conf/TestYarnConfigurationFields.java  |  44 +-
 4 files changed, 488 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6675606/hadoop-common-project/hadoop-common/src/site/markdown/DeprecatedProperties.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/DeprecatedProperties.md b/hadoop-common-project/hadoop-common/src/site/markdown/DeprecatedProperties.md
index f964735..f8bc95b 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/DeprecatedProperties.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/DeprecatedProperties.md
@@ -276,6 +276,7 @@ The following table lists the configuration property names that are deprecated i
 | user.name | mapreduce.job.user.name |
 | webinterface.private.actions | mapreduce.jobtracker.webinterface.trusted |
 | yarn.app.mapreduce.yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts | yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts |
+| yarn.client.app-submission.poll-interval | yarn.client.application-client-protocol.poll-timeout-ms |
 
 The following table lists additional changes to some configuration properties:
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6675606/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 9881771..0415ae4 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -329,6 +329,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3800. Reduce storage footprint for ReservationAllocation. (Anubhav Dhoot
     via curino)
 
+    YARN-3069. Document missing properties in yarn-default.xml.
+    (Ray Chiang via aajisaka)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6675606/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 4987084..2edeef0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -24,7 +24,8 @@
 
 <configuration>
 
-  <!-- IPC Configs -->
+  <!-- IPC Configuration -->
+
   <property>
     <description>Factory to create client IPC classes.</description>
     <name>yarn.ipc.client.factory.class</name>
@@ -46,7 +47,8 @@
     <value>org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC</value>
   </property>
   
-  <!-- Resource Manager Configs -->
+  <!-- Resource Manager Configuration -->
+
   <property>
     <description>The hostname of the RM.</description>
     <name>yarn.resourcemanager.hostname</name>
@@ -143,6 +145,32 @@
   </property>
 
   <property>
+    <description>
+    The Kerberos keytab file to be used for spnego filter for the RM web
+    interface.
+    </description>
+    <name>yarn.resourcemanager.webapp.spnego-keytab-file</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+    The Kerberos principal to be used for spnego filter for the RM web
+    interface.
+    </description>
+    <name>yarn.resourcemanager.webapp.spnego-principal</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+    Add button to kill application in the RM Application view.
+    </description>
+    <name>yarn.resourcemanager.webapp.ui-actions.enabled</name>
+    <value>true</value>
+  </property>
+
+  <property>
     <name>yarn.resourcemanager.resource-tracker.address</name>
     <value>${yarn.resourcemanager.hostname}:8031</value>
   </property>
@@ -280,7 +308,16 @@
   </property>
 
   <property>
-    <description>Enable RM to recover state after starting. If true, then 
+    <description>
+    Used by node labels.  If set to true, the port should be included in the
+    node name.  Only usable if your scheduler supports node labels.
+    </description>
+    <name>yarn.scheduler.include-port-in-node-name</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <description>Enable RM to recover state after starting. If true, then
       yarn.resourcemanager.store.class must be specified. </description>
     <name>yarn.resourcemanager.recovery.enabled</name>
     <value>false</value>
@@ -673,6 +710,16 @@
   </property>
 
   <property>
+    <description>
+    The value specifies the file system (e.g. HDFS) path where ResourceManager
+    loads configuration if yarn.resourcemanager.configuration.provider-class
+    is set to org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider.
+    </description>
+    <name>yarn.resourcemanager.configuration.file-system-based-store</name>
+    <value>/yarn/conf</value>
+  </property>
+
+  <property>
     <description>The setting that controls whether yarn system metrics is
     published on the timeline server or not by RM.</description>
     <name>yarn.resourcemanager.system-metrics-publisher.enabled</name>
@@ -695,6 +742,119 @@
   </property>
 
   <!-- Node Manager Configs -->
+
+  <property>
+    <description>
+    RM DelegationTokenRenewer thread count
+    </description>
+    <name>yarn.resourcemanager.delegation-token-renewer.thread-count</name>
+    <value>50</value>
+  </property>
+
+  <property>
+    <description>
+    RM secret key update interval in ms
+    </description>
+    <name>yarn.resourcemanager.delegation.key.update-interval</name>
+    <value>86400000</value>
+  </property>
+
+  <property>
+    <description>
+    RM delegation token maximum lifetime in ms
+    </description>
+    <name>yarn.resourcemanager.delegation.token.max-lifetime</name>
+    <value>604800000</value>
+  </property>
+
+  <property>
+    <description>
+    RM delegation token update interval in ms
+    </description>
+    <name>yarn.resourcemanager.delegation.token.renew-interval</name>
+    <value>86400000</value>
+  </property>
+
+  <property>
+    <description>
+    Thread pool size for RMApplicationHistoryWriter.
+    </description>
+    <name>yarn.resourcemanager.history-writer.multi-threaded-dispatcher.pool-size</name>
+    <value>10</value>
+  </property>
+
+  <property>
+    <description>
+    Comma-separated list of values (in minutes) for schedule queue related
+    metrics.
+    </description>
+    <name>yarn.resourcemanager.metrics.runtime.buckets</name>
+    <value>60,300,1440</value>
+  </property>
+
+  <property>
+    <description>
+    Interval for the roll over for the master key used to generate
+    NodeManager tokens.  It is expected to be set to a value much larger
+    than yarn.nm.liveness-monitor.expiry-interval-ms.
+    </description>
+    <name>yarn.resourcemanager.nm-tokens.master-key-rolling-interval-secs</name>
+    <value>86400</value>
+  </property>
+
+  <property>
+    <description>
+    Flag to enable the ResourceManager reservation system.
+    </description>
+    <name>yarn.resourcemanager.reservation-system.enable</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <description>
+    The Java class to use as the ResourceManager reservation system.
+    By default, is set to
+    org.apache.hadoop.yarn.server.resourcemanager.reservation.CapacityReservationSystem
+    when using CapacityScheduler and is set to
+    org.apache.hadoop.yarn.server.resourcemanager.reservation.FairReservationSystem
+    when using FairScheduler.
+    </description>
+    <name>yarn.resourcemanager.reservation-system.class</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+    The plan follower policy class name to use for the ResourceManager
+    reservation system.
+    By default, is set to
+    org.apache.hadoop.yarn.server.resourcemanager.reservation.CapacitySchedulerPlanFollower
+    is used when using CapacityScheduler, and is set to
+    org.apache.hadoop.yarn.server.resourcemanager.reservation.FairSchedulerPlanFollower
+    when using FairScheduler.
+    </description>
+    <name>yarn.resourcemanager.reservation-system.plan.follower</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+    Step size of the reservation system in ms
+    </description>
+    <name>yarn.resourcemanager.reservation-system.planfollower.time-step</name>
+    <value>1000</value>
+  </property>
+
+  <property>
+    <description>
+    The expiry interval for a container
+    </description>
+    <name>yarn.resourcemanager.rm.container-allocation.expiry-interval-ms</name>
+    <value>600000</value>
+  </property>
+
+  <!-- Node Manager Configuration -->
+
   <property>
     <description>The hostname of the NM.</description>
     <name>yarn.nodemanager.hostname</name>
@@ -727,7 +887,7 @@
   <property>
     <description>Environment variables that containers may override rather than use NodeManager's default.</description>
     <name>yarn.nodemanager.env-whitelist</name>
-    <value>JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,HADOOP_YARN_HOME</value>
+    <value>JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,CLASSPATH_PREPEND_DISTCACHE,HADOOP_YARN_HOME</value>
   </property>
 
   <property>
@@ -840,7 +1000,14 @@
 
   <property>
     <description>
-      Where to store container logs. An application's localized log directory 
+    </description>
+    <name>yarn.nodemanager.container-localizer.java.opts</name>
+    <value>-Xmx256m</value>
+  </property>
+
+  <property>
+    <description>
+      Where to store container logs. An application's localized log directory
       will be found in ${yarn.nodemanager.log-dirs}/application_${appid}.
       Individual containers' log directories will be below this, in directories 
       named container_{$contid}. Each container directory will contain the files
@@ -880,6 +1047,17 @@
   </property>
 
   <property>
+    <description>
+    How long for ResourceManager to wait for NodeManager to report its
+    log aggregation status. If waiting time of which the log aggregation
+    status is reported from NodeManager exceeds the configured value, RM
+    will report log aggregation status for this NodeManager as TIME_OUT
+    </description>
+    <name>yarn.log-aggregation-status.time-out.ms</name>
+    <value>600000</value>
+  </property>
+
+  <property>
     <description>Time in seconds to retain user logs. Only applicable if
     log aggregation is disabled
     </description>
@@ -1006,6 +1184,32 @@
   </property>
 
   <property>
+    <description>
+    The https adddress of the NM web application.
+    </description>
+    <name>yarn.nodemanager.webapp.https.address</name>
+    <value>0.0.0.0:8044</value>
+  </property>
+
+  <property>
+    <description>
+    The Kerberos keytab file to be used for spnego filter for the NM web
+    interface.
+    </description>
+    <name>yarn.nodemanager.webapp.spnego-keytab-file</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+    The Kerberos principal to be used for spnego filter for the NM web
+    interface.
+    </description>
+    <name>yarn.nodemanager.webapp.spnego-principal</name>
+    <value></value>
+  </property>
+
+  <property>
     <description>How often to monitor containers.</description>
     <name>yarn.nodemanager.container-monitor.interval-ms</name>
     <value>3000</value>
@@ -1113,6 +1317,12 @@
   </property>
 
   <property>
+    <description>Delay in ms between attempts to remove linux cgroup</description>
+    <name>yarn.nodemanager.linux-container-executor.cgroups.delete-delay-ms</name>
+    <value>20</value>
+  </property>
+
+  <property>
     <description>This determines which of the two modes that LCE should use on
       a non-secure cluster.  If this value is set to true, then all containers
       will be launched as the user specified in
@@ -1166,6 +1376,23 @@
   </property>
 
   <property>
+    <description>
+    Interval of time the linux container executor should try cleaning up
+    cgroups entry when cleaning up a container.
+    </description>
+    <name>yarn.nodemanager.linux-container-executor.cgroups.delete-timeout-ms</name>
+    <value>1000</value>
+  </property>
+
+  <property>
+    <description>
+    The UNIX group that the linux-container-executor should run as.
+    </description>
+    <name>yarn.nodemanager.linux-container-executor.group</name>
+    <value></value>
+  </property>
+
+  <property>
     <description>T-file compression types used to compress aggregated logs.</description>
     <name>yarn.nodemanager.log-aggregation.compression-type</name>
     <value>none</value>
@@ -1273,6 +1500,66 @@
   <!--Docker configuration-->
 
   <property>
+    <description>
+    Adjustment to the container OS scheduling priority.  In Linux, passed
+    directly to the nice command.
+    </description>
+    <name>yarn.nodemanager.container-executor.os.sched.priority.adjustment</name>
+    <value>0</value>
+  </property>
+
+  <property>
+    <description>
+    Flag to enable container metrics
+    </description>
+    <name>yarn.nodemanager.container-metrics.enable</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <description>
+    Container metrics flush period in ms.  Set to -1 for flush on completion.
+    </description>
+    <name>yarn.nodemanager.container-metrics.period-ms</name>
+    <value>-1</value>
+  </property>
+
+  <property>
+    <description>
+    Class used to calculate current container resource utilization.
+    </description>
+    <name>yarn.nodemanager.container-monitor.process-tree.class</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+    Flag to enable NodeManager disk health checker
+    </description>
+    <name>yarn.nodemanager.disk-health-checker.enable</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <description>
+    Number of threads to use in NM log cleanup.  Used when log aggregation
+    is disabled.
+    </description>
+    <name>yarn.nodemanager.log.deletion-threads-count</name>
+    <value>4</value>
+  </property>
+
+  <property>
+    <description>
+    The Windows group that the windows-container-executor should run as.
+    </description>
+    <name>yarn.nodemanager.windows-secure-container-executor.group</name>
+    <value></value>
+  </property>
+
+  <!-- Docker Configuration -->
+
+  <property>
     <name>yarn.nodemanager.docker-container-executor.exec-name</name>
     <value>/usr/bin/docker</value>
     <description>
@@ -1280,24 +1567,28 @@
     </description>
   </property>
 
-  <!--Map Reduce configuration-->
   <property>
-    <name>yarn.nodemanager.aux-services.mapreduce_shuffle.class</name>
-    <value>org.apache.hadoop.mapred.ShuffleHandler</value>
+    <description>
+    The Docker image name to use for DockerContainerExecutor
+    </description>
+    <name>yarn.nodemanager.docker-container-executor.image-name</name>
+    <value></value>
   </property>
 
-  <property>
-    <name>mapreduce.job.jar</name>
-    <value/>
-  </property>
+  <!-- Map Reduce Configuration -->
 
   <property>
     <name>mapreduce.job.hdfs-servers</name>
     <value>${fs.defaultFS}</value>
   </property>
 
-  <!-- WebAppProxy Configuration-->
-  
+  <property>
+    <name>yarn.nodemanager.aux-services.mapreduce_shuffle.class</name>
+    <value>org.apache.hadoop.mapred.ShuffleHandler</value>
+  </property>
+
+  <!-- WebAppProxy Configuration -->
+
   <property>
     <description>The kerberos principal for the proxy, if the proxy is not
     running as part of the RM.</description>
@@ -1318,8 +1609,8 @@
      <value/>
   </property>
 
-  <!-- Applications' Configuration-->
-  
+  <!-- Applications' Configuration -->
+
   <property>
     <description>
       CLASSPATH for YARN applications. A comma-separated list
@@ -1346,7 +1637,7 @@
     <value></value>
   </property>
 
-  <!-- Timeline Service's Configuration-->
+  <!-- Timeline Service Configuration -->
 
   <property>
     <description>Indicate to clients whether timeline service is enabled or not.
@@ -1530,6 +1821,7 @@
   </property>
 
   <!--  Shared Cache Configuration -->
+
   <property>
     <description>Whether the shared cache is enabled</description>
     <name>yarn.sharedcache.enabled</name>
@@ -1671,7 +1963,99 @@
     <value>20</value>
   </property>
 
-  <!-- Other configuration -->
+  <property>
+    <description>
+    ACL protocol for use in the Timeline server.
+    </description>
+    <name>security.applicationhistory.protocol.acl</name>
+    <value></value>
+  </property>
+
+  <!-- Minicluster Configuration (for testing only!) -->
+
+  <property>
+    <description>
+    Set to true for MiniYARNCluster unit tests
+    </description>
+    <name>yarn.is.minicluster</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <description>
+    Set for MiniYARNCluster unit tests to control resource monitoring
+    </description>
+    <name>yarn.minicluster.control-resource-monitoring</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <description>
+    Set to false in order to allow MiniYARNCluster to run tests without
+    port conflicts.
+    </description>
+    <name>yarn.minicluster.fixed.ports</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <description>
+    Set to false in order to allow the NodeManager in MiniYARNCluster to
+    use RPC to talk to the RM.
+    </description>
+    <name>yarn.minicluster.use-rpc</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <description>
+    As yarn.nodemanager.resource.memory-mb property but for the NodeManager
+    in a MiniYARNCluster.
+    </description>
+    <name>yarn.minicluster.yarn.nodemanager.resource.memory-mb</name>
+    <value>4096</value>
+  </property>
+
+  <!-- Node Labels Configuration -->
+
+  <property>
+    <description>
+    Enable node labels feature
+    </description>
+    <name>yarn.node-labels.enabled</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <description>
+    Retry policy used for FileSystem node label store. The policy is
+    specified by N pairs of sleep-time in milliseconds and number-of-retries
+    "s1,n1,s2,n2,...".
+    </description>
+    <name>yarn.node-labels.fs-store.retry-policy-spec</name>
+    <value>2000, 500</value>
+  </property>
+
+  <property>
+    <description>
+    URI for NodeLabelManager.  The default value is
+    /tmp/hadoop-yarn-${user}/node-labels/ in the local filesystem.
+    </description>
+    <name>yarn.node-labels.fs-store.root-dir</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+    Set configuration type for node labels. Administrators can specify
+    "centralized" or "distributed".
+    </description>
+    <name>yarn.node-labels.configuration-type</name>
+    <value>centralized</value>
+  </property>
+
+  <!-- Other Configuration -->
+
   <property>
     <description>The interval that the yarn client library uses to poll the
     completion status of the asynchronous API of application client protocol.
@@ -1681,7 +2065,16 @@
   </property>
 
   <property>
-    <description>RSS usage of a process computed via 
+    <description>
+    The duration (in ms) the YARN client waits for an expected state change
+    to occur.  -1 means unlimited wait time.
+    </description>
+    <name>yarn.client.application-client-protocol.poll-timeout-ms</name>
+    <value>-1</value>
+  </property>
+
+  <property>
+    <description>RSS usage of a process computed via
     /proc/pid/stat is not very accurate as it includes shared pages of a
     process. /proc/pid/smaps provides useful information like
     Private_Dirty, Private_Clean, Shared_Dirty, Shared_Clean which can be used
@@ -1694,6 +2087,30 @@
   </property>
 
   <property>
+    <description>
+    URL for log aggregation server
+    </description>
+    <name>yarn.log.server.url</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+    RM Application Tracking URL
+    </description>
+    <name>yarn.tracking.url.generator</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+    Class to be used for YarnAuthorizationProvider
+    </description>
+    <name>yarn.authorization-provider</name>
+    <value></value>
+  </property>
+
+  <property>
     <description>Defines how often NMs wake up to upload log files.
     The default value is -1. By default, the logs will be uploaded when
     the application is finished. By setting this configure, logs can be uploaded
@@ -1703,4 +2120,15 @@
     <name>yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds</name>
     <value>-1</value>
   </property>
+
+  <property>
+    <description>
+    Enable/disable intermediate-data encryption at YARN level. For now,
+    this only is used by the FileSystemRMStateStore to setup right
+    file-system security attributes.
+    </description>
+    <name>yarn.intermediate-data-encryption.enable</name>
+    <value>false</value>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6675606/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
index 9075d9f..10e1ca6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
@@ -33,18 +33,19 @@ import org.apache.hadoop.conf.TestConfigurationFieldsBase;
  */
 public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
 
+  @SuppressWarnings("deprecation")
   @Override
   public void initializeMemberVariables() {
     xmlFilename = new String("yarn-default.xml");
     configurationClasses = new Class[] { YarnConfiguration.class };
-        
 
     // Allocate for usage
     configurationPropsToSkipCompare = new HashSet<String>();
+    configurationPrefixToSkipCompare = new HashSet<String>();
 
     // Set error modes
     errorIfMissingConfigProps = true;
-    errorIfMissingXmlProps = false;
+    errorIfMissingXmlProps = true;
 
     // Specific properties to skip
     configurationPropsToSkipCompare
@@ -79,6 +80,37 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     configurationPropsToSkipCompare
         .add(YarnConfiguration
 	    .YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCETRACKER_PROTOCOL);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.DEFAULT_SCM_STORE_CLASS);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.DEFAULT_SCM_APP_CHECKER_CLASS);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.DEFAULT_SHARED_CACHE_CHECKSUM_ALGO_IMPL);
+
+    // Ignore all YARN Application Timeline Service (version 1) properties
+    configurationPrefixToSkipCompare.add("yarn.timeline-service.");
+
+    // Used as Java command line properties, not XML
+    configurationPrefixToSkipCompare.add("yarn.app.container");
+
+    // Ignore NodeManager "work in progress" variables
+    configurationPrefixToSkipCompare
+        .add(YarnConfiguration.NM_NETWORK_RESOURCE_ENABLED);
+    configurationPrefixToSkipCompare
+        .add(YarnConfiguration.NM_NETWORK_RESOURCE_INTERFACE);
+    configurationPrefixToSkipCompare
+        .add(YarnConfiguration.NM_NETWORK_RESOURCE_OUTBOUND_BANDWIDTH_MBIT);
+    configurationPrefixToSkipCompare
+        .add(YarnConfiguration.NM_NETWORK_RESOURCE_OUTBOUND_BANDWIDTH_YARN_MBIT);
+    configurationPrefixToSkipCompare
+        .add(YarnConfiguration.NM_DISK_RESOURCE_ENABLED);
+
+    // Set by container-executor.cfg
+    configurationPrefixToSkipCompare.add(YarnConfiguration.NM_USER_HOME_DIR);
+
+    // Ignore deprecated properties
+    configurationPrefixToSkipCompare
+        .add(YarnConfiguration.YARN_CLIENT_APP_SUBMISSION_POLL_INTERVAL_MS);
 
     // Allocate for usage
     xmlPropsToSkipCompare = new HashSet<String>();
@@ -94,13 +126,11 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
 
     // Used in the XML file as a variable reference internal to the XML file
     xmlPropsToSkipCompare.add("yarn.nodemanager.hostname");
-    xmlPropsToSkipCompare.add("yarn.timeline-service.hostname");
 
-    // Currently defined in TimelineAuthenticationFilterInitializer
-    xmlPrefixToSkipCompare.add("yarn.timeline-service.http-authentication");
+    // Ignore all YARN Application Timeline Service (version 1) properties
+    xmlPrefixToSkipCompare.add("yarn.timeline-service");
 
-    // Currently defined in RegistryConstants
+    // Currently defined in RegistryConstants/core-site.xml
     xmlPrefixToSkipCompare.add("hadoop.registry");
   }
-
 }


[50/50] hadoop git commit: Merge remote-tracking branch 'apache-commit/trunk' into HDFS-7240

Posted by ar...@apache.org.
Merge remote-tracking branch 'apache-commit/trunk' into HDFS-7240

# Conflicts:
#	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
#	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java


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

Branch: refs/heads/HDFS-7240
Commit: 6da5a33bbe8b626dc9a3ad133abb2b1581560a97
Parents: 9e63be7 979c9ca
Author: Arpit Agarwal <ar...@apache.org>
Authored: Tue Jul 14 13:24:46 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Tue Jul 14 13:24:46 2015 -0700

----------------------------------------------------------------------
 dev-support/releasedocmaker.py                  |  76 +-
 hadoop-common-project/hadoop-common/CHANGES.txt |  32 +-
 hadoop-common-project/hadoop-common/pom.xml     |   5 +
 .../java/org/apache/hadoop/fs/FileContext.java  |   3 +
 .../java/org/apache/hadoop/fs/FileSystem.java   |  20 +-
 .../main/java/org/apache/hadoop/fs/Globber.java |  17 +
 .../hadoop/fs/sftp/SFTPConnectionPool.java      | 303 +++++++
 .../apache/hadoop/fs/sftp/SFTPFileSystem.java   | 671 ++++++++++++++
 .../apache/hadoop/fs/sftp/SFTPInputStream.java  | 130 +++
 .../org/apache/hadoop/fs/sftp/package-info.java |  19 +
 .../java/org/apache/hadoop/fs/shell/Touch.java  |  84 ++
 .../java/org/apache/hadoop/fs/shell/Touchz.java |  84 --
 .../hadoop/io/compress/bzip2/Bzip2Factory.java  |   2 +-
 .../org/apache/hadoop/jmx/package-info.java     |   5 +-
 .../org/apache/hadoop/util/CpuTimeTracker.java  | 115 +++
 .../java/org/apache/hadoop/util/SysInfo.java    | 137 +++
 .../org/apache/hadoop/util/SysInfoLinux.java    | 690 +++++++++++++++
 .../org/apache/hadoop/util/SysInfoWindows.java  | 208 +++++
 .../src/site/markdown/DeprecatedProperties.md   |   1 +
 ...yptoStreamsWithOpensslAesCtrCryptoCodec.java |   3 +
 .../hadoop/fs/sftp/TestSFTPFileSystem.java      | 308 +++++++
 .../apache/hadoop/util/TestSysInfoLinux.java    | 432 ++++++++++
 .../apache/hadoop/util/TestSysInfoWindows.java  | 100 +++
 .../dev-support/findbugsExcludeFile.xml         |   4 +
 hadoop-hdfs-project/hadoop-hdfs-client/pom.xml  |  42 +
 .../hdfs/client/HdfsClientConfigKeys.java       |   3 +-
 .../org/apache/hadoop/hdfs/inotify/Event.java   |  37 +-
 .../hadoop/hdfs/protocol/ClientProtocol.java    | 606 ++++++-------
 .../src/main/proto/ClientDatanodeProtocol.proto | 247 ++++++
 .../src/main/proto/ClientNamenodeProtocol.proto | 863 +++++++++++++++++++
 .../hadoop-hdfs-client/src/main/proto/acl.proto | 108 +++
 .../src/main/proto/datatransfer.proto           | 304 +++++++
 .../src/main/proto/encryption.proto             |  67 ++
 .../src/main/proto/hdfs.proto                   | 619 +++++++++++++
 .../src/main/proto/inotify.proto                | 133 +++
 .../src/main/proto/xattr.proto                  |  75 ++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  41 +-
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |  10 +-
 .../hadoop-hdfs/src/contrib/bkjournal/pom.xml   |   2 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |   1 +
 .../ClientNamenodeProtocolTranslatorPB.java     |   2 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  17 +
 .../hadoop/hdfs/server/balancer/Dispatcher.java |  18 +
 .../hdfs/server/blockmanagement/BlockInfo.java  |   7 +-
 .../blockmanagement/BlockInfoContiguous.java    |   9 +-
 .../BlockInfoUnderConstruction.java             |  22 +-
 .../BlockInfoUnderConstructionContiguous.java   |  13 +-
 .../server/blockmanagement/BlockManager.java    | 141 +--
 .../hdfs/server/blockmanagement/BlocksMap.java  |   4 +-
 .../ContiguousBlockStorageOp.java               |   7 +-
 .../blockmanagement/CorruptReplicasMap.java     |  62 +-
 .../hdfs/server/datanode/BlockReceiver.java     |  34 +-
 .../server/datanode/fsdataset/FsVolumeSpi.java  |  32 +-
 .../apache/hadoop/hdfs/server/mover/Mover.java  |  27 +-
 .../hdfs/server/namenode/FSDirConcatOp.java     |   2 +-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |   6 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |   2 +-
 .../hdfs/server/namenode/FSEditLogOp.java       |   4 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  33 +-
 .../namenode/InotifyFSEditLogOpTranslator.java  |   4 +
 .../hdfs/server/namenode/NamenodeFsck.java      |  12 +-
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |  10 +-
 .../src/main/proto/ClientDatanodeProtocol.proto | 247 ------
 .../src/main/proto/ClientNamenodeProtocol.proto | 863 -------------------
 .../hadoop-hdfs/src/main/proto/acl.proto        | 113 ---
 .../src/main/proto/datatransfer.proto           | 304 -------
 .../hadoop-hdfs/src/main/proto/editlog.proto    |  35 +
 .../hadoop-hdfs/src/main/proto/encryption.proto |  67 --
 .../hadoop-hdfs/src/main/proto/hdfs.proto       | 619 -------------
 .../hadoop-hdfs/src/main/proto/inotify.proto    | 126 ---
 .../hadoop-hdfs/src/main/proto/xattr.proto      |  80 --
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |   6 +-
 .../hdfs/TestDFSInotifyEventInputStream.java    |  17 +-
 .../hadoop/hdfs/TestDistributedFileSystem.java  |  43 +-
 .../java/org/apache/hadoop/hdfs/TestQuota.java  |  21 +
 .../blockmanagement/BlockManagerTestUtil.java   |   7 +-
 .../server/blockmanagement/TestBlockInfo.java   |  10 +-
 .../blockmanagement/TestBlockManager.java       |  10 +-
 .../blockmanagement/TestCorruptReplicaInfo.java |  15 +-
 .../hadoop/hdfs/server/mover/TestMover.java     |   2 +-
 .../hdfs/server/namenode/TestFileTruncate.java  |  21 +-
 .../hdfs/server/namenode/TestINodeFile.java     |   9 +
 .../hdfs/tools/TestDelegationTokenFetcher.java  |  39 +
 hadoop-mapreduce-project/CHANGES.txt            |  10 +
 .../dev-support/findbugs-exclude.xml            |   1 +
 .../mapreduce/v2/app/job/impl/JobImpl.java      |   4 +-
 .../v2/app/job/impl/TaskAttemptImpl.java        |   4 +-
 .../mapreduce/v2/app/job/impl/TaskImpl.java     |   4 +-
 .../src/site/markdown/MapReduceTutorial.md      |   2 +-
 .../hadoop/mapred/TestShuffleHandler.java       |   6 +-
 hadoop-maven-plugins/pom.xml                    |   8 +
 .../hadoop/maven/plugin/protoc/ProtocMojo.java  | 188 +++-
 hadoop-project/pom.xml                          |   5 +
 .../gridmix/DummyResourceCalculatorPlugin.java  |  36 +
 .../hadoop/yarn/sls/nodemanager/NodeInfo.java   |   8 +-
 .../yarn/sls/scheduler/RMNodeWrapper.java       |   5 +
 hadoop-yarn-project/CHANGES.txt                 |  38 +
 .../yarn/server/api/ContainerContext.java       |  19 +
 .../api/ContainerInitializationContext.java     |   7 +
 .../server/api/ContainerTerminationContext.java |   7 +
 .../hadoop/yarn/server/api/ContainerType.java   |  34 +
 .../apache/hadoop/yarn/util/package-info.java   |   2 -
 .../src/main/proto/yarn_protos.proto            |   5 +
 .../api/async/impl/NMClientAsyncImpl.java       |   4 +-
 .../yarn/client/TestResourceTrackerOnHA.java    |   2 +-
 .../yarn/api/records/impl/pb/ProtoUtils.java    |  12 +
 .../yarn/client/api/impl/package-info.java      |   4 -
 .../hadoop/yarn/client/api/package-info.java    |   4 -
 .../hadoop/yarn/factories/package-info.java     |   2 -
 .../yarn/factory/providers/package-info.java    |   2 -
 .../yarn/security/ContainerTokenIdentifier.java |  43 +-
 .../state/InvalidStateTransitionException.java  |  51 ++
 .../state/InvalidStateTransitonException.java   |  21 +-
 .../apache/hadoop/yarn/state/StateMachine.java  |   2 +-
 .../hadoop/yarn/state/StateMachineFactory.java  |  10 +-
 .../apache/hadoop/yarn/util/CpuTimeTracker.java | 100 ---
 .../util/LinuxResourceCalculatorPlugin.java     | 392 +--------
 .../yarn/util/ProcfsBasedProcessTree.java       |  34 +-
 .../yarn/util/ResourceCalculatorPlugin.java     | 107 ++-
 .../yarn/util/WindowsBasedProcessTree.java      |   2 +-
 .../util/WindowsResourceCalculatorPlugin.java   | 158 +---
 .../main/proto/server/yarn_security_token.proto |   1 +
 .../src/main/resources/yarn-default.xml         | 466 +++++++++-
 .../yarn/conf/TestYarnConfigurationFields.java  |  44 +-
 .../yarn/security/TestYARNTokenIdentifier.java  |  53 ++
 .../util/TestLinuxResourceCalculatorPlugin.java | 324 -------
 .../util/TestResourceCalculatorProcessTree.java |   2 +-
 .../TestWindowsResourceCalculatorPlugin.java    |  86 --
 .../yarn/server/api/records/NodeStatus.java     |  43 +-
 .../server/api/records/ResourceUtilization.java | 133 +++
 .../api/records/impl/pb/NodeStatusPBImpl.java   |  34 +-
 .../impl/pb/ResourceUtilizationPBImpl.java      | 104 +++
 .../yarn/server/api/records/package-info.java   |  19 +
 .../main/proto/yarn_server_common_protos.proto  |   7 +
 .../nodemanager/NodeStatusUpdaterImpl.java      |  19 +-
 .../containermanager/AuxServices.java           |   6 +-
 .../application/ApplicationImpl.java            |   4 +-
 .../container/ContainerImpl.java                |   4 +-
 .../localizer/LocalizedResource.java            |   4 +-
 .../monitor/ContainersMonitor.java              |   3 +-
 .../monitor/ContainersMonitorImpl.java          |  28 +
 .../TestPrivilegedOperationExecutor.java        |   6 +-
 .../ProportionalCapacityPreemptionPolicy.java   |   4 +-
 .../resourcemanager/recovery/RMStateStore.java  |   4 +-
 .../recovery/ZKRMStateStore.java                |   3 +-
 .../reservation/GreedyReservationAgent.java     |  27 +-
 .../reservation/InMemoryPlan.java               |   9 +-
 .../InMemoryReservationAllocation.java          |  24 +-
 .../RLESparseResourceAllocation.java            |  43 +-
 .../reservation/ReservationAllocation.java      |   3 +-
 .../reservation/ReservationSystemUtil.java      |  51 ++
 .../server/resourcemanager/rmapp/RMAppImpl.java |   4 +-
 .../rmapp/attempt/RMAppAttemptImpl.java         |   4 +-
 .../rmcontainer/RMContainerImpl.java            |   4 +-
 .../server/resourcemanager/rmnode/RMNode.java   |   2 +
 .../resourcemanager/rmnode/RMNodeImpl.java      |  47 +-
 .../scheduler/SchedulerApplicationAttempt.java  |  17 +-
 .../scheduler/capacity/LeafQueue.java           |   3 +-
 .../scheduler/capacity/ParentQueue.java         |   2 +-
 .../scheduler/fair/FSLeafQueue.java             |   9 +-
 .../scheduler/fair/FairScheduler.java           |  40 +-
 .../scheduler/fair/SchedulingPolicy.java        |  11 +
 .../DominantResourceFairnessPolicy.java         |  18 +-
 .../fair/policies/FairSharePolicy.java          |  11 +-
 .../scheduler/fair/policies/FifoPolicy.java     |  15 +-
 .../security/RMContainerTokenSecretManager.java |  13 +-
 .../resourcemanager/webapp/RMAppsBlock.java     |  10 +-
 .../server/resourcemanager/Application.java     |  12 +
 .../yarn/server/resourcemanager/MockNodes.java  |   5 +
 .../resourcemanager/TestRMNodeTransitions.java  |  36 +-
 ...estProportionalCapacityPreemptionPolicy.java | 253 ++++--
 ...pacityPreemptionPolicyForNodePartitions.java | 114 ++-
 .../reservation/ReservationSystemTestUtil.java  |  11 +-
 .../reservation/TestCapacityOverTimePolicy.java |  16 +-
 .../reservation/TestGreedyReservationAgent.java |   2 +-
 .../reservation/TestInMemoryPlan.java           |  37 +-
 .../TestInMemoryReservationAllocation.java      |  29 +-
 .../TestRLESparseResourceAllocation.java        |  33 +-
 .../TestSimpleCapacityReplanner.java            |  11 +-
 .../capacity/TestContainerAllocation.java       |   5 +-
 .../scheduler/capacity/TestQueueParsing.java    |  38 +
 .../scheduler/fair/FairSchedulerTestBase.java   |  31 +-
 .../scheduler/fair/TestFSLeafQueue.java         |  64 ++
 .../scheduler/fair/TestFairScheduler.java       | 207 ++++-
 .../scheduler/fifo/TestFifoScheduler.java       |   4 +
 185 files changed, 9070 insertions(+), 4678 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6da5a33b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
index 0000000,86fb462..2df5955
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
@@@ -1,0 -1,611 +1,619 @@@
+ /**
+  * 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.
+  */
+ 
+ /**
+  * These .proto interfaces are private and stable.
+  * Please see http://wiki.apache.org/hadoop/Compatibility
+  * for what changes are allowed for a *stable* .proto interface.
+  */
+ 
+ // This file contains protocol buffers that are used throughout HDFS -- i.e.
+ // by the client, server, and data transfer protocols.
+ 
+ 
+ option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+ option java_outer_classname = "HdfsProtos";
+ option java_generate_equals_and_hash = true;
+ package hadoop.hdfs;
+ 
+ import "Security.proto";
+ 
+ /**
+  * Extended block idenfies a block
+  */
+ message ExtendedBlockProto {
+   required string poolId = 1;   // Block pool id - gloablly unique across clusters
+   required uint64 blockId = 2;  // the local id within a pool
+   required uint64 generationStamp = 3;
+   optional uint64 numBytes = 4 [default = 0];  // len does not belong in ebid 
+                                                // here for historical reasons
+ }
+ 
+ /**
+  * Identifies a Datanode
+  */
+ message DatanodeIDProto {
+   required string ipAddr = 1;    // IP address
+   required string hostName = 2;  // hostname
+   required string datanodeUuid = 3;     // UUID assigned to the Datanode. For
+                                         // upgraded clusters this is the same
+                                         // as the original StorageID of the
+                                         // Datanode.
+   required uint32 xferPort = 4;  // data streaming port
+   required uint32 infoPort = 5;  // datanode http port
+   required uint32 ipcPort = 6;   // ipc server port
+   optional uint32 infoSecurePort = 7 [default = 0]; // datanode https port
+ }
+ 
+ /**
+  * Datanode local information
+  */
+ message DatanodeLocalInfoProto {
+   required string softwareVersion = 1;
+   required string configVersion = 2;
+   required uint64 uptime = 3;
+ }
+ 
+ /**
+  * DatanodeInfo array
+  */
+ message DatanodeInfosProto {
+   repeated DatanodeInfoProto datanodes = 1;
+ }
+ 
+ /**
+  * The status of a Datanode
+  */
+ message DatanodeInfoProto {
+   required DatanodeIDProto id = 1;
+   optional uint64 capacity = 2 [default = 0];
+   optional uint64 dfsUsed = 3 [default = 0];
+   optional uint64 remaining = 4 [default = 0];
+   optional uint64 blockPoolUsed = 5 [default = 0];
+   optional uint64 lastUpdate = 6 [default = 0];
+   optional uint32 xceiverCount = 7 [default = 0];
+   optional string location = 8;
+   enum AdminState {
+     NORMAL = 0;
+     DECOMMISSION_INPROGRESS = 1;
+     DECOMMISSIONED = 2;
+   }
+ 
+   optional AdminState adminState = 10 [default = NORMAL];
+   optional uint64 cacheCapacity = 11 [default = 0];
+   optional uint64 cacheUsed = 12 [default = 0];
+   optional uint64 lastUpdateMonotonic = 13 [default = 0];
+ }
+ 
+ /**
+  * Represents a storage available on the datanode
+  */
+ message DatanodeStorageProto {
+   enum StorageState {
+     NORMAL = 0;
+     READ_ONLY_SHARED = 1;
+   }
+ 
+   required string storageUuid = 1;
+   optional StorageState state = 2 [default = NORMAL];
+   optional StorageTypeProto storageType = 3 [default = DISK];
+ }
+ 
+ message StorageReportProto {
+   required string storageUuid = 1 [ deprecated = true ];
+   optional bool failed = 2 [ default = false ];
+   optional uint64 capacity = 3 [ default = 0 ];
+   optional uint64 dfsUsed = 4 [ default = 0 ];
+   optional uint64 remaining = 5 [ default = 0 ];
+   optional uint64 blockPoolUsed = 6 [ default = 0 ];
+   optional DatanodeStorageProto storage = 7; // supersedes StorageUuid
+ }
+ 
+ /**
+  * Summary of a file or directory
+  */
+ message ContentSummaryProto {
+   required uint64 length = 1;
+   required uint64 fileCount = 2;
+   required uint64 directoryCount = 3;
+   required uint64 quota = 4;
+   required uint64 spaceConsumed = 5;
+   required uint64 spaceQuota = 6;
+   optional StorageTypeQuotaInfosProto typeQuotaInfos = 7;
+ }
+ 
+ /**
+  * Storage type quota and usage information of a file or directory
+  */
+ message StorageTypeQuotaInfosProto {
+   repeated StorageTypeQuotaInfoProto typeQuotaInfo = 1;
+ }
+ 
+ message StorageTypeQuotaInfoProto {
+   required StorageTypeProto type = 1;
+   required uint64 quota = 2;
+   required uint64 consumed = 3;
+ }
+ 
+ /**
+  * Contains a list of paths corresponding to corrupt files and a cookie
+  * used for iterative calls to NameNode.listCorruptFileBlocks.
+  *
+  */
+ message CorruptFileBlocksProto {
+  repeated string files = 1;
+  required string   cookie = 2;
+ }
+ 
+ /**
+  * File or Directory permision - same spec as posix
+  */
+ message FsPermissionProto {
+   required uint32 perm = 1;       // Actually a short - only 16bits used
+ }
+ 
+ /**
+  * Types of recognized storage media.
+  */
+ enum StorageTypeProto {
+   DISK = 1;
+   SSD = 2;
+   ARCHIVE = 3;
+   RAM_DISK = 4;
+ }
+ 
+ /**
+  * A list of storage types. 
+  */
+ message StorageTypesProto {
+   repeated StorageTypeProto storageTypes = 1;
+ }
+ 
+ /**
+  * Block replica storage policy.
+  */
+ message BlockStoragePolicyProto {
+   required uint32 policyId = 1;
+   required string name = 2;
+   // a list of storage types for storing the block replicas when creating a
+   // block.
+   required StorageTypesProto creationPolicy = 3;
+   // A list of storage types for creation fallback storage.
+   optional StorageTypesProto creationFallbackPolicy = 4;
+   optional StorageTypesProto replicationFallbackPolicy = 5;
+ }
+ 
+ /**
+  * A list of storage IDs. 
+  */
+ message StorageUuidsProto {
+   repeated string storageUuids = 1;
+ }
+ 
+ /**
+  * A LocatedBlock gives information about a block and its location.
+  */ 
+ message LocatedBlockProto {
+   required ExtendedBlockProto b  = 1;
+   required uint64 offset = 2;           // offset of first byte of block in the file
+   repeated DatanodeInfoProto locs = 3;  // Locations ordered by proximity to client ip
+   required bool corrupt = 4;            // true if all replicas of a block are corrupt, else false
+                                         // If block has few corrupt replicas, they are filtered and 
+                                         // their locations are not part of this object
+ 
+   required hadoop.common.TokenProto blockToken = 5;
+   repeated bool isCached = 6 [packed=true]; // if a location in locs is cached
+   repeated StorageTypeProto storageTypes = 7;
+   repeated string storageIDs = 8;
+ }
+ 
+ message DataEncryptionKeyProto {
+   required uint32 keyId = 1;
+   required string blockPoolId = 2;
+   required bytes nonce = 3;
+   required bytes encryptionKey = 4;
+   required uint64 expiryDate = 5;
+   optional string encryptionAlgorithm = 6;
+ }
+ 
+ /**
+  * Cipher suite.
+  */
+ enum CipherSuiteProto {
+     UNKNOWN = 1;
+     AES_CTR_NOPADDING = 2;
+ }
+ 
+ /**
+  * Crypto protocol version used to access encrypted files.
+  */
+ enum CryptoProtocolVersionProto {
+     UNKNOWN_PROTOCOL_VERSION = 1;
+     ENCRYPTION_ZONES = 2;
+ }
+ 
+ /**
+  * Encryption information for a file.
+  */
+ message FileEncryptionInfoProto {
+   required CipherSuiteProto suite = 1;
+   required CryptoProtocolVersionProto cryptoProtocolVersion = 2;
+   required bytes key = 3;
+   required bytes iv = 4;
+   required string keyName = 5;
+   required string ezKeyVersionName = 6;
+ }
+ 
+ /**
+  * Encryption information for an individual
+  * file within an encryption zone
+  */
+ message PerFileEncryptionInfoProto {
+   required bytes key = 1;
+   required bytes iv = 2;
+   required string ezKeyVersionName = 3;
+ }
+ 
+ /**
+  * Encryption information for an encryption
+  * zone
+  */
+ message ZoneEncryptionInfoProto {
+   required CipherSuiteProto suite = 1;
+   required CryptoProtocolVersionProto cryptoProtocolVersion = 2;
+   required string keyName = 3;
+ }
+ 
+ /**
+  * Cipher option
+  */
+ message CipherOptionProto {
+   required CipherSuiteProto suite = 1;
+   optional bytes inKey = 2;
+   optional bytes inIv = 3;
+   optional bytes outKey = 4;
+   optional bytes outIv = 5;
+ }
+ 
+ /**
+  * A set of file blocks and their locations.
+  */
+ message LocatedBlocksProto {
+   required uint64 fileLength = 1;
+   repeated LocatedBlockProto blocks = 2;
+   required bool underConstruction = 3;
+   optional LocatedBlockProto lastBlock = 4;
+   required bool isLastBlockComplete = 5;
+   optional FileEncryptionInfoProto fileEncryptionInfo = 6;
+ }
+ 
+ /**
+  * Status of a file, directory or symlink
+  * Optionally includes a file's block locations if requested by client on the rpc call.
+  */
+ message HdfsFileStatusProto {
+   enum FileType {
+     IS_DIR = 1;
+     IS_FILE = 2;
+     IS_SYMLINK = 3;
+   }
+   required FileType fileType = 1;
+   required bytes path = 2;          // local name of inode encoded java UTF8
+   required uint64 length = 3;
+   required FsPermissionProto permission = 4;
+   required string owner = 5;
+   required string group = 6;
+   required uint64 modification_time = 7;
+   required uint64 access_time = 8;
+ 
+   // Optional fields for symlink
+   optional bytes symlink = 9;             // if symlink, target encoded java UTF8 
+ 
+   // Optional fields for file
+   optional uint32 block_replication = 10 [default = 0]; // only 16bits used
+   optional uint64 blocksize = 11 [default = 0];
+   optional LocatedBlocksProto locations = 12;  // suppled only if asked by client
+ 
+   // Optional field for fileId
+   optional uint64 fileId = 13 [default = 0]; // default as an invalid id
+   optional int32 childrenNum = 14 [default = -1];
+   // Optional field for file encryption
+   optional FileEncryptionInfoProto fileEncryptionInfo = 15;
+ 
+   optional uint32 storagePolicy = 16 [default = 0]; // block storage policy id
+ } 
+ 
+ /**
+  * Checksum algorithms/types used in HDFS
+  * Make sure this enum's integer values match enum values' id properties defined
+  * in org.apache.hadoop.util.DataChecksum.Type
+  */
+ enum ChecksumTypeProto {
+   CHECKSUM_NULL = 0;
+   CHECKSUM_CRC32 = 1;
+   CHECKSUM_CRC32C = 2;
+ }
+ 
+ /**
+  * HDFS Server Defaults
+  */
+ message FsServerDefaultsProto {
+   required uint64 blockSize = 1;
+   required uint32 bytesPerChecksum = 2;
+   required uint32 writePacketSize = 3;
+   required uint32 replication = 4; // Actually a short - only 16 bits used
+   required uint32 fileBufferSize = 5;
+   optional bool encryptDataTransfer = 6 [default = false];
+   optional uint64 trashInterval = 7 [default = 0];
+   optional ChecksumTypeProto checksumType = 8 [default = CHECKSUM_CRC32];
+ }
+ 
+ 
+ /**
+  * Directory listing
+  */
+ message DirectoryListingProto {
+   repeated HdfsFileStatusProto partialListing = 1;
+   required uint32 remainingEntries  = 2;
+ }
+ 
+ /**
+  * Status of a snapshottable directory: besides the normal information for 
+  * a directory status, also include snapshot quota, number of snapshots, and
+  * the full path of the parent directory. 
+  */
+ message SnapshottableDirectoryStatusProto {
+   required HdfsFileStatusProto dirStatus = 1;
+ 
+   // Fields specific for snapshottable directory
+   required uint32 snapshot_quota = 2;
+   required uint32 snapshot_number = 3;
+   required bytes parent_fullpath = 4;
+ }
+ 
+ /**
+  * Snapshottable directory listing
+  */
+ message SnapshottableDirectoryListingProto {
+   repeated SnapshottableDirectoryStatusProto snapshottableDirListing = 1;
+ }
+ 
+ /**
+  * Snapshot diff report entry
+  */
+ message SnapshotDiffReportEntryProto {
+   required bytes fullpath = 1;
+   required string modificationLabel = 2;
+   optional bytes targetPath = 3;
+ }
+ 
+ /**
+  * Snapshot diff report
+  */
+ message SnapshotDiffReportProto {
+   // full path of the directory where snapshots were taken
+   required string snapshotRoot = 1;
+   required string fromSnapshot = 2;
+   required string toSnapshot = 3;
+   repeated SnapshotDiffReportEntryProto diffReportEntries = 4;
+ }
+ 
+ /**
+  * Common node information shared by all the nodes in the cluster
+  */
+ message StorageInfoProto {
+   required uint32 layoutVersion = 1; // Layout version of the file system
+   required uint32 namespceID = 2;    // File system namespace ID
+   required string clusterID = 3;     // ID of the cluster
+   required uint64 cTime = 4;         // File system creation time
++
++  enum NodeTypeProto {
++    NAME_NODE = 1;
++    DATA_NODE = 2;
++    JOURNAL_NODE = 3;
++    STORAGE_CONTAINER_SERVICE = 4;
++  }
++  optional NodeTypeProto nodeType = 5;
+ }
+ 
+ /**
+  * Information sent by a namenode to identify itself to the primary namenode.
+  */
+ message NamenodeRegistrationProto {
+   required string rpcAddress = 1;    // host:port of the namenode RPC address
+   required string httpAddress = 2;   // host:port of the namenode http server
+   enum NamenodeRoleProto {
+     NAMENODE = 1;
+     BACKUP = 2;
+     CHECKPOINT = 3;
+   }
+   required StorageInfoProto storageInfo = 3;  // Node information
+   optional NamenodeRoleProto role = 4 [default = NAMENODE];        // Namenode role
+ }
+ 
+ /**
+  * Unique signature to identify checkpoint transactions.
+  */
+ message CheckpointSignatureProto {
+   required string blockPoolId = 1;
+   required uint64 mostRecentCheckpointTxId = 2;
+   required uint64 curSegmentTxId = 3;
+   required StorageInfoProto storageInfo = 4;
+ }
+ 
+ /**
+  * Command sent from one namenode to another namenode.
+  */
+ message NamenodeCommandProto {
+   enum Type {
+     NamenodeCommand = 0;      // Base command
+     CheckPointCommand = 1;    // Check point command
+   }
+   required uint32 action = 1;
+   required Type type = 2;
+   optional CheckpointCommandProto checkpointCmd = 3;
+ }
+ 
+ /**
+  * Command returned from primary to checkpointing namenode.
+  * This command has checkpoint signature that identifies
+  * checkpoint transaction and is needed for further
+  * communication related to checkpointing.
+  */
+ message CheckpointCommandProto {
+   // Unique signature to identify checkpoint transation
+   required CheckpointSignatureProto signature = 1; 
+ 
+   // If true, return transfer image to primary upon the completion of checkpoint
+   required bool needToReturnImage = 2;
+ }
+ 
+ /**
+  * Block information
+  *
+  * Please be wary of adding additional fields here, since INodeFiles
+  * need to fit in PB's default max message size of 64MB.
+  * We restrict the max # of blocks per file
+  * (dfs.namenode.fs-limits.max-blocks-per-file), but it's better
+  * to avoid changing this.
+  */
+ message BlockProto {
+   required uint64 blockId = 1;
+   required uint64 genStamp = 2;
+   optional uint64 numBytes = 3 [default = 0];
+ }
+ 
+ /**
+  * Block and datanodes where is it located
+  */
+ message BlockWithLocationsProto {
+   required BlockProto block = 1;   // Block
+   repeated string datanodeUuids = 2; // Datanodes with replicas of the block
+   repeated string storageUuids = 3;  // Storages with replicas of the block
+   repeated StorageTypeProto storageTypes = 4;
+ }
+ 
+ /**
+  * List of block with locations
+  */
+ message BlocksWithLocationsProto {
+   repeated BlockWithLocationsProto blocks = 1;
+ }
+ 
+ /**
+  * Editlog information with available transactions
+  */
+ message RemoteEditLogProto {
+   required uint64 startTxId = 1;  // Starting available edit log transaction
+   required uint64 endTxId = 2;    // Ending available edit log transaction
+   optional bool isInProgress = 3 [default = false];
+ }
+ 
+ /**
+  * Enumeration of editlogs available on a remote namenode
+  */
+ message RemoteEditLogManifestProto {
+   repeated RemoteEditLogProto logs = 1;
+ }
+ 
+ /**
+  * Namespace information that describes namespace on a namenode
+  */
+ message NamespaceInfoProto {
+   required string buildVersion = 1;         // Software revision version (e.g. an svn or git revision)
+   required uint32 unused = 2;               // Retained for backward compatibility
+   required string blockPoolID = 3;          // block pool used by the namespace
+   required StorageInfoProto storageInfo = 4;// Node information
+   required string softwareVersion = 5;      // Software version number (e.g. 2.0.0)
+   optional uint64 capabilities = 6 [default = 0]; // feature flags
+ }
+ 
+ /**
+  * Block access token information
+  */
+ message BlockKeyProto {
+   required uint32 keyId = 1;      // Key identifier
+   required uint64 expiryDate = 2; // Expiry time in milliseconds
+   optional bytes keyBytes = 3;    // Key secret
+ }
+ 
+ /**
+  * Current key and set of block keys at the namenode.
+  */
+ message ExportedBlockKeysProto {
+   required bool isBlockTokenEnabled = 1;
+   required uint64 keyUpdateInterval = 2;
+   required uint64 tokenLifeTime = 3;
+   required BlockKeyProto currentKey = 4;
+   repeated BlockKeyProto allKeys = 5;
+ }
+ 
+ /**
+  * State of a block replica at a datanode
+  */
+ enum ReplicaStateProto {
+   FINALIZED = 0;  // State of a replica when it is not modified
+   RBW = 1;        // State of replica that is being written to
+   RWR = 2;        // State of replica that is waiting to be recovered
+   RUR = 3;        // State of replica that is under recovery
+   TEMPORARY = 4;  // State of replica that is created for replication
+ }
+ 
+ /**
+  * Block that needs to be recovered with at a given location
+  */
+ message RecoveringBlockProto {
+   required uint64 newGenStamp = 1;        // New genstamp post recovery
+   required LocatedBlockProto block = 2;   // Block to be recovered
+   optional BlockProto truncateBlock = 3;  // New block for recovery (truncate)
+ }
+ 
+ /**
+  * void request
+  */
+ message VersionRequestProto {
+ }
+ 
+ /**
+  * Version response from namenode.
+  */
+ message VersionResponseProto {
+   required NamespaceInfoProto info = 1;
+ }
+ 
+ /**
+  * Information related to a snapshot
+  * TODO: add more information
+  */
+ message SnapshotInfoProto {
+   required string snapshotName = 1;
+   required string snapshotRoot = 2;
+   required FsPermissionProto permission = 3;
+   required string owner = 4;
+   required string group = 5;
+   required string createTime = 6;
+   // TODO: do we need access time?
+ }
+ 
+ /**
+  * Rolling upgrade status
+  */
+ message RollingUpgradeStatusProto {
+   required string blockPoolId = 1;
+   optional bool finalized = 2 [default = false];
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6da5a33b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6da5a33b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6da5a33b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6da5a33b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6da5a33b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
index 64ad701,ee01924..d58ce79
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
@@@ -42,9 -38,29 +42,9 @@@ public interface FsVolumeSpi extends Vo
     */
    FsVolumeReference obtainReference() throws ClosedChannelException;
  
 -  /** @return the StorageUuid of the volume */
 -  String getStorageID();
 -
 -  /** @return a list of block pools. */
 -  String[] getBlockPoolList();
 -
 -  /** @return the available storage space in bytes. */
 -  long getAvailable() throws IOException;
 -
 -  /** @return the base path to the volume */
 -  String getBasePath();
 -
 -  /** @return the path to the volume */
 -  String getPath(String bpid) throws IOException;
 -
    /** @return the directory for the finalized blocks in the block pool. */
-   public File getFinalizedDir(String bpid) throws IOException;
+   File getFinalizedDir(String bpid) throws IOException;
    
 -  StorageType getStorageType();
 -
 -  /** Returns true if the volume is NOT backed by persistent storage. */
 -  boolean isTransientStorage();
 -
    /**
     * Reserve disk space for an RBW block so a writer does not run out of
     * space before the block is full.
@@@ -164,9 -180,10 +164,9 @@@
     * @throws IOException     If there was an IO error loading the saved
     *                           block iterator.
     */
-   public BlockIterator loadBlockIterator(String bpid, String name)
 -  BlockIterator loadBlockIterator(String bpid, String name) throws IOException;
++  BlockIterator loadBlockIterator(String bpid, String name)
 +      throws IOException;
  
 -  /**
 -   * Get the FSDatasetSpi which this volume is a part of.
 -   */
 +  @Override
-   public FsDatasetSpi getDataset();
+   FsDatasetSpi getDataset();
  }


[21/50] hadoop git commit: YARN-1012. Report NM aggregated container resource utilization in heartbeat. (Inigo Goiri via kasha)

Posted by ar...@apache.org.
YARN-1012. Report NM aggregated container resource utilization in heartbeat. (Inigo Goiri via kasha)


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

Branch: refs/heads/HDFS-7240
Commit: 527c40e4d664c721b8f32d7cd8df21b2666fea8a
Parents: fffb15b
Author: Karthik Kambatla <ka...@apache.org>
Authored: Thu Jul 9 09:35:14 2015 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Thu Jul 9 09:35:14 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../yarn/client/TestResourceTrackerOnHA.java    |   2 +-
 .../yarn/server/api/records/NodeStatus.java     |  43 +++++-
 .../server/api/records/ResourceUtilization.java | 133 +++++++++++++++++++
 .../api/records/impl/pb/NodeStatusPBImpl.java   |  34 ++++-
 .../impl/pb/ResourceUtilizationPBImpl.java      | 104 +++++++++++++++
 .../yarn/server/api/records/package-info.java   |  19 +++
 .../main/proto/yarn_server_common_protos.proto  |   7 +
 .../nodemanager/NodeStatusUpdaterImpl.java      |  19 ++-
 .../monitor/ContainersMonitor.java              |   3 +-
 .../monitor/ContainersMonitorImpl.java          |  28 ++++
 11 files changed, 387 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/527c40e4/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index d1960e6..19f0854 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -131,6 +131,9 @@ Release 2.8.0 - UNRELEASED
     YARN-41. The RM should handle the graceful shutdown of the NM. (Devaraj K via 
     junping_du)
 
+    YARN-1012. Report NM aggregated container resource utilization in heartbeat. 
+    (Inigo Goiri via kasha)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/527c40e4/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 8167a58..c51570c 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);
     NodeHeartbeatRequest request2 =
         NodeHeartbeatRequest.newInstance(status, null, null,null);
     resourceTracker.nodeHeartbeat(request2);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/527c40e4/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 aad819d..38b0381 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
@@ -19,24 +19,48 @@ package org.apache.hadoop.yarn.server.api.records;
 
 import java.util.List;
 
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.util.Records;
 
-
+/**
+ * {@code NodeStatus} is a summary of the status of the node.
+ * <p>
+ * It includes information such as:
+ * <ul>
+ *   <li>Node information and status..</li>
+ *   <li>Container status.</li>
+ * </ul>
+ */
 public abstract class NodeStatus {
-  
+
+  /**
+   * Create a new {@code NodeStatus}.
+   * @param nodeId Identifier for this node.
+   * @param responseId Identifier for the response.
+   * @param containerStatuses Status of the containers running in this node.
+   * @param keepAliveApplications Applications to keep alive.
+   * @param nodeHealthStatus Health status of the node.
+   * @param containersUtilizations Utilization of the containers in this node.
+   * @return New {@code NodeStatus} with the provided information.
+   */
   public static NodeStatus newInstance(NodeId nodeId, int responseId,
       List<ContainerStatus> containerStatuses,
       List<ApplicationId> keepAliveApplications,
-      NodeHealthStatus nodeHealthStatus) {
+      NodeHealthStatus nodeHealthStatus,
+      ResourceUtilization containersUtilization) {
     NodeStatus nodeStatus = Records.newRecord(NodeStatus.class);
     nodeStatus.setResponseId(responseId);
     nodeStatus.setNodeId(nodeId);
     nodeStatus.setContainersStatuses(containerStatuses);
     nodeStatus.setKeepAliveApplications(keepAliveApplications);
     nodeStatus.setNodeHealthStatus(nodeHealthStatus);
+    nodeStatus.setContainersUtilization(containersUtilization);
     return nodeStatus;
   }
 
@@ -55,4 +79,17 @@ public abstract class NodeStatus {
 
   public abstract void setNodeId(NodeId nodeId);
   public abstract void setResponseId(int responseId);
+
+  /**
+   * Get the <em>resource utilization</em> of the containers.
+   * @return <em>resource utilization</em> of the containers
+   */
+  @Public
+  @Stable
+  public abstract ResourceUtilization getContainersUtilization();
+
+  @Private
+  @Unstable
+  public abstract void setContainersUtilization(
+      ResourceUtilization containersUtilization);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/527c40e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/ResourceUtilization.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/ResourceUtilization.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/ResourceUtilization.java
new file mode 100644
index 0000000..39896a3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/ResourceUtilization.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.yarn.server.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * <code>ResourceUtilization</code> models the utilization of a set of computer
+ * resources in the cluster.
+ * </p>
+ */
+@Private
+@Evolving
+public abstract class ResourceUtilization implements
+    Comparable<ResourceUtilization> {
+  public static ResourceUtilization newInstance(int pmem, int vmem, float cpu) {
+    ResourceUtilization utilization =
+        Records.newRecord(ResourceUtilization.class);
+    utilization.setPhysicalMemory(pmem);
+    utilization.setVirtualMemory(vmem);
+    utilization.setCPU(cpu);
+    return utilization;
+  }
+
+  /**
+   * Get used <em>virtual memory</em>.
+   *
+   * @return <em>virtual memory</em> in MB
+   */
+  public abstract int getVirtualMemory();
+
+  /**
+   * Set used <em>virtual memory</em>.
+   *
+   * @param vmem <em>virtual memory</em> in MB
+   */
+  public abstract void setVirtualMemory(int vmem);
+
+  /**
+   * Get <em>physical memory</em>.
+   *
+   * @return <em>physical memory</em> in MB
+   */
+  public abstract int getPhysicalMemory();
+
+  /**
+   * Set <em>physical memory</em>.
+   *
+   * @param pmem <em>physical memory</em> in MB
+   */
+  public abstract void setPhysicalMemory(int pmem);
+
+  /**
+   * Get <em>CPU</em> utilization.
+   *
+   * @return <em>CPU utilization</em> normalized to 1 CPU
+   */
+  public abstract float getCPU();
+
+  /**
+   * Set <em>CPU</em> utilization.
+   *
+   * @param cpu <em>CPU utilization</em> normalized to 1 CPU
+   */
+  public abstract void setCPU(float cpu);
+
+  @Override
+  public int hashCode() {
+    final int prime = 263167;
+    int result = 3571;
+    result = prime * result + getVirtualMemory();
+    result = prime * result + getPhysicalMemory();
+    result = 31 * result + Float.valueOf(getCPU()).hashCode();
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (!(obj instanceof ResourceUtilization)) {
+      return false;
+    }
+    ResourceUtilization other = (ResourceUtilization) obj;
+    if (getVirtualMemory() != other.getVirtualMemory()
+        || getPhysicalMemory() != other.getPhysicalMemory()
+        || getCPU() != other.getCPU()) {
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return "<pmem:" + getPhysicalMemory() + ", vmem:" + getVirtualMemory()
+        + ", vCores:" + getCPU() + ">";
+  }
+
+  /**
+   * Add utilization to the current one.
+   * @param pmem Physical memory used to add.
+   * @param vmem Virtual memory used to add.
+   * @param cpu CPU utilization to add.
+   */
+  public void addTo(int pmem, int vmem, float cpu) {
+    this.setPhysicalMemory(this.getPhysicalMemory() + pmem);
+    this.setVirtualMemory(this.getVirtualMemory() + vmem);
+    this.setCPU(this.getCPU() + cpu);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/527c40e4/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 65376dc..fffd6a9 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
@@ -35,9 +35,10 @@ import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeHealthStatusProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.ResourceUtilizationProto;
 import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
-    
+import org.apache.hadoop.yarn.server.api.records.ResourceUtilization;
 
 public class NodeStatusPBImpl extends NodeStatus {
   NodeStatusProto proto = NodeStatusProto.getDefaultInstance();
@@ -291,6 +292,28 @@ public class NodeStatusPBImpl extends NodeStatus {
     this.nodeHealthStatus = healthStatus;
   }
 
+  @Override
+  public ResourceUtilization getContainersUtilization() {
+    NodeStatusProtoOrBuilder p =
+        this.viaProto ? this.proto : this.builder;
+    if (!p.hasContainersUtilization()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getContainersUtilization());
+  }
+
+  @Override
+  public void setContainersUtilization(
+      ResourceUtilization containersUtilization) {
+    maybeInitBuilder();
+    if (containersUtilization == null) {
+      this.builder.clearContainersUtilization();
+      return;
+    }
+    this.builder
+        .setContainersUtilization(convertToProtoFormat(containersUtilization));
+  }
+
   private NodeIdProto convertToProtoFormat(NodeId nodeId) {
     return ((NodeIdPBImpl)nodeId).getProto();
   }
@@ -323,4 +346,13 @@ public class NodeStatusPBImpl extends NodeStatus {
   private ApplicationIdProto convertToProtoFormat(ApplicationId c) {
     return ((ApplicationIdPBImpl)c).getProto();
   }
+
+  private ResourceUtilizationProto convertToProtoFormat(ResourceUtilization r) {
+    return ((ResourceUtilizationPBImpl) r).getProto();
+  }
+
+  private ResourceUtilizationPBImpl convertFromProtoFormat(
+      ResourceUtilizationProto p) {
+    return new ResourceUtilizationPBImpl(p);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/527c40e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/ResourceUtilizationPBImpl.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/ResourceUtilizationPBImpl.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/ResourceUtilizationPBImpl.java
new file mode 100644
index 0000000..01cda7a
--- /dev/null
+++ 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/ResourceUtilizationPBImpl.java
@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.api.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.ResourceUtilizationProto;
+import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.ResourceUtilizationProtoOrBuilder;
+import org.apache.hadoop.yarn.server.api.records.ResourceUtilization;
+
+@Private
+@Unstable
+public class ResourceUtilizationPBImpl extends ResourceUtilization {
+  private ResourceUtilizationProto proto = ResourceUtilizationProto
+      .getDefaultInstance();
+  private ResourceUtilizationProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public ResourceUtilizationPBImpl() {
+    builder = ResourceUtilizationProto.newBuilder();
+  }
+
+  public ResourceUtilizationPBImpl(ResourceUtilizationProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ResourceUtilizationProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ResourceUtilizationProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public int getPhysicalMemory() {
+    ResourceUtilizationProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.getPmem());
+  }
+
+  @Override
+  public void setPhysicalMemory(int pmem) {
+    maybeInitBuilder();
+    builder.setPmem(pmem);
+  }
+
+  @Override
+  public int getVirtualMemory() {
+    ResourceUtilizationProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.getVmem());
+  }
+
+  @Override
+  public void setVirtualMemory(int vmem) {
+    maybeInitBuilder();
+    builder.setPmem(vmem);
+  }
+
+  @Override
+  public float getCPU() {
+    ResourceUtilizationProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getCpu();
+  }
+
+  @Override
+  public void setCPU(float cpu) {
+    maybeInitBuilder();
+    builder.setCpu(cpu);
+  }
+
+  @Override
+  public int compareTo(ResourceUtilization other) {
+    int diff = this.getPhysicalMemory() - other.getPhysicalMemory();
+    if (diff == 0) {
+      diff = this.getVirtualMemory() - other.getVirtualMemory();
+      if (diff == 0) {
+        diff = Float.compare(this.getCPU(), other.getCPU());
+      }
+    }
+    return diff;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/527c40e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/package-info.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/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/package-info.java
new file mode 100644
index 0000000..bf8497f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+/** Server records. */
+package org.apache.hadoop.yarn.server.api.records;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/527c40e4/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 99149ac..a810813 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
@@ -36,6 +36,7 @@ message NodeStatusProto {
   repeated ContainerStatusProto containersStatuses = 3;
   optional NodeHealthStatusProto nodeHealthStatus = 4;
   repeated ApplicationIdProto keep_alive_applications = 5;
+  optional ResourceUtilizationProto containers_utilization = 6;
 }
 
 message MasterKeyProto {
@@ -52,4 +53,10 @@ message NodeHealthStatusProto {
 message VersionProto {
   optional int32 major_version = 1;
   optional int32 minor_version = 2;
+}
+
+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/527c40e4/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 3721b0e..30a2bd5 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
@@ -73,13 +73,14 @@ import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
 import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
+import org.apache.hadoop.yarn.server.api.records.ResourceUtilization;
 import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
 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.monitor.ContainersMonitor;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeLabelsProvider;
-import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils;
 import org.apache.hadoop.yarn.util.YarnVersionInfo;
 
@@ -429,13 +430,27 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
           + ", " + nodeHealthStatus.getHealthReport());
     }
     List<ContainerStatus> containersStatuses = getContainerStatuses();
+    ResourceUtilization containersUtilization = getContainersUtilization();
     NodeStatus nodeStatus =
         NodeStatus.newInstance(nodeId, responseId, containersStatuses,
-          createKeepAliveApplicationList(), nodeHealthStatus);
+          createKeepAliveApplicationList(), nodeHealthStatus,
+          containersUtilization);
 
     return nodeStatus;
   }
 
+  /**
+   * Get the aggregated utilization of the containers in this node.
+   * @return Resource utilization of all the containers.
+   */
+  private ResourceUtilization getContainersUtilization() {
+    ContainerManagerImpl containerManager =
+        (ContainerManagerImpl) this.context.getContainerManager();
+    ContainersMonitor containersMonitor =
+        containerManager.getContainersMonitor();
+    return containersMonitor.getContainersUtilization();
+  }
+
   // Iterate through the NMContext and clone and get all the containers'
   // statuses. If it's a completed container, add into the
   // recentlyStoppedContainers collections.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/527c40e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitor.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/ContainersMonitor.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/ContainersMonitor.java
index d3e2bf2..f0dd2e7 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/ContainersMonitor.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/ContainersMonitor.java
@@ -19,10 +19,11 @@
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor;
 
 import org.apache.hadoop.service.Service;
+import org.apache.hadoop.yarn.server.api.records.ResourceUtilization;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.nodemanager.ResourceView;
 
 public interface ContainersMonitor extends Service,
     EventHandler<ContainersMonitorEvent>, ResourceView {
-
+  public ResourceUtilization getContainersUtilization();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/527c40e4/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 b5f154d..57d1bad 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
@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 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.ContainerKillEvent;
@@ -78,6 +79,8 @@ public class ContainersMonitorImpl extends AbstractService implements
   private static final long UNKNOWN_MEMORY_LIMIT = -1L;
   private int nodeCpuPercentageForYARN;
 
+  private ResourceUtilization containersUtilization;
+
   public ContainersMonitorImpl(ContainerExecutor exec,
       AsyncDispatcher dispatcher, Context context) {
     super("containers-monitor");
@@ -89,6 +92,8 @@ public class ContainersMonitorImpl extends AbstractService implements
     this.containersToBeAdded = new HashMap<ContainerId, ProcessTreeInfo>();
     this.containersToBeRemoved = new ArrayList<ContainerId>();
     this.monitoringThread = new MonitoringThread();
+
+    this.containersUtilization = ResourceUtilization.newInstance(0, 0, 0.0f);
   }
 
   @Override
@@ -384,6 +389,11 @@ public class ContainersMonitorImpl extends AbstractService implements
           containersToBeRemoved.clear();
         }
 
+        // Temporary structure to calculate the total resource utilization of
+        // the containers
+        ResourceUtilization trackedContainersUtilization  =
+            ResourceUtilization.newInstance(0, 0, 0.0f);
+
         // Now do the monitoring for the trackingContainers
         // Check memory usage and kill any overflowing containers
         long vmemUsageByAllContainers = 0;
@@ -463,6 +473,12 @@ public class ContainersMonitorImpl extends AbstractService implements
                       currentPmemUsage, pmemLimit));
             }
 
+            // Add resource utilization for this container
+            trackedContainersUtilization.addTo(
+                (int) (currentPmemUsage >> 20),
+                (int) (currentVmemUsage >> 20),
+                milliVcoresUsed / 1000.0f);
+
             // Add usage to container metrics
             if (containerMetricsEnabled) {
               ContainerMetrics.forContainer(
@@ -542,6 +558,9 @@ public class ContainersMonitorImpl extends AbstractService implements
               + cpuUsagePercentPerCoreByAllContainers);
         }
 
+        // Save the aggregated utilization of the containers
+        setContainersUtilization(trackedContainersUtilization);
+
         try {
           Thread.sleep(monitoringInterval);
         } catch (InterruptedException e) {
@@ -614,6 +633,15 @@ public class ContainersMonitorImpl extends AbstractService implements
   }
 
   @Override
+  public ResourceUtilization getContainersUtilization() {
+    return this.containersUtilization;
+  }
+
+  public void setContainersUtilization(ResourceUtilization utilization) {
+    this.containersUtilization = utilization;
+  }
+
+  @Override
   public void handle(ContainersMonitorEvent monitoringEvent) {
 
     if (!isEnabled()) {


[11/50] hadoop git commit: Revert "HADOOP-11807. add a lint mode to releasedocmaker (ramtin via aw)"

Posted by ar...@apache.org.
Revert "HADOOP-11807. add a lint mode to releasedocmaker (ramtin via aw)"

This reverts commit 79ed0f959ffc490414ca56a73e026500c24e7078.

Conflicts:
	hadoop-common-project/hadoop-common/CHANGES.txt


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

Branch: refs/heads/HDFS-7240
Commit: 98e59261b7fb6f9d6d10a5c897f62222da7d2a1f
Parents: bd4e109
Author: Allen Wittenauer <aw...@apache.org>
Authored: Wed Jul 8 08:06:30 2015 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Wed Jul 8 08:06:30 2015 -0700

----------------------------------------------------------------------
 dev-support/releasedocmaker.py                  | 76 +++-----------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  2 -
 2 files changed, 10 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/98e59261/dev-support/releasedocmaker.py
----------------------------------------------------------------------
diff --git a/dev-support/releasedocmaker.py b/dev-support/releasedocmaker.py
index 8e68b3c..2ccc1c0 100755
--- a/dev-support/releasedocmaker.py
+++ b/dev-support/releasedocmaker.py
@@ -87,15 +87,8 @@ def notableclean(str):
   str=str.rstrip()
   return str
 
-# clean output dir
-def cleanOutputDir(dir):
-    files = os.listdir(dir)
-    for name in files:
-        os.remove(os.path.join(dir,name))
-    os.rmdir(dir)
-
 def mstr(obj):
-  if (obj is None):
+  if (obj == None):
     return ""
   return unicode(obj)
 
@@ -155,7 +148,7 @@ class Jira:
     return mstr(self.fields['description'])
 
   def getReleaseNote(self):
-    if (self.notes is None):
+    if (self.notes == None):
       field = self.parent.fieldIdMap['Release Note']
       if (self.fields.has_key(field)):
         self.notes=mstr(self.fields[field])
@@ -166,14 +159,14 @@ class Jira:
   def getPriority(self):
     ret = ""
     pri = self.fields['priority']
-    if(pri is not None):
+    if(pri != None):
       ret = pri['name']
     return mstr(ret)
 
   def getAssignee(self):
     ret = ""
     mid = self.fields['assignee']
-    if(mid is not None):
+    if(mid != None):
       ret = mid['displayName']
     return mstr(ret)
 
@@ -189,21 +182,21 @@ class Jira:
   def getType(self):
     ret = ""
     mid = self.fields['issuetype']
-    if(mid is not None):
+    if(mid != None):
       ret = mid['name']
     return mstr(ret)
 
   def getReporter(self):
     ret = ""
     mid = self.fields['reporter']
-    if(mid is not None):
+    if(mid != None):
       ret = mid['displayName']
     return mstr(ret)
 
   def getProject(self):
     ret = ""
     mid = self.fields['project']
-    if(mid is not None):
+    if(mid != None):
       ret = mid['key']
     return mstr(ret)
 
@@ -221,7 +214,7 @@ class Jira:
     return False
 
   def getIncompatibleChange(self):
-    if (self.incompat is None):
+    if (self.incompat == None):
       field = self.parent.fieldIdMap['Hadoop Flags']
       self.reviewed=False
       self.incompat=False
@@ -234,24 +227,6 @@ class Jira:
               self.reviewed=True
     return self.incompat
 
-  def checkMissingComponent(self):
-      if (len(self.fields['components'])>0):
-          return False
-      return True
-
-  def checkMissingAssignee(self):
-      if (self.fields['assignee'] is not None):
-          return False
-      return True
-
-  def checkVersionString(self):
-      field = self.parent.fieldIdMap['Fix Version/s']
-      for h in self.fields[field]:
-          found = re.match('^((\d+)(\.\d+)*).*$|^(\w+\-\d+)$', h['name'])
-          if not found:
-              return True
-      return False
-
   def getReleaseDate(self,version):
     for j in range(len(self.fields['fixVersions'])):
       if self.fields['fixVersions'][j]==version:
@@ -364,11 +339,9 @@ def main():
              help="build an index file")
   parser.add_option("-u","--usetoday", dest="usetoday", action="store_true",
              help="use current date for unreleased versions")
-  parser.add_option("-n","--lint", dest="lint", action="store_true",
-             help="use lint flag to exit on failures")
   (options, args) = parser.parse_args()
 
-  if (options.versions is None):
+  if (options.versions == None):
     options.versions = []
 
   if (len(args) > 2):
@@ -423,9 +396,6 @@ def main():
   reloutputs.writeAll(relhead)
   choutputs.writeAll(chhead)
 
-  errorCount=0
-  warningCount=0
-  lintMessage=""
   incompatlist=[]
   buglist=[]
   improvementlist=[]
@@ -438,14 +408,6 @@ def main():
   for jira in sorted(jlist):
     if jira.getIncompatibleChange():
       incompatlist.append(jira)
-      if (len(jira.getReleaseNote())==0):
-          warningCount+=1
-
-    if jira.checkVersionString():
-       warningCount+=1
-
-    if jira.checkMissingComponent() or jira.checkMissingAssignee():
-      errorCount+=1
     elif jira.getType() == "Bug":
       buglist.append(jira)
     elif jira.getType() == "Improvement":
@@ -469,33 +431,15 @@ def main():
       reloutputs.writeKeyRaw(jira.getProject(),"\n---\n\n")
       reloutputs.writeKeyRaw(jira.getProject(), line)
       line ='\n**WARNING: No release note provided for this incompatible change.**\n\n'
-      lintMessage += "\nWARNING: incompatible change %s lacks release notes." % (notableclean(jira.getId()))
+      print 'WARNING: incompatible change %s lacks release notes.' % (notableclean(jira.getId()))
       reloutputs.writeKeyRaw(jira.getProject(), line)
 
-    if jira.checkVersionString():
-        lintMessage += "\nWARNING: Version string problem for %s " % jira.getId()
-
-    if (jira.checkMissingComponent() or jira.checkMissingAssignee()):
-        errorMessage=[]
-        jira.checkMissingComponent() and errorMessage.append("component")
-        jira.checkMissingAssignee() and errorMessage.append("assignee")
-        lintMessage += "\nERROR: missing %s for %s " %  (" and ".join(errorMessage) , jira.getId())
-
     if (len(jira.getReleaseNote())>0):
       reloutputs.writeKeyRaw(jira.getProject(),"\n---\n\n")
       reloutputs.writeKeyRaw(jira.getProject(), line)
       line ='\n%s\n\n' % (tableclean(jira.getReleaseNote()))
       reloutputs.writeKeyRaw(jira.getProject(), line)
 
-  if (options.lint is True):
-      print lintMessage
-      print "======================================="
-      print "Error:%d, Warning:%d \n" % (errorCount, warningCount)
-
-      if (errorCount>0):
-          cleanOutputDir(version)
-          sys.exit(1)
-
   reloutputs.writeAll("\n\n")
   reloutputs.close()
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98e59261/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 859e58a..8ab109d 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -230,8 +230,6 @@ Trunk (Unreleased)
     HADOOP-11142. Remove hdfs dfs reference from file system shell
     documentation (Kengo Seki via aw)
 
-    HADOOP-11807. add a lint mode to releasedocmaker (ramtin via aw)
-
     HADOOP-12149. copy all of test-patch BINDIR prior to re-exec (aw)
 
   BUG FIXES


[14/50] hadoop git commit: HDFS-8726. Move protobuf files that define the client-sever protocols to hdfs-client. Contributed by Haohui Mai.

Posted by ar...@apache.org.
HDFS-8726. Move protobuf files that define the client-sever protocols to hdfs-client. Contributed by Haohui Mai.


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

Branch: refs/heads/HDFS-7240
Commit: fc6182d5ed92ac70de1f4633edd5265b7be1a8dc
Parents: 4119ad3
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Jul 8 10:37:10 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Wed Jul 8 10:37:10 2015 -0700

----------------------------------------------------------------------
 .../dev-support/findbugsExcludeFile.xml         |   4 +
 hadoop-hdfs-project/hadoop-hdfs-client/pom.xml  |  42 +
 .../src/main/proto/ClientDatanodeProtocol.proto | 247 ++++++
 .../src/main/proto/ClientNamenodeProtocol.proto | 863 +++++++++++++++++++
 .../hadoop-hdfs-client/src/main/proto/acl.proto | 108 +++
 .../src/main/proto/datatransfer.proto           | 304 +++++++
 .../src/main/proto/encryption.proto             |  67 ++
 .../src/main/proto/hdfs.proto                   | 611 +++++++++++++
 .../src/main/proto/inotify.proto                | 126 +++
 .../src/main/proto/xattr.proto                  |  75 ++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |  10 +-
 .../hadoop-hdfs/src/contrib/bkjournal/pom.xml   |   2 +-
 .../hdfs/server/namenode/FSEditLogOp.java       |   4 +-
 .../src/main/proto/ClientDatanodeProtocol.proto | 247 ------
 .../src/main/proto/ClientNamenodeProtocol.proto | 863 -------------------
 .../hadoop-hdfs/src/main/proto/acl.proto        | 113 ---
 .../src/main/proto/datatransfer.proto           | 304 -------
 .../hadoop-hdfs/src/main/proto/editlog.proto    |  35 +
 .../hadoop-hdfs/src/main/proto/encryption.proto |  67 --
 .../hadoop-hdfs/src/main/proto/hdfs.proto       | 611 -------------
 .../hadoop-hdfs/src/main/proto/inotify.proto    | 126 ---
 .../hadoop-hdfs/src/main/proto/xattr.proto      |  80 --
 23 files changed, 2490 insertions(+), 2422 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/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 be2911f..ba6453d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
@@ -17,4 +17,8 @@
     </Or>
     <Bug pattern="EI_EXPOSE_REP,EI_EXPOSE_REP2" />
   </Match>
+  <Match>
+    <Package name="org.apache.hadoop.hdfs.protocol.proto" />
+    <Bug pattern="SE_BAD_FIELD,MS_SHOULD_BE_FINAL,UCF_USELESS_CONTROL_FLOW" />
+  </Match>
 </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/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 1b45095..aeaa980 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
@@ -63,6 +63,48 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
           </excludes>
         </configuration>
       </plugin>
+      <plugin>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-maven-plugins</artifactId>
+        <executions>
+          <execution>
+            <id>compile-protoc</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>protoc</goal>
+            </goals>
+            <configuration>
+              <protocVersion>${protobuf.version}</protocVersion>
+              <protocCommand>${protoc.path}</protocCommand>
+              <imports>
+                <param>${basedir}/../../hadoop-common-project/hadoop-common/src/main/proto</param>
+                <param>${basedir}/src/main/proto</param>
+              </imports>
+              <source>
+                <directory>${basedir}/src/main/proto</directory>
+                <includes>
+                  <include>ClientDatanodeProtocol.proto</include>
+                  <include>ClientNamenodeProtocol.proto</include>
+                  <include>acl.proto</include>
+                  <include>xattr.proto</include>
+                  <include>datatransfer.proto</include>
+                  <include>hdfs.proto</include>
+                  <include>encryption.proto</include>
+                  <include>inotify.proto</include>
+                </includes>
+              </source>
+              <output>${project.build.directory}/generated-sources/java</output>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-javadoc-plugin</artifactId>
+        <configuration>
+          <excludePackageNames>org.apache.hadoop.hdfs.protocol.proto</excludePackageNames>
+        </configuration>
+      </plugin>
     </plugins>
   </build>
 </project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
new file mode 100644
index 0000000..e0d1f5f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
@@ -0,0 +1,247 @@
+/**
+ * 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.
+ */
+
+/**
+ * These .proto interfaces are private and stable.
+ * Please see http://wiki.apache.org/hadoop/Compatibility
+ * for what changes are allowed for a *stable* .proto interface.
+ */
+
+// This file contains protocol buffers that are used throughout HDFS -- i.e.
+// by the client, server, and data transfer protocols.
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "ClientDatanodeProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
+
+import "Security.proto";
+import "hdfs.proto";
+
+/**
+ * block - block for which visible length is requested
+ */
+message GetReplicaVisibleLengthRequestProto {
+  required ExtendedBlockProto block = 1;
+}
+
+/**
+ * length - visible length of the block
+ */
+message GetReplicaVisibleLengthResponseProto {
+  required uint64 length = 1;
+}
+
+/**
+ * void request
+ */
+message RefreshNamenodesRequestProto {
+}
+
+/**
+ * void response
+ */
+message RefreshNamenodesResponseProto {
+}
+
+/**
+ * blockPool - block pool to be deleted
+ * force - if false, delete the block pool only if it is empty.
+ *         if true, delete the block pool even if it has blocks.
+ */
+message DeleteBlockPoolRequestProto {
+  required string blockPool = 1;
+  required bool force = 2;
+}
+
+/**
+ * void response
+ */
+message DeleteBlockPoolResponseProto {
+}
+
+/**
+ * Gets the file information where block and its metadata is stored
+ * block - block for which path information is being requested
+ * token - block token
+ *
+ * This message is deprecated in favor of file descriptor passing.
+ */
+message GetBlockLocalPathInfoRequestProto {
+  required ExtendedBlockProto block = 1;
+  required hadoop.common.TokenProto token = 2;
+}
+
+/**
+ * block - block for which file path information is being returned
+ * localPath - file path where the block data is stored
+ * localMetaPath - file path where the block meta data is stored
+ *
+ * This message is deprecated in favor of file descriptor passing.
+ */
+message GetBlockLocalPathInfoResponseProto {
+  required ExtendedBlockProto block = 1;
+  required string localPath = 2;
+  required string localMetaPath = 3;
+}
+
+/**
+ * Query for the disk locations of a number of blocks on this DN.
+ * blockPoolId - the pool to query
+ * blockIds - list of block IDs to query
+ * tokens - list of access tokens corresponding to list of block IDs
+ */
+message GetHdfsBlockLocationsRequestProto {
+  // Removed: HDFS-3969
+  // repeated ExtendedBlockProto blocks = 1;
+  repeated hadoop.common.TokenProto tokens = 2;
+
+  required string blockPoolId = 3;
+  repeated sfixed64 blockIds = 4 [ packed = true ];
+}
+
+/**
+ * volumeIds - id of each volume, potentially multiple bytes
+ * volumeIndexes - for each block, an index into volumeIds specifying the volume
+ *               on which it is located. If block is not present on any volume,
+ *               index is set to MAX_INT.
+ */
+message GetHdfsBlockLocationsResponseProto {
+  repeated bytes volumeIds = 1;
+  repeated uint32 volumeIndexes = 2 [ packed = true ];
+}
+
+/**
+ * forUpgrade - if true, clients are advised to wait for restart and quick
+ *              upgrade restart is instrumented. Otherwise, datanode does
+ *              the regular shutdown.
+ */
+message ShutdownDatanodeRequestProto {
+  required bool forUpgrade = 1;
+}
+
+message ShutdownDatanodeResponseProto {
+}
+
+/**
+ * Ping datanode for liveness and quick info
+ */
+message GetDatanodeInfoRequestProto {
+}
+
+message GetDatanodeInfoResponseProto {
+  required DatanodeLocalInfoProto localInfo = 1;
+}
+
+/** Asks DataNode to reload configuration file. */
+message StartReconfigurationRequestProto {
+}
+
+message StartReconfigurationResponseProto {
+}
+
+message TriggerBlockReportRequestProto {
+  required bool incremental = 1;
+}
+
+message TriggerBlockReportResponseProto {
+}
+
+/** Query the running status of reconfiguration process */
+message GetReconfigurationStatusRequestProto {
+}
+
+message GetReconfigurationStatusConfigChangeProto {
+  required string name = 1;
+  required string oldValue = 2;
+  optional string newValue = 3;
+  optional string errorMessage = 4;  // It is empty if success.
+}
+
+message GetReconfigurationStatusResponseProto {
+  required int64 startTime = 1;
+  optional int64 endTime = 2;
+  repeated GetReconfigurationStatusConfigChangeProto changes = 3;
+}
+
+message ListReconfigurablePropertiesRequestProto {
+}
+
+/** Query the reconfigurable properties on DataNode. */
+message ListReconfigurablePropertiesResponseProto {
+  repeated string name = 1;
+}
+
+/**
+ * Protocol used from client to the Datanode.
+ * See the request and response for details of rpc call.
+ */
+service ClientDatanodeProtocolService {
+  /**
+   * Returns the visible length of the replica
+   */
+  rpc getReplicaVisibleLength(GetReplicaVisibleLengthRequestProto)
+      returns(GetReplicaVisibleLengthResponseProto);
+
+  /**
+   * Refresh the list of federated namenodes from updated configuration.
+   * Adds new namenodes and stops the deleted namenodes.
+   */
+  rpc refreshNamenodes(RefreshNamenodesRequestProto)
+      returns(RefreshNamenodesResponseProto);
+
+  /**
+   * Delete the block pool from the datanode.
+   */
+  rpc deleteBlockPool(DeleteBlockPoolRequestProto)
+      returns(DeleteBlockPoolResponseProto);
+
+  /**
+   * Retrieves the path names of the block file and metadata file stored on the
+   * local file system.
+   */
+  rpc getBlockLocalPathInfo(GetBlockLocalPathInfoRequestProto)
+      returns(GetBlockLocalPathInfoResponseProto);
+
+  /**
+   * Retrieve additional HDFS-specific metadata about a set of blocks stored
+   * on the local file system.
+   */
+  rpc getHdfsBlockLocations(GetHdfsBlockLocationsRequestProto)
+      returns(GetHdfsBlockLocationsResponseProto);
+
+  rpc shutdownDatanode(ShutdownDatanodeRequestProto)
+      returns(ShutdownDatanodeResponseProto);
+
+  rpc getDatanodeInfo(GetDatanodeInfoRequestProto)
+      returns(GetDatanodeInfoResponseProto);
+
+  rpc getReconfigurationStatus(GetReconfigurationStatusRequestProto)
+      returns(GetReconfigurationStatusResponseProto);
+
+  rpc startReconfiguration(StartReconfigurationRequestProto)
+      returns(StartReconfigurationResponseProto);
+
+  rpc listReconfigurableProperties(
+      ListReconfigurablePropertiesRequestProto)
+      returns(ListReconfigurablePropertiesResponseProto);
+
+  rpc triggerBlockReport(TriggerBlockReportRequestProto)
+      returns(TriggerBlockReportResponseProto);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
new file mode 100644
index 0000000..b44c556
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
@@ -0,0 +1,863 @@
+/**
+ * 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.
+ */
+
+/**
+ * These .proto interfaces are private and stable.
+ * Please see http://wiki.apache.org/hadoop/Compatibility
+ * for what changes are allowed for a *stable* .proto interface.
+ */
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "ClientNamenodeProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
+
+import "Security.proto";
+import "hdfs.proto";
+import "acl.proto";
+import "xattr.proto";
+import "encryption.proto";
+import "inotify.proto";
+
+/**
+ * The ClientNamenodeProtocol Service defines the interface between a client 
+ * (as runnign inside a MR Task) and the Namenode.
+ * See org.apache.hadoop.hdfs.protocol.ClientProtocol for the javadoc 
+ * for each of the methods.
+ * The exceptions declared in the above class also apply to this protocol.
+ * Exceptions are unwrapped and thrown by the  PB libraries.
+ */
+
+message GetBlockLocationsRequestProto {
+  required string src = 1;     // file name
+  required uint64 offset = 2;  // range start offset
+  required uint64 length = 3;  // range length
+}
+
+message GetBlockLocationsResponseProto {
+  optional LocatedBlocksProto locations = 1;
+}
+
+message GetServerDefaultsRequestProto { // No parameters
+}
+
+message GetServerDefaultsResponseProto {
+  required FsServerDefaultsProto serverDefaults = 1;
+}
+
+enum CreateFlagProto {
+  CREATE = 0x01;    // Create a file
+  OVERWRITE = 0x02; // Truncate/overwrite a file. Same as POSIX O_TRUNC
+  APPEND = 0x04;    // Append to a file
+  LAZY_PERSIST = 0x10; // File with reduced durability guarantees.
+  NEW_BLOCK = 0x20; // Write data to a new block when appending
+}
+
+message CreateRequestProto {
+  required string src = 1;
+  required FsPermissionProto masked = 2;
+  required string clientName = 3;
+  required uint32 createFlag = 4;  // bits set using CreateFlag
+  required bool createParent = 5;
+  required uint32 replication = 6; // Short: Only 16 bits used
+  required uint64 blockSize = 7;
+  repeated CryptoProtocolVersionProto cryptoProtocolVersion = 8;
+}
+
+message CreateResponseProto {
+  optional HdfsFileStatusProto fs = 1;
+}
+
+message AppendRequestProto {
+  required string src = 1;
+  required string clientName = 2;
+  optional uint32 flag = 3; // bits set using CreateFlag
+}
+
+message AppendResponseProto {
+  optional LocatedBlockProto block = 1;
+  optional HdfsFileStatusProto stat = 2;
+}
+
+message SetReplicationRequestProto {
+  required string src = 1;
+  required uint32 replication = 2; // Short: Only 16 bits used
+}
+
+message SetReplicationResponseProto {
+  required bool result = 1;
+}
+
+message SetStoragePolicyRequestProto {
+  required string src = 1;
+  required string policyName = 2;
+}
+
+message SetStoragePolicyResponseProto { // void response
+}
+
+message GetStoragePoliciesRequestProto { // void request
+}
+
+message GetStoragePoliciesResponseProto {
+  repeated BlockStoragePolicyProto policies = 1;
+}
+
+message SetPermissionRequestProto {
+  required string src = 1;
+  required FsPermissionProto permission = 2;
+}
+
+message SetPermissionResponseProto { // void response
+}
+
+message SetOwnerRequestProto {
+  required string src = 1;
+  optional string username = 2;
+  optional string groupname = 3;
+}
+
+message SetOwnerResponseProto { // void response
+}
+
+message AbandonBlockRequestProto {
+  required ExtendedBlockProto b = 1;
+  required string src = 2;
+  required string holder = 3;
+  optional uint64 fileId = 4 [default = 0];  // default to GRANDFATHER_INODE_ID
+}
+
+message AbandonBlockResponseProto { // void response
+}
+
+message AddBlockRequestProto {
+  required string src = 1;
+  required string clientName = 2;
+  optional ExtendedBlockProto previous = 3;
+  repeated DatanodeInfoProto excludeNodes = 4;
+  optional uint64 fileId = 5 [default = 0];  // default as a bogus id
+  repeated string favoredNodes = 6; //the set of datanodes to use for the block
+}
+
+message AddBlockResponseProto {
+  required LocatedBlockProto block = 1;
+}
+
+message GetAdditionalDatanodeRequestProto {
+  required string src = 1;
+  required ExtendedBlockProto blk = 2;
+  repeated DatanodeInfoProto existings = 3;
+  repeated DatanodeInfoProto excludes = 4;
+  required uint32 numAdditionalNodes = 5;
+  required string clientName = 6;
+  repeated string existingStorageUuids = 7;
+  optional uint64 fileId = 8 [default = 0];  // default to GRANDFATHER_INODE_ID
+}
+
+message GetAdditionalDatanodeResponseProto {
+  required LocatedBlockProto block = 1;
+}
+
+message CompleteRequestProto {
+  required string src = 1;
+  required string clientName = 2;
+  optional ExtendedBlockProto last = 3;
+  optional uint64 fileId = 4 [default = 0];  // default to GRANDFATHER_INODE_ID
+}
+
+message CompleteResponseProto {
+  required bool result = 1;
+}
+
+message ReportBadBlocksRequestProto {
+  repeated LocatedBlockProto blocks = 1;
+}
+
+message ReportBadBlocksResponseProto { // void response
+}
+
+message ConcatRequestProto {
+  required string trg = 1;
+  repeated string srcs = 2;
+}
+
+message ConcatResponseProto { // void response
+}
+
+message TruncateRequestProto {
+  required string src = 1;
+  required uint64 newLength = 2;
+  required string clientName = 3;
+}
+
+message TruncateResponseProto {
+  required bool result = 1;
+}
+
+message RenameRequestProto {
+  required string src = 1;
+  required string dst = 2;
+}
+
+message RenameResponseProto {
+  required bool result = 1;
+}
+
+
+message Rename2RequestProto {
+  required string src = 1;
+  required string dst = 2;
+  required bool overwriteDest = 3;
+}
+
+message Rename2ResponseProto { // void response
+}
+
+message DeleteRequestProto {
+  required string src = 1;
+  required bool recursive = 2;
+}
+
+message DeleteResponseProto {
+    required bool result = 1;
+}
+
+message MkdirsRequestProto {
+  required string src = 1;
+  required FsPermissionProto masked = 2;
+  required bool createParent = 3;
+}
+message MkdirsResponseProto {
+    required bool result = 1;
+}
+
+message GetListingRequestProto {
+  required string src = 1;
+  required bytes startAfter = 2;
+  required bool needLocation = 3;
+}
+message GetListingResponseProto {
+  optional DirectoryListingProto dirList = 1;
+}
+
+message GetSnapshottableDirListingRequestProto { // no input parameters
+}
+message GetSnapshottableDirListingResponseProto {
+  optional SnapshottableDirectoryListingProto snapshottableDirList = 1;
+}
+
+message GetSnapshotDiffReportRequestProto {
+  required string snapshotRoot = 1;
+  required string fromSnapshot = 2;
+  required string toSnapshot = 3;
+}
+message GetSnapshotDiffReportResponseProto {
+  required SnapshotDiffReportProto diffReport = 1;
+}
+
+message RenewLeaseRequestProto {
+  required string clientName = 1;
+}
+
+message RenewLeaseResponseProto { //void response
+}
+
+message RecoverLeaseRequestProto {
+  required string src = 1;
+  required string clientName = 2;
+}
+message RecoverLeaseResponseProto {
+  required bool result = 1;
+}
+
+message GetFsStatusRequestProto { // no input paramters
+}
+
+message GetFsStatsResponseProto {
+  required uint64 capacity = 1;
+  required uint64 used = 2;
+  required uint64 remaining = 3;
+  required uint64 under_replicated = 4;
+  required uint64 corrupt_blocks = 5;
+  required uint64 missing_blocks = 6;
+  optional uint64 missing_repl_one_blocks = 7;
+}
+
+enum DatanodeReportTypeProto {  // type of the datanode report
+  ALL = 1;
+  LIVE = 2;
+  DEAD = 3;
+  DECOMMISSIONING = 4;
+}
+
+message GetDatanodeReportRequestProto {
+  required DatanodeReportTypeProto type = 1;
+}
+
+message GetDatanodeReportResponseProto {
+  repeated DatanodeInfoProto di = 1;
+}
+
+message GetDatanodeStorageReportRequestProto {
+  required DatanodeReportTypeProto type = 1;
+}
+
+message DatanodeStorageReportProto {
+  required DatanodeInfoProto datanodeInfo = 1;
+  repeated StorageReportProto storageReports = 2;
+}
+
+message GetDatanodeStorageReportResponseProto {
+  repeated DatanodeStorageReportProto datanodeStorageReports = 1;
+}
+
+message GetPreferredBlockSizeRequestProto {
+  required string filename = 1;
+}
+
+message GetPreferredBlockSizeResponseProto {
+  required uint64 bsize = 1;
+}
+
+enum SafeModeActionProto {
+  SAFEMODE_LEAVE = 1;
+  SAFEMODE_ENTER = 2;
+  SAFEMODE_GET = 3;
+}
+
+message SetSafeModeRequestProto {
+  required SafeModeActionProto action = 1;
+  optional bool checked = 2 [default = false];
+}
+
+message SetSafeModeResponseProto {
+  required bool result = 1;
+}
+
+message SaveNamespaceRequestProto {
+  optional uint64 timeWindow = 1 [default = 0];
+  optional uint64 txGap = 2 [default = 0];
+}
+
+message SaveNamespaceResponseProto { // void response
+  optional bool saved = 1 [default = true];
+}
+
+message RollEditsRequestProto { // no parameters
+}
+
+message RollEditsResponseProto { // response
+  required uint64 newSegmentTxId = 1;
+}
+
+message RestoreFailedStorageRequestProto {
+  required string arg = 1;
+}
+
+message RestoreFailedStorageResponseProto {
+    required bool result = 1;
+}
+
+message RefreshNodesRequestProto { // no parameters
+}
+
+message RefreshNodesResponseProto { // void response
+}
+
+message FinalizeUpgradeRequestProto { // no parameters
+}
+
+message FinalizeUpgradeResponseProto { // void response
+}
+
+enum RollingUpgradeActionProto {
+  QUERY = 1;
+  START = 2;
+  FINALIZE = 3;
+}
+
+message RollingUpgradeRequestProto {
+  required RollingUpgradeActionProto action = 1;
+}
+
+message RollingUpgradeInfoProto {
+  required RollingUpgradeStatusProto status = 1;
+  required uint64 startTime = 2;
+  required uint64 finalizeTime = 3;
+  required bool createdRollbackImages = 4;
+}
+
+message RollingUpgradeResponseProto {
+  optional RollingUpgradeInfoProto rollingUpgradeInfo= 1;
+}
+
+message ListCorruptFileBlocksRequestProto {
+  required string path = 1;
+  optional string cookie = 2;
+}
+
+message ListCorruptFileBlocksResponseProto {
+  required CorruptFileBlocksProto corrupt = 1;
+}
+
+message MetaSaveRequestProto {
+  required string filename = 1;
+}
+
+message MetaSaveResponseProto { // void response
+}
+
+message GetFileInfoRequestProto {
+  required string src = 1;
+}
+
+message GetFileInfoResponseProto {
+  optional HdfsFileStatusProto fs = 1;
+}
+
+message IsFileClosedRequestProto {
+  required string src = 1;
+}
+
+message IsFileClosedResponseProto {
+  required bool result = 1;
+}
+
+message CacheDirectiveInfoProto {
+  optional int64 id = 1;
+  optional string path = 2;
+  optional uint32 replication = 3;
+  optional string pool = 4;
+  optional CacheDirectiveInfoExpirationProto expiration = 5;
+}
+
+message CacheDirectiveInfoExpirationProto {
+  required int64 millis = 1;
+  required bool isRelative = 2;
+}
+
+message CacheDirectiveStatsProto {
+  required int64 bytesNeeded = 1;
+  required int64 bytesCached = 2;
+  required int64 filesNeeded = 3;
+  required int64 filesCached = 4;
+  required bool hasExpired = 5;
+}
+
+enum CacheFlagProto {
+  FORCE = 0x01;    // Ignore pool resource limits
+}
+
+message AddCacheDirectiveRequestProto {
+  required CacheDirectiveInfoProto info = 1;
+  optional uint32 cacheFlags = 2;  // bits set using CacheFlag
+}
+
+message AddCacheDirectiveResponseProto {
+  required int64 id = 1;
+}
+
+message ModifyCacheDirectiveRequestProto {
+  required CacheDirectiveInfoProto info = 1;
+  optional uint32 cacheFlags = 2;  // bits set using CacheFlag
+}
+
+message ModifyCacheDirectiveResponseProto {
+}
+
+message RemoveCacheDirectiveRequestProto {
+  required int64 id = 1;
+}
+
+message RemoveCacheDirectiveResponseProto {
+}
+
+message ListCacheDirectivesRequestProto {
+  required int64 prevId = 1;
+  required CacheDirectiveInfoProto filter = 2;
+}
+
+message CacheDirectiveEntryProto {
+  required CacheDirectiveInfoProto info = 1;
+  required CacheDirectiveStatsProto stats = 2;
+}
+
+message ListCacheDirectivesResponseProto {
+  repeated CacheDirectiveEntryProto elements = 1;
+  required bool hasMore = 2;
+}
+
+message CachePoolInfoProto {
+  optional string poolName = 1;
+  optional string ownerName = 2;
+  optional string groupName = 3;
+  optional int32 mode = 4;
+  optional int64 limit = 5;
+  optional int64 maxRelativeExpiry = 6;
+}
+
+message CachePoolStatsProto {
+  required int64 bytesNeeded = 1;
+  required int64 bytesCached = 2;
+  required int64 bytesOverlimit = 3;
+  required int64 filesNeeded = 4;
+  required int64 filesCached = 5;
+}
+
+message AddCachePoolRequestProto {
+  required CachePoolInfoProto info = 1;
+}
+
+message AddCachePoolResponseProto { // void response
+}
+
+message ModifyCachePoolRequestProto {
+  required CachePoolInfoProto info = 1;
+}
+
+message ModifyCachePoolResponseProto { // void response
+}
+
+message RemoveCachePoolRequestProto {
+  required string poolName = 1;
+}
+
+message RemoveCachePoolResponseProto { // void response
+}
+
+message ListCachePoolsRequestProto {
+  required string prevPoolName = 1;
+}
+
+message ListCachePoolsResponseProto {
+  repeated CachePoolEntryProto entries = 1;
+  required bool hasMore = 2;
+}
+
+message CachePoolEntryProto {
+  required CachePoolInfoProto info = 1;
+  required CachePoolStatsProto stats = 2;
+}
+
+message GetFileLinkInfoRequestProto {
+  required string src = 1;
+}
+
+message GetFileLinkInfoResponseProto {
+  optional HdfsFileStatusProto fs = 1;
+}
+
+message GetContentSummaryRequestProto {
+  required string path = 1;
+}
+
+message GetContentSummaryResponseProto {
+  required ContentSummaryProto summary = 1;
+}
+
+message SetQuotaRequestProto {
+  required string path = 1;
+  required uint64 namespaceQuota = 2;
+  required uint64 storagespaceQuota = 3;
+  optional StorageTypeProto storageType = 4;
+}
+
+message SetQuotaResponseProto { // void response
+}
+
+message FsyncRequestProto {
+  required string src = 1;
+  required string client = 2;
+  optional sint64 lastBlockLength = 3 [default = -1];
+  optional uint64 fileId = 4 [default = 0];  // default to GRANDFATHER_INODE_ID
+}
+
+message FsyncResponseProto { // void response
+}
+
+message SetTimesRequestProto {
+  required string src = 1;
+  required uint64 mtime = 2;
+  required uint64 atime = 3;
+}
+
+message SetTimesResponseProto { // void response
+}
+
+message CreateSymlinkRequestProto {
+  required string target = 1;
+  required string link = 2;
+  required FsPermissionProto dirPerm = 3;
+  required bool createParent = 4;
+}
+
+message CreateSymlinkResponseProto { // void response
+}
+
+message GetLinkTargetRequestProto {
+  required string path = 1;
+}
+message GetLinkTargetResponseProto {
+  optional string targetPath = 1;
+}
+
+message UpdateBlockForPipelineRequestProto {
+  required ExtendedBlockProto block = 1;
+  required string clientName = 2;
+}
+
+message UpdateBlockForPipelineResponseProto {
+  required LocatedBlockProto block = 1;
+}
+
+message UpdatePipelineRequestProto {
+  required string clientName = 1;
+  required ExtendedBlockProto oldBlock = 2;
+  required ExtendedBlockProto newBlock = 3;
+  repeated DatanodeIDProto newNodes = 4;
+  repeated string storageIDs = 5;
+}
+
+message UpdatePipelineResponseProto { // void response
+}
+
+message SetBalancerBandwidthRequestProto {
+  required int64 bandwidth = 1;
+}
+
+message SetBalancerBandwidthResponseProto { // void response
+}
+
+message GetDataEncryptionKeyRequestProto { // no parameters
+}
+
+message GetDataEncryptionKeyResponseProto {
+  optional DataEncryptionKeyProto dataEncryptionKey = 1;
+}
+
+message CreateSnapshotRequestProto {
+  required string snapshotRoot = 1;
+  optional string snapshotName = 2;
+}
+
+message CreateSnapshotResponseProto {
+  required string snapshotPath = 1;
+}
+
+message RenameSnapshotRequestProto {
+  required string snapshotRoot = 1;
+  required string snapshotOldName = 2;
+  required string snapshotNewName = 3;
+}
+
+message RenameSnapshotResponseProto { // void response
+}
+
+message AllowSnapshotRequestProto {
+  required string snapshotRoot = 1;
+}
+
+message AllowSnapshotResponseProto {
+}
+
+message DisallowSnapshotRequestProto {
+  required string snapshotRoot = 1;
+}
+
+message DisallowSnapshotResponseProto {
+}
+
+message DeleteSnapshotRequestProto {
+  required string snapshotRoot = 1;
+  required string snapshotName = 2;
+}
+
+message DeleteSnapshotResponseProto { // void response
+}
+
+message CheckAccessRequestProto {
+  required string path = 1;
+  required AclEntryProto.FsActionProto mode = 2;
+}
+
+message CheckAccessResponseProto { // void response
+}
+
+message GetCurrentEditLogTxidRequestProto {
+}
+
+message GetCurrentEditLogTxidResponseProto {
+  required int64 txid = 1;
+}
+
+message GetEditsFromTxidRequestProto {
+  required int64 txid = 1;
+}
+
+message GetEditsFromTxidResponseProto {
+  required EventsListProto eventsList = 1;
+}
+
+service ClientNamenodeProtocol {
+  rpc getBlockLocations(GetBlockLocationsRequestProto)
+      returns(GetBlockLocationsResponseProto);
+  rpc getServerDefaults(GetServerDefaultsRequestProto)
+      returns(GetServerDefaultsResponseProto);
+  rpc create(CreateRequestProto)returns(CreateResponseProto);
+  rpc append(AppendRequestProto) returns(AppendResponseProto);
+  rpc setReplication(SetReplicationRequestProto)
+      returns(SetReplicationResponseProto);
+  rpc setStoragePolicy(SetStoragePolicyRequestProto)
+      returns(SetStoragePolicyResponseProto);
+  rpc getStoragePolicies(GetStoragePoliciesRequestProto)
+      returns(GetStoragePoliciesResponseProto);
+  rpc setPermission(SetPermissionRequestProto)
+      returns(SetPermissionResponseProto);
+  rpc setOwner(SetOwnerRequestProto) returns(SetOwnerResponseProto);
+  rpc abandonBlock(AbandonBlockRequestProto) returns(AbandonBlockResponseProto);
+  rpc addBlock(AddBlockRequestProto) returns(AddBlockResponseProto);
+  rpc getAdditionalDatanode(GetAdditionalDatanodeRequestProto)
+      returns(GetAdditionalDatanodeResponseProto);
+  rpc complete(CompleteRequestProto) returns(CompleteResponseProto);
+  rpc reportBadBlocks(ReportBadBlocksRequestProto)
+      returns(ReportBadBlocksResponseProto);
+  rpc concat(ConcatRequestProto) returns(ConcatResponseProto);
+  rpc truncate(TruncateRequestProto) returns(TruncateResponseProto);
+  rpc rename(RenameRequestProto) returns(RenameResponseProto);
+  rpc rename2(Rename2RequestProto) returns(Rename2ResponseProto);
+  rpc delete(DeleteRequestProto) returns(DeleteResponseProto);
+  rpc mkdirs(MkdirsRequestProto) returns(MkdirsResponseProto);
+  rpc getListing(GetListingRequestProto) returns(GetListingResponseProto);
+  rpc renewLease(RenewLeaseRequestProto) returns(RenewLeaseResponseProto);
+  rpc recoverLease(RecoverLeaseRequestProto)
+      returns(RecoverLeaseResponseProto);
+  rpc getFsStats(GetFsStatusRequestProto) returns(GetFsStatsResponseProto);
+  rpc getDatanodeReport(GetDatanodeReportRequestProto)
+      returns(GetDatanodeReportResponseProto);
+  rpc getDatanodeStorageReport(GetDatanodeStorageReportRequestProto)
+      returns(GetDatanodeStorageReportResponseProto);
+  rpc getPreferredBlockSize(GetPreferredBlockSizeRequestProto)
+      returns(GetPreferredBlockSizeResponseProto);
+  rpc setSafeMode(SetSafeModeRequestProto)
+      returns(SetSafeModeResponseProto);
+  rpc saveNamespace(SaveNamespaceRequestProto)
+      returns(SaveNamespaceResponseProto);
+  rpc rollEdits(RollEditsRequestProto)
+      returns(RollEditsResponseProto);
+  rpc restoreFailedStorage(RestoreFailedStorageRequestProto)
+      returns(RestoreFailedStorageResponseProto);
+  rpc refreshNodes(RefreshNodesRequestProto) returns(RefreshNodesResponseProto);
+  rpc finalizeUpgrade(FinalizeUpgradeRequestProto)
+      returns(FinalizeUpgradeResponseProto);
+  rpc rollingUpgrade(RollingUpgradeRequestProto)
+      returns(RollingUpgradeResponseProto);
+  rpc listCorruptFileBlocks(ListCorruptFileBlocksRequestProto)
+      returns(ListCorruptFileBlocksResponseProto);
+  rpc metaSave(MetaSaveRequestProto) returns(MetaSaveResponseProto);
+  rpc getFileInfo(GetFileInfoRequestProto) returns(GetFileInfoResponseProto);
+  rpc addCacheDirective(AddCacheDirectiveRequestProto)
+      returns (AddCacheDirectiveResponseProto);
+  rpc modifyCacheDirective(ModifyCacheDirectiveRequestProto)
+      returns (ModifyCacheDirectiveResponseProto);
+  rpc removeCacheDirective(RemoveCacheDirectiveRequestProto)
+      returns (RemoveCacheDirectiveResponseProto);
+  rpc listCacheDirectives(ListCacheDirectivesRequestProto)
+      returns (ListCacheDirectivesResponseProto);
+  rpc addCachePool(AddCachePoolRequestProto)
+      returns(AddCachePoolResponseProto);
+  rpc modifyCachePool(ModifyCachePoolRequestProto)
+      returns(ModifyCachePoolResponseProto);
+  rpc removeCachePool(RemoveCachePoolRequestProto)
+      returns(RemoveCachePoolResponseProto);
+  rpc listCachePools(ListCachePoolsRequestProto)
+      returns(ListCachePoolsResponseProto);
+  rpc getFileLinkInfo(GetFileLinkInfoRequestProto)
+      returns(GetFileLinkInfoResponseProto);
+  rpc getContentSummary(GetContentSummaryRequestProto)
+      returns(GetContentSummaryResponseProto);
+  rpc setQuota(SetQuotaRequestProto) returns(SetQuotaResponseProto);
+  rpc fsync(FsyncRequestProto) returns(FsyncResponseProto);
+  rpc setTimes(SetTimesRequestProto) returns(SetTimesResponseProto);
+  rpc createSymlink(CreateSymlinkRequestProto)
+      returns(CreateSymlinkResponseProto);
+  rpc getLinkTarget(GetLinkTargetRequestProto)
+      returns(GetLinkTargetResponseProto);
+  rpc updateBlockForPipeline(UpdateBlockForPipelineRequestProto)
+      returns(UpdateBlockForPipelineResponseProto);
+  rpc updatePipeline(UpdatePipelineRequestProto)
+      returns(UpdatePipelineResponseProto);
+  rpc getDelegationToken(hadoop.common.GetDelegationTokenRequestProto)
+      returns(hadoop.common.GetDelegationTokenResponseProto);
+  rpc renewDelegationToken(hadoop.common.RenewDelegationTokenRequestProto)
+      returns(hadoop.common.RenewDelegationTokenResponseProto);
+  rpc cancelDelegationToken(hadoop.common.CancelDelegationTokenRequestProto)
+      returns(hadoop.common.CancelDelegationTokenResponseProto);
+  rpc setBalancerBandwidth(SetBalancerBandwidthRequestProto)
+      returns(SetBalancerBandwidthResponseProto);
+  rpc getDataEncryptionKey(GetDataEncryptionKeyRequestProto)
+      returns(GetDataEncryptionKeyResponseProto);
+  rpc createSnapshot(CreateSnapshotRequestProto)
+      returns(CreateSnapshotResponseProto);
+  rpc renameSnapshot(RenameSnapshotRequestProto)
+      returns(RenameSnapshotResponseProto);
+  rpc allowSnapshot(AllowSnapshotRequestProto)
+      returns(AllowSnapshotResponseProto);
+  rpc disallowSnapshot(DisallowSnapshotRequestProto)
+      returns(DisallowSnapshotResponseProto);   
+  rpc getSnapshottableDirListing(GetSnapshottableDirListingRequestProto)
+      returns(GetSnapshottableDirListingResponseProto);
+  rpc deleteSnapshot(DeleteSnapshotRequestProto)
+      returns(DeleteSnapshotResponseProto);
+  rpc getSnapshotDiffReport(GetSnapshotDiffReportRequestProto)
+      returns(GetSnapshotDiffReportResponseProto);
+  rpc isFileClosed(IsFileClosedRequestProto)
+      returns(IsFileClosedResponseProto);
+  rpc modifyAclEntries(ModifyAclEntriesRequestProto)
+      returns(ModifyAclEntriesResponseProto);
+  rpc removeAclEntries(RemoveAclEntriesRequestProto)
+      returns(RemoveAclEntriesResponseProto);
+  rpc removeDefaultAcl(RemoveDefaultAclRequestProto)
+      returns(RemoveDefaultAclResponseProto);
+  rpc removeAcl(RemoveAclRequestProto)
+      returns(RemoveAclResponseProto);
+  rpc setAcl(SetAclRequestProto)
+      returns(SetAclResponseProto);
+  rpc getAclStatus(GetAclStatusRequestProto)
+      returns(GetAclStatusResponseProto);
+  rpc setXAttr(SetXAttrRequestProto)
+      returns(SetXAttrResponseProto);
+  rpc getXAttrs(GetXAttrsRequestProto)
+      returns(GetXAttrsResponseProto);
+  rpc listXAttrs(ListXAttrsRequestProto)
+      returns(ListXAttrsResponseProto);
+  rpc removeXAttr(RemoveXAttrRequestProto)
+      returns(RemoveXAttrResponseProto);
+  rpc checkAccess(CheckAccessRequestProto)
+      returns(CheckAccessResponseProto);
+  rpc createEncryptionZone(CreateEncryptionZoneRequestProto)
+      returns(CreateEncryptionZoneResponseProto);
+  rpc listEncryptionZones(ListEncryptionZonesRequestProto)
+      returns(ListEncryptionZonesResponseProto);
+  rpc getEZForPath(GetEZForPathRequestProto)
+      returns(GetEZForPathResponseProto);
+  rpc getCurrentEditLogTxid(GetCurrentEditLogTxidRequestProto)
+      returns(GetCurrentEditLogTxidResponseProto);
+  rpc getEditsFromTxid(GetEditsFromTxidRequestProto)
+      returns(GetEditsFromTxidResponseProto);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/acl.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/acl.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/acl.proto
new file mode 100644
index 0000000..bb7fdb0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/acl.proto
@@ -0,0 +1,108 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "AclProtos";
+option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
+
+import "hdfs.proto";
+
+message AclEntryProto {
+  enum AclEntryScopeProto {
+    ACCESS  = 0x0;
+    DEFAULT = 0x1;
+  }
+
+  enum AclEntryTypeProto {
+    USER  = 0x0;
+    GROUP = 0x1;
+    MASK  = 0x2;
+    OTHER = 0x3;
+  }
+
+  enum FsActionProto {
+    NONE          = 0x0;
+    EXECUTE       = 0x1;
+    WRITE         = 0x2;
+    WRITE_EXECUTE = 0x3;
+    READ          = 0x4;
+    READ_EXECUTE  = 0x5;
+    READ_WRITE    = 0x6;
+    PERM_ALL      = 0x7;
+  }
+
+  required AclEntryTypeProto type    = 1;
+  required AclEntryScopeProto scope  = 2;
+  required FsActionProto permissions = 3;
+  optional string name               = 4;
+}
+
+message AclStatusProto {
+  required string owner          = 1;
+  required string group          = 2;
+  required bool sticky           = 3;
+  repeated AclEntryProto entries = 4;
+  optional FsPermissionProto permission = 5;
+}
+
+message ModifyAclEntriesRequestProto {
+  required string src            = 1;
+  repeated AclEntryProto aclSpec = 2;
+}
+
+message ModifyAclEntriesResponseProto {
+}
+
+message RemoveAclRequestProto {
+  required string src = 1;
+}
+
+message RemoveAclResponseProto {
+}
+
+message RemoveAclEntriesRequestProto {
+  required string src            = 1;
+  repeated AclEntryProto aclSpec = 2;
+}
+
+message RemoveAclEntriesResponseProto {
+}
+
+message RemoveDefaultAclRequestProto {
+  required string src = 1;
+}
+
+message RemoveDefaultAclResponseProto {
+}
+
+message SetAclRequestProto {
+  required string src            = 1;
+  repeated AclEntryProto aclSpec = 2;
+}
+
+message SetAclResponseProto {
+}
+
+message GetAclStatusRequestProto {
+  required string src = 1;
+}
+
+message GetAclStatusResponseProto {
+  required AclStatusProto result = 1;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto
new file mode 100644
index 0000000..5071d15
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto
@@ -0,0 +1,304 @@
+/**
+ * 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.
+ */
+
+/**
+ * These .proto interfaces are private and stable.
+ * Please see http://wiki.apache.org/hadoop/Compatibility
+ * for what changes are allowed for a *stable* .proto interface.
+ */
+
+// This file contains protocol buffers that are used to transfer data
+// to and from the datanode, as well as between datanodes.
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "DataTransferProtos";
+option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
+
+import "Security.proto";
+import "hdfs.proto";
+
+message DataTransferEncryptorMessageProto {
+  enum DataTransferEncryptorStatus {
+    SUCCESS = 0;
+    ERROR_UNKNOWN_KEY = 1;
+    ERROR = 2;
+  }
+  required DataTransferEncryptorStatus status = 1;
+  optional bytes payload = 2;
+  optional string message = 3;
+  repeated CipherOptionProto cipherOption = 4;
+}
+
+message BaseHeaderProto {
+  required ExtendedBlockProto block = 1;
+  optional hadoop.common.TokenProto token = 2;
+  optional DataTransferTraceInfoProto traceInfo = 3;
+}
+
+message DataTransferTraceInfoProto {
+  required uint64 traceId = 1;
+  required uint64 parentId = 2;
+}
+
+message ClientOperationHeaderProto {
+  required BaseHeaderProto baseHeader = 1;
+  required string clientName = 2;
+}
+
+message CachingStrategyProto {
+  optional bool dropBehind = 1;
+  optional int64 readahead = 2;
+}
+
+message OpReadBlockProto {
+  required ClientOperationHeaderProto header = 1;
+  required uint64 offset = 2;
+  required uint64 len = 3;
+  optional bool sendChecksums = 4 [default = true];
+  optional CachingStrategyProto cachingStrategy = 5;
+}
+
+
+message ChecksumProto {
+  required ChecksumTypeProto type = 1;
+  required uint32 bytesPerChecksum = 2;
+}
+  
+message OpWriteBlockProto {
+  required ClientOperationHeaderProto header = 1;
+  repeated DatanodeInfoProto targets = 2;
+  optional DatanodeInfoProto source = 3;
+  enum BlockConstructionStage {
+    PIPELINE_SETUP_APPEND = 0;
+    // pipeline set up for failed PIPELINE_SETUP_APPEND recovery
+    PIPELINE_SETUP_APPEND_RECOVERY = 1;
+    // data streaming
+    DATA_STREAMING = 2;
+    // pipeline setup for failed data streaming recovery
+    PIPELINE_SETUP_STREAMING_RECOVERY = 3;
+    // close the block and pipeline
+    PIPELINE_CLOSE = 4;
+    // Recover a failed PIPELINE_CLOSE
+    PIPELINE_CLOSE_RECOVERY = 5;
+    // pipeline set up for block creation
+    PIPELINE_SETUP_CREATE = 6;
+    // transfer RBW for adding datanodes
+    TRANSFER_RBW = 7;
+    // transfer Finalized for adding datanodes
+    TRANSFER_FINALIZED = 8;
+  }
+  required BlockConstructionStage stage = 4;
+  required uint32 pipelineSize = 5;
+  required uint64 minBytesRcvd = 6;
+  required uint64 maxBytesRcvd = 7;
+  required uint64 latestGenerationStamp = 8;
+
+  /**
+   * The requested checksum mechanism for this block write.
+   */
+  required ChecksumProto requestedChecksum = 9;
+  optional CachingStrategyProto cachingStrategy = 10;
+  optional StorageTypeProto storageType = 11 [default = DISK];
+  repeated StorageTypeProto targetStorageTypes = 12;
+
+  /**
+   * Hint to the DataNode that the block can be allocated on transient
+   * storage i.e. memory and written to disk lazily. The DataNode is free
+   * to ignore this hint.
+   */
+  optional bool allowLazyPersist = 13 [default = false];
+  //whether to pin the block, so Balancer won't move it.
+  optional bool pinning = 14 [default = false];
+  repeated bool targetPinnings = 15;
+}
+  
+message OpTransferBlockProto {
+  required ClientOperationHeaderProto header = 1;
+  repeated DatanodeInfoProto targets = 2;
+  repeated StorageTypeProto targetStorageTypes = 3;
+}
+
+message OpReplaceBlockProto {
+  required BaseHeaderProto header = 1;
+  required string delHint = 2;
+  required DatanodeInfoProto source = 3;
+  optional StorageTypeProto storageType = 4 [default = DISK];
+}
+
+message OpCopyBlockProto {
+  required BaseHeaderProto header = 1;
+}
+
+message OpBlockChecksumProto { 
+  required BaseHeaderProto header = 1;
+}
+
+/**
+ * An ID uniquely identifying a shared memory segment.
+ */
+message ShortCircuitShmIdProto { 
+  required int64 hi = 1;
+  required int64 lo = 2;
+}
+
+/**
+ * An ID uniquely identifying a slot within a shared memory segment.
+ */
+message ShortCircuitShmSlotProto {
+  required ShortCircuitShmIdProto shmId = 1;
+  required int32 slotIdx = 2; 
+}
+
+message OpRequestShortCircuitAccessProto { 
+  required BaseHeaderProto header = 1;
+
+  /** In order to get short-circuit access to block data, clients must set this
+   * to the highest version of the block data that they can understand.
+   * Currently 1 is the only version, but more versions may exist in the future
+   * if the on-disk format changes.
+   */
+  required uint32 maxVersion = 2;
+
+  /**
+   * The shared memory slot to use, if we are using one.
+   */
+  optional ShortCircuitShmSlotProto slotId = 3;
+
+  /**
+   * True if the client supports verifying that the file descriptor has been
+   * sent successfully.
+   */
+  optional bool supportsReceiptVerification = 4 [default = false];
+}
+
+message ReleaseShortCircuitAccessRequestProto {
+  required ShortCircuitShmSlotProto slotId = 1;
+  optional DataTransferTraceInfoProto traceInfo = 2;
+}
+
+message ReleaseShortCircuitAccessResponseProto {
+  required Status status = 1;
+  optional string error = 2;
+}
+
+message ShortCircuitShmRequestProto { 
+  // The name of the client requesting the shared memory segment.  This is
+  // purely for logging / debugging purposes.
+  required string clientName = 1;
+  optional DataTransferTraceInfoProto traceInfo = 2;
+}
+
+message ShortCircuitShmResponseProto { 
+  required Status status = 1;
+  optional string error = 2;
+  optional ShortCircuitShmIdProto id = 3;
+}
+
+message PacketHeaderProto {
+  // All fields must be fixed-length!
+  required sfixed64 offsetInBlock = 1;
+  required sfixed64 seqno = 2;
+  required bool lastPacketInBlock = 3;
+  required sfixed32 dataLen = 4;
+  optional bool syncBlock = 5 [default = false];
+}
+
+// Status is a 4-bit enum
+enum Status {
+  SUCCESS = 0;
+  ERROR = 1;
+  ERROR_CHECKSUM = 2;
+  ERROR_INVALID = 3;
+  ERROR_EXISTS = 4;
+  ERROR_ACCESS_TOKEN = 5;
+  CHECKSUM_OK = 6;
+  ERROR_UNSUPPORTED = 7;
+  OOB_RESTART = 8;            // Quick restart
+  OOB_RESERVED1 = 9;          // Reserved
+  OOB_RESERVED2 = 10;         // Reserved
+  OOB_RESERVED3 = 11;         // Reserved
+  IN_PROGRESS = 12;
+}
+
+enum ShortCircuitFdResponse {
+  DO_NOT_USE_RECEIPT_VERIFICATION = 0;
+  USE_RECEIPT_VERIFICATION = 1;
+}
+
+message PipelineAckProto {
+  required sint64 seqno = 1;
+  repeated Status reply = 2;
+  optional uint64 downstreamAckTimeNanos = 3 [default = 0];
+  repeated uint32 flag = 4 [packed=true];
+}
+
+/**
+ * Sent as part of the BlockOpResponseProto
+ * for READ_BLOCK and COPY_BLOCK operations.
+ */
+message ReadOpChecksumInfoProto {
+  required ChecksumProto checksum = 1;
+
+  /**
+   * The offset into the block at which the first packet
+   * will start. This is necessary since reads will align
+   * backwards to a checksum chunk boundary.
+   */
+  required uint64 chunkOffset = 2;
+}
+
+message BlockOpResponseProto {
+  required Status status = 1;
+
+  optional string firstBadLink = 2;
+  optional OpBlockChecksumResponseProto checksumResponse = 3;
+  optional ReadOpChecksumInfoProto readOpChecksumInfo = 4;
+
+  /** explanatory text which may be useful to log on the client side */
+  optional string message = 5;
+
+  /** If the server chooses to agree to the request of a client for
+   * short-circuit access, it will send a response message with the relevant
+   * file descriptors attached.
+   *
+   * In the body of the message, this version number will be set to the
+   * specific version number of the block data that the client is about to
+   * read.
+   */
+  optional uint32 shortCircuitAccessVersion = 6;
+}
+
+/**
+ * Message sent from the client to the DN after reading the entire
+ * read request.
+ */
+message ClientReadStatusProto {
+  required Status status = 1;
+}
+
+message DNTransferAckProto {
+  required Status status = 1;
+}
+
+message OpBlockChecksumResponseProto {
+  required uint32 bytesPerCrc = 1;
+  required uint64 crcPerBlock = 2;
+  required bytes md5 = 3;
+  optional ChecksumTypeProto crcType = 4;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/encryption.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/encryption.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/encryption.proto
new file mode 100644
index 0000000..68b2f3a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/encryption.proto
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * These .proto interfaces are private and stable.
+ * Please see http://wiki.apache.org/hadoop/Compatibility
+ * for what changes are allowed for a *stable* .proto interface.
+ */
+
+// This file contains protocol buffers that are used throughout HDFS -- i.e.
+// by the client, server, and data transfer protocols.
+
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "EncryptionZonesProtos";
+option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
+
+import "hdfs.proto";
+
+message CreateEncryptionZoneRequestProto {
+  required string src = 1;
+  optional string keyName = 2;
+}
+
+message CreateEncryptionZoneResponseProto {
+}
+
+message ListEncryptionZonesRequestProto {
+  required int64 id = 1;
+}
+
+message EncryptionZoneProto {
+  required int64 id = 1;
+  required string path = 2;
+  required CipherSuiteProto suite = 3;
+  required CryptoProtocolVersionProto cryptoProtocolVersion = 4;
+  required string keyName = 5;
+}
+
+message ListEncryptionZonesResponseProto {
+  repeated EncryptionZoneProto zones = 1;
+  required bool hasMore = 2;
+}
+
+message GetEZForPathRequestProto {
+    required string src = 1;
+}
+
+message GetEZForPathResponseProto {
+    optional EncryptionZoneProto zone = 1;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
new file mode 100644
index 0000000..86fb462
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
@@ -0,0 +1,611 @@
+/**
+ * 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.
+ */
+
+/**
+ * These .proto interfaces are private and stable.
+ * Please see http://wiki.apache.org/hadoop/Compatibility
+ * for what changes are allowed for a *stable* .proto interface.
+ */
+
+// This file contains protocol buffers that are used throughout HDFS -- i.e.
+// by the client, server, and data transfer protocols.
+
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "HdfsProtos";
+option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
+
+import "Security.proto";
+
+/**
+ * Extended block idenfies a block
+ */
+message ExtendedBlockProto {
+  required string poolId = 1;   // Block pool id - gloablly unique across clusters
+  required uint64 blockId = 2;  // the local id within a pool
+  required uint64 generationStamp = 3;
+  optional uint64 numBytes = 4 [default = 0];  // len does not belong in ebid 
+                                               // here for historical reasons
+}
+
+/**
+ * Identifies a Datanode
+ */
+message DatanodeIDProto {
+  required string ipAddr = 1;    // IP address
+  required string hostName = 2;  // hostname
+  required string datanodeUuid = 3;     // UUID assigned to the Datanode. For
+                                        // upgraded clusters this is the same
+                                        // as the original StorageID of the
+                                        // Datanode.
+  required uint32 xferPort = 4;  // data streaming port
+  required uint32 infoPort = 5;  // datanode http port
+  required uint32 ipcPort = 6;   // ipc server port
+  optional uint32 infoSecurePort = 7 [default = 0]; // datanode https port
+}
+
+/**
+ * Datanode local information
+ */
+message DatanodeLocalInfoProto {
+  required string softwareVersion = 1;
+  required string configVersion = 2;
+  required uint64 uptime = 3;
+}
+
+/**
+ * DatanodeInfo array
+ */
+message DatanodeInfosProto {
+  repeated DatanodeInfoProto datanodes = 1;
+}
+
+/**
+ * The status of a Datanode
+ */
+message DatanodeInfoProto {
+  required DatanodeIDProto id = 1;
+  optional uint64 capacity = 2 [default = 0];
+  optional uint64 dfsUsed = 3 [default = 0];
+  optional uint64 remaining = 4 [default = 0];
+  optional uint64 blockPoolUsed = 5 [default = 0];
+  optional uint64 lastUpdate = 6 [default = 0];
+  optional uint32 xceiverCount = 7 [default = 0];
+  optional string location = 8;
+  enum AdminState {
+    NORMAL = 0;
+    DECOMMISSION_INPROGRESS = 1;
+    DECOMMISSIONED = 2;
+  }
+
+  optional AdminState adminState = 10 [default = NORMAL];
+  optional uint64 cacheCapacity = 11 [default = 0];
+  optional uint64 cacheUsed = 12 [default = 0];
+  optional uint64 lastUpdateMonotonic = 13 [default = 0];
+}
+
+/**
+ * Represents a storage available on the datanode
+ */
+message DatanodeStorageProto {
+  enum StorageState {
+    NORMAL = 0;
+    READ_ONLY_SHARED = 1;
+  }
+
+  required string storageUuid = 1;
+  optional StorageState state = 2 [default = NORMAL];
+  optional StorageTypeProto storageType = 3 [default = DISK];
+}
+
+message StorageReportProto {
+  required string storageUuid = 1 [ deprecated = true ];
+  optional bool failed = 2 [ default = false ];
+  optional uint64 capacity = 3 [ default = 0 ];
+  optional uint64 dfsUsed = 4 [ default = 0 ];
+  optional uint64 remaining = 5 [ default = 0 ];
+  optional uint64 blockPoolUsed = 6 [ default = 0 ];
+  optional DatanodeStorageProto storage = 7; // supersedes StorageUuid
+}
+
+/**
+ * Summary of a file or directory
+ */
+message ContentSummaryProto {
+  required uint64 length = 1;
+  required uint64 fileCount = 2;
+  required uint64 directoryCount = 3;
+  required uint64 quota = 4;
+  required uint64 spaceConsumed = 5;
+  required uint64 spaceQuota = 6;
+  optional StorageTypeQuotaInfosProto typeQuotaInfos = 7;
+}
+
+/**
+ * Storage type quota and usage information of a file or directory
+ */
+message StorageTypeQuotaInfosProto {
+  repeated StorageTypeQuotaInfoProto typeQuotaInfo = 1;
+}
+
+message StorageTypeQuotaInfoProto {
+  required StorageTypeProto type = 1;
+  required uint64 quota = 2;
+  required uint64 consumed = 3;
+}
+
+/**
+ * Contains a list of paths corresponding to corrupt files and a cookie
+ * used for iterative calls to NameNode.listCorruptFileBlocks.
+ *
+ */
+message CorruptFileBlocksProto {
+ repeated string files = 1;
+ required string   cookie = 2;
+}
+
+/**
+ * File or Directory permision - same spec as posix
+ */
+message FsPermissionProto {
+  required uint32 perm = 1;       // Actually a short - only 16bits used
+}
+
+/**
+ * Types of recognized storage media.
+ */
+enum StorageTypeProto {
+  DISK = 1;
+  SSD = 2;
+  ARCHIVE = 3;
+  RAM_DISK = 4;
+}
+
+/**
+ * A list of storage types. 
+ */
+message StorageTypesProto {
+  repeated StorageTypeProto storageTypes = 1;
+}
+
+/**
+ * Block replica storage policy.
+ */
+message BlockStoragePolicyProto {
+  required uint32 policyId = 1;
+  required string name = 2;
+  // a list of storage types for storing the block replicas when creating a
+  // block.
+  required StorageTypesProto creationPolicy = 3;
+  // A list of storage types for creation fallback storage.
+  optional StorageTypesProto creationFallbackPolicy = 4;
+  optional StorageTypesProto replicationFallbackPolicy = 5;
+}
+
+/**
+ * A list of storage IDs. 
+ */
+message StorageUuidsProto {
+  repeated string storageUuids = 1;
+}
+
+/**
+ * A LocatedBlock gives information about a block and its location.
+ */ 
+message LocatedBlockProto {
+  required ExtendedBlockProto b  = 1;
+  required uint64 offset = 2;           // offset of first byte of block in the file
+  repeated DatanodeInfoProto locs = 3;  // Locations ordered by proximity to client ip
+  required bool corrupt = 4;            // true if all replicas of a block are corrupt, else false
+                                        // If block has few corrupt replicas, they are filtered and 
+                                        // their locations are not part of this object
+
+  required hadoop.common.TokenProto blockToken = 5;
+  repeated bool isCached = 6 [packed=true]; // if a location in locs is cached
+  repeated StorageTypeProto storageTypes = 7;
+  repeated string storageIDs = 8;
+}
+
+message DataEncryptionKeyProto {
+  required uint32 keyId = 1;
+  required string blockPoolId = 2;
+  required bytes nonce = 3;
+  required bytes encryptionKey = 4;
+  required uint64 expiryDate = 5;
+  optional string encryptionAlgorithm = 6;
+}
+
+/**
+ * Cipher suite.
+ */
+enum CipherSuiteProto {
+    UNKNOWN = 1;
+    AES_CTR_NOPADDING = 2;
+}
+
+/**
+ * Crypto protocol version used to access encrypted files.
+ */
+enum CryptoProtocolVersionProto {
+    UNKNOWN_PROTOCOL_VERSION = 1;
+    ENCRYPTION_ZONES = 2;
+}
+
+/**
+ * Encryption information for a file.
+ */
+message FileEncryptionInfoProto {
+  required CipherSuiteProto suite = 1;
+  required CryptoProtocolVersionProto cryptoProtocolVersion = 2;
+  required bytes key = 3;
+  required bytes iv = 4;
+  required string keyName = 5;
+  required string ezKeyVersionName = 6;
+}
+
+/**
+ * Encryption information for an individual
+ * file within an encryption zone
+ */
+message PerFileEncryptionInfoProto {
+  required bytes key = 1;
+  required bytes iv = 2;
+  required string ezKeyVersionName = 3;
+}
+
+/**
+ * Encryption information for an encryption
+ * zone
+ */
+message ZoneEncryptionInfoProto {
+  required CipherSuiteProto suite = 1;
+  required CryptoProtocolVersionProto cryptoProtocolVersion = 2;
+  required string keyName = 3;
+}
+
+/**
+ * Cipher option
+ */
+message CipherOptionProto {
+  required CipherSuiteProto suite = 1;
+  optional bytes inKey = 2;
+  optional bytes inIv = 3;
+  optional bytes outKey = 4;
+  optional bytes outIv = 5;
+}
+
+/**
+ * A set of file blocks and their locations.
+ */
+message LocatedBlocksProto {
+  required uint64 fileLength = 1;
+  repeated LocatedBlockProto blocks = 2;
+  required bool underConstruction = 3;
+  optional LocatedBlockProto lastBlock = 4;
+  required bool isLastBlockComplete = 5;
+  optional FileEncryptionInfoProto fileEncryptionInfo = 6;
+}
+
+/**
+ * Status of a file, directory or symlink
+ * Optionally includes a file's block locations if requested by client on the rpc call.
+ */
+message HdfsFileStatusProto {
+  enum FileType {
+    IS_DIR = 1;
+    IS_FILE = 2;
+    IS_SYMLINK = 3;
+  }
+  required FileType fileType = 1;
+  required bytes path = 2;          // local name of inode encoded java UTF8
+  required uint64 length = 3;
+  required FsPermissionProto permission = 4;
+  required string owner = 5;
+  required string group = 6;
+  required uint64 modification_time = 7;
+  required uint64 access_time = 8;
+
+  // Optional fields for symlink
+  optional bytes symlink = 9;             // if symlink, target encoded java UTF8 
+
+  // Optional fields for file
+  optional uint32 block_replication = 10 [default = 0]; // only 16bits used
+  optional uint64 blocksize = 11 [default = 0];
+  optional LocatedBlocksProto locations = 12;  // suppled only if asked by client
+
+  // Optional field for fileId
+  optional uint64 fileId = 13 [default = 0]; // default as an invalid id
+  optional int32 childrenNum = 14 [default = -1];
+  // Optional field for file encryption
+  optional FileEncryptionInfoProto fileEncryptionInfo = 15;
+
+  optional uint32 storagePolicy = 16 [default = 0]; // block storage policy id
+} 
+
+/**
+ * Checksum algorithms/types used in HDFS
+ * Make sure this enum's integer values match enum values' id properties defined
+ * in org.apache.hadoop.util.DataChecksum.Type
+ */
+enum ChecksumTypeProto {
+  CHECKSUM_NULL = 0;
+  CHECKSUM_CRC32 = 1;
+  CHECKSUM_CRC32C = 2;
+}
+
+/**
+ * HDFS Server Defaults
+ */
+message FsServerDefaultsProto {
+  required uint64 blockSize = 1;
+  required uint32 bytesPerChecksum = 2;
+  required uint32 writePacketSize = 3;
+  required uint32 replication = 4; // Actually a short - only 16 bits used
+  required uint32 fileBufferSize = 5;
+  optional bool encryptDataTransfer = 6 [default = false];
+  optional uint64 trashInterval = 7 [default = 0];
+  optional ChecksumTypeProto checksumType = 8 [default = CHECKSUM_CRC32];
+}
+
+
+/**
+ * Directory listing
+ */
+message DirectoryListingProto {
+  repeated HdfsFileStatusProto partialListing = 1;
+  required uint32 remainingEntries  = 2;
+}
+
+/**
+ * Status of a snapshottable directory: besides the normal information for 
+ * a directory status, also include snapshot quota, number of snapshots, and
+ * the full path of the parent directory. 
+ */
+message SnapshottableDirectoryStatusProto {
+  required HdfsFileStatusProto dirStatus = 1;
+
+  // Fields specific for snapshottable directory
+  required uint32 snapshot_quota = 2;
+  required uint32 snapshot_number = 3;
+  required bytes parent_fullpath = 4;
+}
+
+/**
+ * Snapshottable directory listing
+ */
+message SnapshottableDirectoryListingProto {
+  repeated SnapshottableDirectoryStatusProto snapshottableDirListing = 1;
+}
+
+/**
+ * Snapshot diff report entry
+ */
+message SnapshotDiffReportEntryProto {
+  required bytes fullpath = 1;
+  required string modificationLabel = 2;
+  optional bytes targetPath = 3;
+}
+
+/**
+ * Snapshot diff report
+ */
+message SnapshotDiffReportProto {
+  // full path of the directory where snapshots were taken
+  required string snapshotRoot = 1;
+  required string fromSnapshot = 2;
+  required string toSnapshot = 3;
+  repeated SnapshotDiffReportEntryProto diffReportEntries = 4;
+}
+
+/**
+ * Common node information shared by all the nodes in the cluster
+ */
+message StorageInfoProto {
+  required uint32 layoutVersion = 1; // Layout version of the file system
+  required uint32 namespceID = 2;    // File system namespace ID
+  required string clusterID = 3;     // ID of the cluster
+  required uint64 cTime = 4;         // File system creation time
+}
+
+/**
+ * Information sent by a namenode to identify itself to the primary namenode.
+ */
+message NamenodeRegistrationProto {
+  required string rpcAddress = 1;    // host:port of the namenode RPC address
+  required string httpAddress = 2;   // host:port of the namenode http server
+  enum NamenodeRoleProto {
+    NAMENODE = 1;
+    BACKUP = 2;
+    CHECKPOINT = 3;
+  }
+  required StorageInfoProto storageInfo = 3;  // Node information
+  optional NamenodeRoleProto role = 4 [default = NAMENODE];        // Namenode role
+}
+
+/**
+ * Unique signature to identify checkpoint transactions.
+ */
+message CheckpointSignatureProto {
+  required string blockPoolId = 1;
+  required uint64 mostRecentCheckpointTxId = 2;
+  required uint64 curSegmentTxId = 3;
+  required StorageInfoProto storageInfo = 4;
+}
+
+/**
+ * Command sent from one namenode to another namenode.
+ */
+message NamenodeCommandProto {
+  enum Type {
+    NamenodeCommand = 0;      // Base command
+    CheckPointCommand = 1;    // Check point command
+  }
+  required uint32 action = 1;
+  required Type type = 2;
+  optional CheckpointCommandProto checkpointCmd = 3;
+}
+
+/**
+ * Command returned from primary to checkpointing namenode.
+ * This command has checkpoint signature that identifies
+ * checkpoint transaction and is needed for further
+ * communication related to checkpointing.
+ */
+message CheckpointCommandProto {
+  // Unique signature to identify checkpoint transation
+  required CheckpointSignatureProto signature = 1; 
+
+  // If true, return transfer image to primary upon the completion of checkpoint
+  required bool needToReturnImage = 2;
+}
+
+/**
+ * Block information
+ *
+ * Please be wary of adding additional fields here, since INodeFiles
+ * need to fit in PB's default max message size of 64MB.
+ * We restrict the max # of blocks per file
+ * (dfs.namenode.fs-limits.max-blocks-per-file), but it's better
+ * to avoid changing this.
+ */
+message BlockProto {
+  required uint64 blockId = 1;
+  required uint64 genStamp = 2;
+  optional uint64 numBytes = 3 [default = 0];
+}
+
+/**
+ * Block and datanodes where is it located
+ */
+message BlockWithLocationsProto {
+  required BlockProto block = 1;   // Block
+  repeated string datanodeUuids = 2; // Datanodes with replicas of the block
+  repeated string storageUuids = 3;  // Storages with replicas of the block
+  repeated StorageTypeProto storageTypes = 4;
+}
+
+/**
+ * List of block with locations
+ */
+message BlocksWithLocationsProto {
+  repeated BlockWithLocationsProto blocks = 1;
+}
+
+/**
+ * Editlog information with available transactions
+ */
+message RemoteEditLogProto {
+  required uint64 startTxId = 1;  // Starting available edit log transaction
+  required uint64 endTxId = 2;    // Ending available edit log transaction
+  optional bool isInProgress = 3 [default = false];
+}
+
+/**
+ * Enumeration of editlogs available on a remote namenode
+ */
+message RemoteEditLogManifestProto {
+  repeated RemoteEditLogProto logs = 1;
+}
+
+/**
+ * Namespace information that describes namespace on a namenode
+ */
+message NamespaceInfoProto {
+  required string buildVersion = 1;         // Software revision version (e.g. an svn or git revision)
+  required uint32 unused = 2;               // Retained for backward compatibility
+  required string blockPoolID = 3;          // block pool used by the namespace
+  required StorageInfoProto storageInfo = 4;// Node information
+  required string softwareVersion = 5;      // Software version number (e.g. 2.0.0)
+  optional uint64 capabilities = 6 [default = 0]; // feature flags
+}
+
+/**
+ * Block access token information
+ */
+message BlockKeyProto {
+  required uint32 keyId = 1;      // Key identifier
+  required uint64 expiryDate = 2; // Expiry time in milliseconds
+  optional bytes keyBytes = 3;    // Key secret
+}
+
+/**
+ * Current key and set of block keys at the namenode.
+ */
+message ExportedBlockKeysProto {
+  required bool isBlockTokenEnabled = 1;
+  required uint64 keyUpdateInterval = 2;
+  required uint64 tokenLifeTime = 3;
+  required BlockKeyProto currentKey = 4;
+  repeated BlockKeyProto allKeys = 5;
+}
+
+/**
+ * State of a block replica at a datanode
+ */
+enum ReplicaStateProto {
+  FINALIZED = 0;  // State of a replica when it is not modified
+  RBW = 1;        // State of replica that is being written to
+  RWR = 2;        // State of replica that is waiting to be recovered
+  RUR = 3;        // State of replica that is under recovery
+  TEMPORARY = 4;  // State of replica that is created for replication
+}
+
+/**
+ * Block that needs to be recovered with at a given location
+ */
+message RecoveringBlockProto {
+  required uint64 newGenStamp = 1;        // New genstamp post recovery
+  required LocatedBlockProto block = 2;   // Block to be recovered
+  optional BlockProto truncateBlock = 3;  // New block for recovery (truncate)
+}
+
+/**
+ * void request
+ */
+message VersionRequestProto {
+}
+
+/**
+ * Version response from namenode.
+ */
+message VersionResponseProto {
+  required NamespaceInfoProto info = 1;
+}
+
+/**
+ * Information related to a snapshot
+ * TODO: add more information
+ */
+message SnapshotInfoProto {
+  required string snapshotName = 1;
+  required string snapshotRoot = 2;
+  required FsPermissionProto permission = 3;
+  required string owner = 4;
+  required string group = 5;
+  required string createTime = 6;
+  // TODO: do we need access time?
+}
+
+/**
+ * Rolling upgrade status
+ */
+message RollingUpgradeStatusProto {
+  required string blockPoolId = 1;
+  optional bool finalized = 2 [default = false];
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/inotify.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/inotify.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/inotify.proto
new file mode 100644
index 0000000..5b78fe6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/inotify.proto
@@ -0,0 +1,126 @@
+/**
+ * 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.
+ */
+
+/**
+ * These .proto interfaces are private and stable.
+ * Please see http://wiki.apache.org/hadoop/Compatibility
+ * for what changes are allowed for a *stable* .proto interface.
+ */
+
+// This file contains protocol buffers used to communicate edits to clients
+// as part of the inotify system.
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "InotifyProtos";
+option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
+
+import "acl.proto";
+import "xattr.proto";
+import "hdfs.proto";
+
+enum EventType {
+  EVENT_CREATE = 0x0;
+  EVENT_CLOSE = 0x1;
+  EVENT_APPEND = 0x2;
+  EVENT_RENAME = 0x3;
+  EVENT_METADATA = 0x4;
+  EVENT_UNLINK = 0x5;
+}
+
+message EventProto {
+  required EventType type = 1;
+  required bytes contents = 2;
+}
+
+message EventBatchProto {
+  required int64 txid = 1;
+  repeated EventProto events = 2;
+}
+
+enum INodeType {
+  I_TYPE_FILE = 0x0;
+  I_TYPE_DIRECTORY = 0x1;
+  I_TYPE_SYMLINK = 0x2;
+}
+
+enum MetadataUpdateType {
+  META_TYPE_TIMES = 0x0;
+  META_TYPE_REPLICATION = 0x1;
+  META_TYPE_OWNER = 0x2;
+  META_TYPE_PERMS = 0x3;
+  META_TYPE_ACLS = 0x4;
+  META_TYPE_XATTRS = 0x5;
+}
+
+message CreateEventProto {
+  required INodeType type = 1;
+  required string path = 2;
+  required int64 ctime = 3;
+  required string ownerName = 4;
+  required string groupName = 5;
+  required FsPermissionProto perms = 6;
+  optional int32 replication = 7;
+  optional string symlinkTarget = 8;
+  optional bool overwrite = 9;
+  optional int64 defaultBlockSize = 10 [default=0];
+}
+
+message CloseEventProto {
+  required string path = 1;
+  required int64 fileSize = 2;
+  required int64 timestamp = 3;
+}
+
+message AppendEventProto {
+  required string path = 1;
+  optional bool newBlock = 2 [default = false];
+}
+
+message RenameEventProto {
+  required string srcPath = 1;
+  required string destPath = 2;
+  required int64 timestamp = 3;
+}
+
+message MetadataUpdateEventProto {
+  required string path = 1;
+  required MetadataUpdateType type = 2;
+  optional int64 mtime = 3;
+  optional int64 atime = 4;
+  optional int32 replication = 5;
+  optional string ownerName = 6;
+  optional string groupName = 7;
+  optional FsPermissionProto perms = 8;
+  repeated AclEntryProto acls = 9;
+  repeated XAttrProto xAttrs = 10;
+  optional bool xAttrsRemoved = 11;
+}
+
+message UnlinkEventProto {
+  required string path = 1;
+  required int64 timestamp = 2;
+}
+
+message EventsListProto {
+  repeated EventProto events = 1; // deprecated
+  required int64 firstTxid = 2;
+  required int64 lastTxid = 3;
+  required int64 syncTxid = 4;
+  repeated EventBatchProto batch = 5;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/xattr.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/xattr.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/xattr.proto
new file mode 100644
index 0000000..6c8b5eb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/xattr.proto
@@ -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.
+ */
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "XAttrProtos";
+option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
+  
+message XAttrProto {
+  enum XAttrNamespaceProto {
+    USER      = 0;
+    TRUSTED   = 1;
+    SECURITY  = 2;
+    SYSTEM    = 3;
+    RAW       = 4;
+  }
+  
+  required XAttrNamespaceProto namespace = 1;
+  required string name = 2;
+  optional bytes value = 3;
+}
+  
+enum XAttrSetFlagProto {
+  XATTR_CREATE     = 0x01;
+  XATTR_REPLACE    = 0x02;
+}
+
+message SetXAttrRequestProto {
+  required string src          = 1;
+  optional XAttrProto xAttr    = 2;
+  optional uint32 flag         = 3; //bits set using XAttrSetFlagProto
+}
+
+message SetXAttrResponseProto {
+}
+
+message GetXAttrsRequestProto {
+  required string src = 1;
+  repeated XAttrProto xAttrs = 2;
+}
+
+message GetXAttrsResponseProto {
+  repeated XAttrProto xAttrs = 1;
+}
+
+message ListXAttrsRequestProto {
+  required string src = 1;
+}
+
+message ListXAttrsResponseProto {
+  repeated XAttrProto xAttrs = 1;
+}
+
+message RemoveXAttrRequestProto {
+  required string src        = 1;
+  optional XAttrProto xAttr  = 2;
+}
+
+message RemoveXAttrResponseProto {
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/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 a7c9e7c..f595751 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -710,6 +710,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8712. Remove 'public' and 'abstracta modifiers in FsVolumeSpi and
     FsDatasetSpi (Lei (Eddy) Xu via vinayakumarb)
 
+    HDFS-8726. Move protobuf files that define the client-sever protocols to
+    hdfs-client. (wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/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 f90644c..db38851 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -328,26 +328,20 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
               <protocCommand>${protoc.path}</protocCommand>
               <imports>
                 <param>${basedir}/../../hadoop-common-project/hadoop-common/src/main/proto</param>
+                <param>${basedir}/../hadoop-hdfs-client/src/main/proto</param>
                 <param>${basedir}/src/main/proto</param>
               </imports>
               <source>
                 <directory>${basedir}/src/main/proto</directory>
                 <includes>
-                  <include>ClientDatanodeProtocol.proto</include>
-                  <include>ClientNamenodeProtocol.proto</include>
                   <include>DatanodeProtocol.proto</include>
                   <include>HAZKInfo.proto</include>
                   <include>InterDatanodeProtocol.proto</include>
                   <include>JournalProtocol.proto</include>
                   <include>NamenodeProtocol.proto</include>
                   <include>QJournalProtocol.proto</include>
-                  <include>acl.proto</include>
-                  <include>xattr.proto</include>
-                  <include>datatransfer.proto</include>
+                  <include>editlog.proto</include>
                   <include>fsimage.proto</include>
-                  <include>hdfs.proto</include>
-                  <include>encryption.proto</include>
-                  <include>inotify.proto</include>
                 </includes>
               </source>
               <output>${project.build.directory}/generated-sources/java</output>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml
index 3b205e4..7e58606 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml
@@ -112,7 +112,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
               <protocCommand>${protoc.path}</protocCommand>
               <imports>
                 <param>${basedir}/../../../../../hadoop-common-project/hadoop-common/src/main/proto</param>
-                <param>${basedir}/../../../../../hadoop-hdfs-project/hadoop-hdfs/src/main/proto</param>
+                <param>${basedir}/../../../../../hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto</param>
                 <param>${basedir}/src/main/proto</param>
               </imports>
               <source>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/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 c8e565e..ab36f17 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
@@ -103,8 +103,8 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
-import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEditLogProto;
-import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrEditLogProto;
+import org.apache.hadoop.hdfs.protocol.proto.EditLogProtos.AclEditLogProto;
+import org.apache.hadoop.hdfs.protocol.proto.EditLogProtos.XAttrEditLogProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;


[17/50] hadoop git commit: HADOOP-12201. Add tracing to FileSystem#createFileSystem and Globber#glob (cmccabe)

Posted by ar...@apache.org.
HADOOP-12201. Add tracing to FileSystem#createFileSystem and Globber#glob (cmccabe)


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

Branch: refs/heads/HDFS-7240
Commit: b8832fcf1e2ae1e43d5e4523016731af40ab58d7
Parents: 2e3d83f
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Wed Jul 8 20:07:21 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Wed Jul 8 20:07:21 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +++
 .../java/org/apache/hadoop/fs/FileSystem.java   | 20 ++++++++++++++++----
 .../main/java/org/apache/hadoop/fs/Globber.java | 17 +++++++++++++++++
 3 files changed, 36 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8832fcf/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 eb18e6c..c99fb5e 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -687,6 +687,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12195. Add annotation to package-info.java file to workaround
     MCOMPILER-205. (wang)
 
+    HADOOP-12201. Add tracing to FileSystem#createFileSystem and Globber#glob
+    (cmccabe)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8832fcf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
index c73caf7..5e03e88 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
@@ -67,6 +67,9 @@ import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.htrace.Span;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -2675,10 +2678,19 @@ public abstract class FileSystem extends Configured implements Closeable {
 
   private static FileSystem createFileSystem(URI uri, Configuration conf
       ) throws IOException {
-    Class<?> clazz = getFileSystemClass(uri.getScheme(), conf);
-    FileSystem fs = (FileSystem)ReflectionUtils.newInstance(clazz, conf);
-    fs.initialize(uri, conf);
-    return fs;
+    TraceScope scope = Trace.startSpan("FileSystem#createFileSystem");
+    Span span = scope.getSpan();
+    if (span != null) {
+      span.addKVAnnotation("scheme", uri.getScheme());
+    }
+    try {
+      Class<?> clazz = getFileSystemClass(uri.getScheme(), conf);
+      FileSystem fs = (FileSystem)ReflectionUtils.newInstance(clazz, conf);
+      fs.initialize(uri, conf);
+      return fs;
+    } finally {
+      scope.close();
+    }
   }
 
   /** Caching FileSystem objects */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8832fcf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java
index 9cb810f..48639b4 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java
@@ -28,6 +28,10 @@ import org.apache.commons.logging.Log;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
+import org.apache.htrace.Span;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 class Globber {
@@ -136,6 +140,19 @@ class Globber {
   }
 
   public FileStatus[] glob() throws IOException {
+    TraceScope scope = Trace.startSpan("Globber#glob");
+    Span span = scope.getSpan();
+    if (span != null) {
+      span.addKVAnnotation("pattern", pathPattern.toUri().getPath());
+    }
+    try {
+      return doGlob();
+    } finally {
+      scope.close();
+    }
+  }
+
+  private FileStatus[] doGlob() throws IOException {
     // First we get the scheme and authority of the pattern that was passed
     // in.
     String scheme = schemeFromPath(pathPattern);


[13/50] hadoop git commit: HDFS-8726. Move protobuf files that define the client-sever protocols to hdfs-client. Contributed by Haohui Mai.

Posted by ar...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto
deleted file mode 100644
index e0d1f5f..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto
+++ /dev/null
@@ -1,247 +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.
- */
-
-/**
- * These .proto interfaces are private and stable.
- * Please see http://wiki.apache.org/hadoop/Compatibility
- * for what changes are allowed for a *stable* .proto interface.
- */
-
-// This file contains protocol buffers that are used throughout HDFS -- i.e.
-// by the client, server, and data transfer protocols.
-
-option java_package = "org.apache.hadoop.hdfs.protocol.proto";
-option java_outer_classname = "ClientDatanodeProtocolProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-package hadoop.hdfs;
-
-import "Security.proto";
-import "hdfs.proto";
-
-/**
- * block - block for which visible length is requested
- */
-message GetReplicaVisibleLengthRequestProto {
-  required ExtendedBlockProto block = 1;
-}
-
-/**
- * length - visible length of the block
- */
-message GetReplicaVisibleLengthResponseProto {
-  required uint64 length = 1;
-}
-
-/**
- * void request
- */
-message RefreshNamenodesRequestProto {
-}
-
-/**
- * void response
- */
-message RefreshNamenodesResponseProto {
-}
-
-/**
- * blockPool - block pool to be deleted
- * force - if false, delete the block pool only if it is empty.
- *         if true, delete the block pool even if it has blocks.
- */
-message DeleteBlockPoolRequestProto {
-  required string blockPool = 1;
-  required bool force = 2;
-}
-
-/**
- * void response
- */
-message DeleteBlockPoolResponseProto {
-}
-
-/**
- * Gets the file information where block and its metadata is stored
- * block - block for which path information is being requested
- * token - block token
- *
- * This message is deprecated in favor of file descriptor passing.
- */
-message GetBlockLocalPathInfoRequestProto {
-  required ExtendedBlockProto block = 1;
-  required hadoop.common.TokenProto token = 2;
-}
-
-/**
- * block - block for which file path information is being returned
- * localPath - file path where the block data is stored
- * localMetaPath - file path where the block meta data is stored
- *
- * This message is deprecated in favor of file descriptor passing.
- */
-message GetBlockLocalPathInfoResponseProto {
-  required ExtendedBlockProto block = 1;
-  required string localPath = 2;
-  required string localMetaPath = 3;
-}
-
-/**
- * Query for the disk locations of a number of blocks on this DN.
- * blockPoolId - the pool to query
- * blockIds - list of block IDs to query
- * tokens - list of access tokens corresponding to list of block IDs
- */
-message GetHdfsBlockLocationsRequestProto {
-  // Removed: HDFS-3969
-  // repeated ExtendedBlockProto blocks = 1;
-  repeated hadoop.common.TokenProto tokens = 2;
-
-  required string blockPoolId = 3;
-  repeated sfixed64 blockIds = 4 [ packed = true ];
-}
-
-/**
- * volumeIds - id of each volume, potentially multiple bytes
- * volumeIndexes - for each block, an index into volumeIds specifying the volume
- *               on which it is located. If block is not present on any volume,
- *               index is set to MAX_INT.
- */
-message GetHdfsBlockLocationsResponseProto {
-  repeated bytes volumeIds = 1;
-  repeated uint32 volumeIndexes = 2 [ packed = true ];
-}
-
-/**
- * forUpgrade - if true, clients are advised to wait for restart and quick
- *              upgrade restart is instrumented. Otherwise, datanode does
- *              the regular shutdown.
- */
-message ShutdownDatanodeRequestProto {
-  required bool forUpgrade = 1;
-}
-
-message ShutdownDatanodeResponseProto {
-}
-
-/**
- * Ping datanode for liveness and quick info
- */
-message GetDatanodeInfoRequestProto {
-}
-
-message GetDatanodeInfoResponseProto {
-  required DatanodeLocalInfoProto localInfo = 1;
-}
-
-/** Asks DataNode to reload configuration file. */
-message StartReconfigurationRequestProto {
-}
-
-message StartReconfigurationResponseProto {
-}
-
-message TriggerBlockReportRequestProto {
-  required bool incremental = 1;
-}
-
-message TriggerBlockReportResponseProto {
-}
-
-/** Query the running status of reconfiguration process */
-message GetReconfigurationStatusRequestProto {
-}
-
-message GetReconfigurationStatusConfigChangeProto {
-  required string name = 1;
-  required string oldValue = 2;
-  optional string newValue = 3;
-  optional string errorMessage = 4;  // It is empty if success.
-}
-
-message GetReconfigurationStatusResponseProto {
-  required int64 startTime = 1;
-  optional int64 endTime = 2;
-  repeated GetReconfigurationStatusConfigChangeProto changes = 3;
-}
-
-message ListReconfigurablePropertiesRequestProto {
-}
-
-/** Query the reconfigurable properties on DataNode. */
-message ListReconfigurablePropertiesResponseProto {
-  repeated string name = 1;
-}
-
-/**
- * Protocol used from client to the Datanode.
- * See the request and response for details of rpc call.
- */
-service ClientDatanodeProtocolService {
-  /**
-   * Returns the visible length of the replica
-   */
-  rpc getReplicaVisibleLength(GetReplicaVisibleLengthRequestProto)
-      returns(GetReplicaVisibleLengthResponseProto);
-
-  /**
-   * Refresh the list of federated namenodes from updated configuration.
-   * Adds new namenodes and stops the deleted namenodes.
-   */
-  rpc refreshNamenodes(RefreshNamenodesRequestProto)
-      returns(RefreshNamenodesResponseProto);
-
-  /**
-   * Delete the block pool from the datanode.
-   */
-  rpc deleteBlockPool(DeleteBlockPoolRequestProto)
-      returns(DeleteBlockPoolResponseProto);
-
-  /**
-   * Retrieves the path names of the block file and metadata file stored on the
-   * local file system.
-   */
-  rpc getBlockLocalPathInfo(GetBlockLocalPathInfoRequestProto)
-      returns(GetBlockLocalPathInfoResponseProto);
-
-  /**
-   * Retrieve additional HDFS-specific metadata about a set of blocks stored
-   * on the local file system.
-   */
-  rpc getHdfsBlockLocations(GetHdfsBlockLocationsRequestProto)
-      returns(GetHdfsBlockLocationsResponseProto);
-
-  rpc shutdownDatanode(ShutdownDatanodeRequestProto)
-      returns(ShutdownDatanodeResponseProto);
-
-  rpc getDatanodeInfo(GetDatanodeInfoRequestProto)
-      returns(GetDatanodeInfoResponseProto);
-
-  rpc getReconfigurationStatus(GetReconfigurationStatusRequestProto)
-      returns(GetReconfigurationStatusResponseProto);
-
-  rpc startReconfiguration(StartReconfigurationRequestProto)
-      returns(StartReconfigurationResponseProto);
-
-  rpc listReconfigurableProperties(
-      ListReconfigurablePropertiesRequestProto)
-      returns(ListReconfigurablePropertiesResponseProto);
-
-  rpc triggerBlockReport(TriggerBlockReportRequestProto)
-      returns(TriggerBlockReportResponseProto);
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
deleted file mode 100644
index b44c556..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
+++ /dev/null
@@ -1,863 +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.
- */
-
-/**
- * These .proto interfaces are private and stable.
- * Please see http://wiki.apache.org/hadoop/Compatibility
- * for what changes are allowed for a *stable* .proto interface.
- */
-
-option java_package = "org.apache.hadoop.hdfs.protocol.proto";
-option java_outer_classname = "ClientNamenodeProtocolProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-package hadoop.hdfs;
-
-import "Security.proto";
-import "hdfs.proto";
-import "acl.proto";
-import "xattr.proto";
-import "encryption.proto";
-import "inotify.proto";
-
-/**
- * The ClientNamenodeProtocol Service defines the interface between a client 
- * (as runnign inside a MR Task) and the Namenode.
- * See org.apache.hadoop.hdfs.protocol.ClientProtocol for the javadoc 
- * for each of the methods.
- * The exceptions declared in the above class also apply to this protocol.
- * Exceptions are unwrapped and thrown by the  PB libraries.
- */
-
-message GetBlockLocationsRequestProto {
-  required string src = 1;     // file name
-  required uint64 offset = 2;  // range start offset
-  required uint64 length = 3;  // range length
-}
-
-message GetBlockLocationsResponseProto {
-  optional LocatedBlocksProto locations = 1;
-}
-
-message GetServerDefaultsRequestProto { // No parameters
-}
-
-message GetServerDefaultsResponseProto {
-  required FsServerDefaultsProto serverDefaults = 1;
-}
-
-enum CreateFlagProto {
-  CREATE = 0x01;    // Create a file
-  OVERWRITE = 0x02; // Truncate/overwrite a file. Same as POSIX O_TRUNC
-  APPEND = 0x04;    // Append to a file
-  LAZY_PERSIST = 0x10; // File with reduced durability guarantees.
-  NEW_BLOCK = 0x20; // Write data to a new block when appending
-}
-
-message CreateRequestProto {
-  required string src = 1;
-  required FsPermissionProto masked = 2;
-  required string clientName = 3;
-  required uint32 createFlag = 4;  // bits set using CreateFlag
-  required bool createParent = 5;
-  required uint32 replication = 6; // Short: Only 16 bits used
-  required uint64 blockSize = 7;
-  repeated CryptoProtocolVersionProto cryptoProtocolVersion = 8;
-}
-
-message CreateResponseProto {
-  optional HdfsFileStatusProto fs = 1;
-}
-
-message AppendRequestProto {
-  required string src = 1;
-  required string clientName = 2;
-  optional uint32 flag = 3; // bits set using CreateFlag
-}
-
-message AppendResponseProto {
-  optional LocatedBlockProto block = 1;
-  optional HdfsFileStatusProto stat = 2;
-}
-
-message SetReplicationRequestProto {
-  required string src = 1;
-  required uint32 replication = 2; // Short: Only 16 bits used
-}
-
-message SetReplicationResponseProto {
-  required bool result = 1;
-}
-
-message SetStoragePolicyRequestProto {
-  required string src = 1;
-  required string policyName = 2;
-}
-
-message SetStoragePolicyResponseProto { // void response
-}
-
-message GetStoragePoliciesRequestProto { // void request
-}
-
-message GetStoragePoliciesResponseProto {
-  repeated BlockStoragePolicyProto policies = 1;
-}
-
-message SetPermissionRequestProto {
-  required string src = 1;
-  required FsPermissionProto permission = 2;
-}
-
-message SetPermissionResponseProto { // void response
-}
-
-message SetOwnerRequestProto {
-  required string src = 1;
-  optional string username = 2;
-  optional string groupname = 3;
-}
-
-message SetOwnerResponseProto { // void response
-}
-
-message AbandonBlockRequestProto {
-  required ExtendedBlockProto b = 1;
-  required string src = 2;
-  required string holder = 3;
-  optional uint64 fileId = 4 [default = 0];  // default to GRANDFATHER_INODE_ID
-}
-
-message AbandonBlockResponseProto { // void response
-}
-
-message AddBlockRequestProto {
-  required string src = 1;
-  required string clientName = 2;
-  optional ExtendedBlockProto previous = 3;
-  repeated DatanodeInfoProto excludeNodes = 4;
-  optional uint64 fileId = 5 [default = 0];  // default as a bogus id
-  repeated string favoredNodes = 6; //the set of datanodes to use for the block
-}
-
-message AddBlockResponseProto {
-  required LocatedBlockProto block = 1;
-}
-
-message GetAdditionalDatanodeRequestProto {
-  required string src = 1;
-  required ExtendedBlockProto blk = 2;
-  repeated DatanodeInfoProto existings = 3;
-  repeated DatanodeInfoProto excludes = 4;
-  required uint32 numAdditionalNodes = 5;
-  required string clientName = 6;
-  repeated string existingStorageUuids = 7;
-  optional uint64 fileId = 8 [default = 0];  // default to GRANDFATHER_INODE_ID
-}
-
-message GetAdditionalDatanodeResponseProto {
-  required LocatedBlockProto block = 1;
-}
-
-message CompleteRequestProto {
-  required string src = 1;
-  required string clientName = 2;
-  optional ExtendedBlockProto last = 3;
-  optional uint64 fileId = 4 [default = 0];  // default to GRANDFATHER_INODE_ID
-}
-
-message CompleteResponseProto {
-  required bool result = 1;
-}
-
-message ReportBadBlocksRequestProto {
-  repeated LocatedBlockProto blocks = 1;
-}
-
-message ReportBadBlocksResponseProto { // void response
-}
-
-message ConcatRequestProto {
-  required string trg = 1;
-  repeated string srcs = 2;
-}
-
-message ConcatResponseProto { // void response
-}
-
-message TruncateRequestProto {
-  required string src = 1;
-  required uint64 newLength = 2;
-  required string clientName = 3;
-}
-
-message TruncateResponseProto {
-  required bool result = 1;
-}
-
-message RenameRequestProto {
-  required string src = 1;
-  required string dst = 2;
-}
-
-message RenameResponseProto {
-  required bool result = 1;
-}
-
-
-message Rename2RequestProto {
-  required string src = 1;
-  required string dst = 2;
-  required bool overwriteDest = 3;
-}
-
-message Rename2ResponseProto { // void response
-}
-
-message DeleteRequestProto {
-  required string src = 1;
-  required bool recursive = 2;
-}
-
-message DeleteResponseProto {
-    required bool result = 1;
-}
-
-message MkdirsRequestProto {
-  required string src = 1;
-  required FsPermissionProto masked = 2;
-  required bool createParent = 3;
-}
-message MkdirsResponseProto {
-    required bool result = 1;
-}
-
-message GetListingRequestProto {
-  required string src = 1;
-  required bytes startAfter = 2;
-  required bool needLocation = 3;
-}
-message GetListingResponseProto {
-  optional DirectoryListingProto dirList = 1;
-}
-
-message GetSnapshottableDirListingRequestProto { // no input parameters
-}
-message GetSnapshottableDirListingResponseProto {
-  optional SnapshottableDirectoryListingProto snapshottableDirList = 1;
-}
-
-message GetSnapshotDiffReportRequestProto {
-  required string snapshotRoot = 1;
-  required string fromSnapshot = 2;
-  required string toSnapshot = 3;
-}
-message GetSnapshotDiffReportResponseProto {
-  required SnapshotDiffReportProto diffReport = 1;
-}
-
-message RenewLeaseRequestProto {
-  required string clientName = 1;
-}
-
-message RenewLeaseResponseProto { //void response
-}
-
-message RecoverLeaseRequestProto {
-  required string src = 1;
-  required string clientName = 2;
-}
-message RecoverLeaseResponseProto {
-  required bool result = 1;
-}
-
-message GetFsStatusRequestProto { // no input paramters
-}
-
-message GetFsStatsResponseProto {
-  required uint64 capacity = 1;
-  required uint64 used = 2;
-  required uint64 remaining = 3;
-  required uint64 under_replicated = 4;
-  required uint64 corrupt_blocks = 5;
-  required uint64 missing_blocks = 6;
-  optional uint64 missing_repl_one_blocks = 7;
-}
-
-enum DatanodeReportTypeProto {  // type of the datanode report
-  ALL = 1;
-  LIVE = 2;
-  DEAD = 3;
-  DECOMMISSIONING = 4;
-}
-
-message GetDatanodeReportRequestProto {
-  required DatanodeReportTypeProto type = 1;
-}
-
-message GetDatanodeReportResponseProto {
-  repeated DatanodeInfoProto di = 1;
-}
-
-message GetDatanodeStorageReportRequestProto {
-  required DatanodeReportTypeProto type = 1;
-}
-
-message DatanodeStorageReportProto {
-  required DatanodeInfoProto datanodeInfo = 1;
-  repeated StorageReportProto storageReports = 2;
-}
-
-message GetDatanodeStorageReportResponseProto {
-  repeated DatanodeStorageReportProto datanodeStorageReports = 1;
-}
-
-message GetPreferredBlockSizeRequestProto {
-  required string filename = 1;
-}
-
-message GetPreferredBlockSizeResponseProto {
-  required uint64 bsize = 1;
-}
-
-enum SafeModeActionProto {
-  SAFEMODE_LEAVE = 1;
-  SAFEMODE_ENTER = 2;
-  SAFEMODE_GET = 3;
-}
-
-message SetSafeModeRequestProto {
-  required SafeModeActionProto action = 1;
-  optional bool checked = 2 [default = false];
-}
-
-message SetSafeModeResponseProto {
-  required bool result = 1;
-}
-
-message SaveNamespaceRequestProto {
-  optional uint64 timeWindow = 1 [default = 0];
-  optional uint64 txGap = 2 [default = 0];
-}
-
-message SaveNamespaceResponseProto { // void response
-  optional bool saved = 1 [default = true];
-}
-
-message RollEditsRequestProto { // no parameters
-}
-
-message RollEditsResponseProto { // response
-  required uint64 newSegmentTxId = 1;
-}
-
-message RestoreFailedStorageRequestProto {
-  required string arg = 1;
-}
-
-message RestoreFailedStorageResponseProto {
-    required bool result = 1;
-}
-
-message RefreshNodesRequestProto { // no parameters
-}
-
-message RefreshNodesResponseProto { // void response
-}
-
-message FinalizeUpgradeRequestProto { // no parameters
-}
-
-message FinalizeUpgradeResponseProto { // void response
-}
-
-enum RollingUpgradeActionProto {
-  QUERY = 1;
-  START = 2;
-  FINALIZE = 3;
-}
-
-message RollingUpgradeRequestProto {
-  required RollingUpgradeActionProto action = 1;
-}
-
-message RollingUpgradeInfoProto {
-  required RollingUpgradeStatusProto status = 1;
-  required uint64 startTime = 2;
-  required uint64 finalizeTime = 3;
-  required bool createdRollbackImages = 4;
-}
-
-message RollingUpgradeResponseProto {
-  optional RollingUpgradeInfoProto rollingUpgradeInfo= 1;
-}
-
-message ListCorruptFileBlocksRequestProto {
-  required string path = 1;
-  optional string cookie = 2;
-}
-
-message ListCorruptFileBlocksResponseProto {
-  required CorruptFileBlocksProto corrupt = 1;
-}
-
-message MetaSaveRequestProto {
-  required string filename = 1;
-}
-
-message MetaSaveResponseProto { // void response
-}
-
-message GetFileInfoRequestProto {
-  required string src = 1;
-}
-
-message GetFileInfoResponseProto {
-  optional HdfsFileStatusProto fs = 1;
-}
-
-message IsFileClosedRequestProto {
-  required string src = 1;
-}
-
-message IsFileClosedResponseProto {
-  required bool result = 1;
-}
-
-message CacheDirectiveInfoProto {
-  optional int64 id = 1;
-  optional string path = 2;
-  optional uint32 replication = 3;
-  optional string pool = 4;
-  optional CacheDirectiveInfoExpirationProto expiration = 5;
-}
-
-message CacheDirectiveInfoExpirationProto {
-  required int64 millis = 1;
-  required bool isRelative = 2;
-}
-
-message CacheDirectiveStatsProto {
-  required int64 bytesNeeded = 1;
-  required int64 bytesCached = 2;
-  required int64 filesNeeded = 3;
-  required int64 filesCached = 4;
-  required bool hasExpired = 5;
-}
-
-enum CacheFlagProto {
-  FORCE = 0x01;    // Ignore pool resource limits
-}
-
-message AddCacheDirectiveRequestProto {
-  required CacheDirectiveInfoProto info = 1;
-  optional uint32 cacheFlags = 2;  // bits set using CacheFlag
-}
-
-message AddCacheDirectiveResponseProto {
-  required int64 id = 1;
-}
-
-message ModifyCacheDirectiveRequestProto {
-  required CacheDirectiveInfoProto info = 1;
-  optional uint32 cacheFlags = 2;  // bits set using CacheFlag
-}
-
-message ModifyCacheDirectiveResponseProto {
-}
-
-message RemoveCacheDirectiveRequestProto {
-  required int64 id = 1;
-}
-
-message RemoveCacheDirectiveResponseProto {
-}
-
-message ListCacheDirectivesRequestProto {
-  required int64 prevId = 1;
-  required CacheDirectiveInfoProto filter = 2;
-}
-
-message CacheDirectiveEntryProto {
-  required CacheDirectiveInfoProto info = 1;
-  required CacheDirectiveStatsProto stats = 2;
-}
-
-message ListCacheDirectivesResponseProto {
-  repeated CacheDirectiveEntryProto elements = 1;
-  required bool hasMore = 2;
-}
-
-message CachePoolInfoProto {
-  optional string poolName = 1;
-  optional string ownerName = 2;
-  optional string groupName = 3;
-  optional int32 mode = 4;
-  optional int64 limit = 5;
-  optional int64 maxRelativeExpiry = 6;
-}
-
-message CachePoolStatsProto {
-  required int64 bytesNeeded = 1;
-  required int64 bytesCached = 2;
-  required int64 bytesOverlimit = 3;
-  required int64 filesNeeded = 4;
-  required int64 filesCached = 5;
-}
-
-message AddCachePoolRequestProto {
-  required CachePoolInfoProto info = 1;
-}
-
-message AddCachePoolResponseProto { // void response
-}
-
-message ModifyCachePoolRequestProto {
-  required CachePoolInfoProto info = 1;
-}
-
-message ModifyCachePoolResponseProto { // void response
-}
-
-message RemoveCachePoolRequestProto {
-  required string poolName = 1;
-}
-
-message RemoveCachePoolResponseProto { // void response
-}
-
-message ListCachePoolsRequestProto {
-  required string prevPoolName = 1;
-}
-
-message ListCachePoolsResponseProto {
-  repeated CachePoolEntryProto entries = 1;
-  required bool hasMore = 2;
-}
-
-message CachePoolEntryProto {
-  required CachePoolInfoProto info = 1;
-  required CachePoolStatsProto stats = 2;
-}
-
-message GetFileLinkInfoRequestProto {
-  required string src = 1;
-}
-
-message GetFileLinkInfoResponseProto {
-  optional HdfsFileStatusProto fs = 1;
-}
-
-message GetContentSummaryRequestProto {
-  required string path = 1;
-}
-
-message GetContentSummaryResponseProto {
-  required ContentSummaryProto summary = 1;
-}
-
-message SetQuotaRequestProto {
-  required string path = 1;
-  required uint64 namespaceQuota = 2;
-  required uint64 storagespaceQuota = 3;
-  optional StorageTypeProto storageType = 4;
-}
-
-message SetQuotaResponseProto { // void response
-}
-
-message FsyncRequestProto {
-  required string src = 1;
-  required string client = 2;
-  optional sint64 lastBlockLength = 3 [default = -1];
-  optional uint64 fileId = 4 [default = 0];  // default to GRANDFATHER_INODE_ID
-}
-
-message FsyncResponseProto { // void response
-}
-
-message SetTimesRequestProto {
-  required string src = 1;
-  required uint64 mtime = 2;
-  required uint64 atime = 3;
-}
-
-message SetTimesResponseProto { // void response
-}
-
-message CreateSymlinkRequestProto {
-  required string target = 1;
-  required string link = 2;
-  required FsPermissionProto dirPerm = 3;
-  required bool createParent = 4;
-}
-
-message CreateSymlinkResponseProto { // void response
-}
-
-message GetLinkTargetRequestProto {
-  required string path = 1;
-}
-message GetLinkTargetResponseProto {
-  optional string targetPath = 1;
-}
-
-message UpdateBlockForPipelineRequestProto {
-  required ExtendedBlockProto block = 1;
-  required string clientName = 2;
-}
-
-message UpdateBlockForPipelineResponseProto {
-  required LocatedBlockProto block = 1;
-}
-
-message UpdatePipelineRequestProto {
-  required string clientName = 1;
-  required ExtendedBlockProto oldBlock = 2;
-  required ExtendedBlockProto newBlock = 3;
-  repeated DatanodeIDProto newNodes = 4;
-  repeated string storageIDs = 5;
-}
-
-message UpdatePipelineResponseProto { // void response
-}
-
-message SetBalancerBandwidthRequestProto {
-  required int64 bandwidth = 1;
-}
-
-message SetBalancerBandwidthResponseProto { // void response
-}
-
-message GetDataEncryptionKeyRequestProto { // no parameters
-}
-
-message GetDataEncryptionKeyResponseProto {
-  optional DataEncryptionKeyProto dataEncryptionKey = 1;
-}
-
-message CreateSnapshotRequestProto {
-  required string snapshotRoot = 1;
-  optional string snapshotName = 2;
-}
-
-message CreateSnapshotResponseProto {
-  required string snapshotPath = 1;
-}
-
-message RenameSnapshotRequestProto {
-  required string snapshotRoot = 1;
-  required string snapshotOldName = 2;
-  required string snapshotNewName = 3;
-}
-
-message RenameSnapshotResponseProto { // void response
-}
-
-message AllowSnapshotRequestProto {
-  required string snapshotRoot = 1;
-}
-
-message AllowSnapshotResponseProto {
-}
-
-message DisallowSnapshotRequestProto {
-  required string snapshotRoot = 1;
-}
-
-message DisallowSnapshotResponseProto {
-}
-
-message DeleteSnapshotRequestProto {
-  required string snapshotRoot = 1;
-  required string snapshotName = 2;
-}
-
-message DeleteSnapshotResponseProto { // void response
-}
-
-message CheckAccessRequestProto {
-  required string path = 1;
-  required AclEntryProto.FsActionProto mode = 2;
-}
-
-message CheckAccessResponseProto { // void response
-}
-
-message GetCurrentEditLogTxidRequestProto {
-}
-
-message GetCurrentEditLogTxidResponseProto {
-  required int64 txid = 1;
-}
-
-message GetEditsFromTxidRequestProto {
-  required int64 txid = 1;
-}
-
-message GetEditsFromTxidResponseProto {
-  required EventsListProto eventsList = 1;
-}
-
-service ClientNamenodeProtocol {
-  rpc getBlockLocations(GetBlockLocationsRequestProto)
-      returns(GetBlockLocationsResponseProto);
-  rpc getServerDefaults(GetServerDefaultsRequestProto)
-      returns(GetServerDefaultsResponseProto);
-  rpc create(CreateRequestProto)returns(CreateResponseProto);
-  rpc append(AppendRequestProto) returns(AppendResponseProto);
-  rpc setReplication(SetReplicationRequestProto)
-      returns(SetReplicationResponseProto);
-  rpc setStoragePolicy(SetStoragePolicyRequestProto)
-      returns(SetStoragePolicyResponseProto);
-  rpc getStoragePolicies(GetStoragePoliciesRequestProto)
-      returns(GetStoragePoliciesResponseProto);
-  rpc setPermission(SetPermissionRequestProto)
-      returns(SetPermissionResponseProto);
-  rpc setOwner(SetOwnerRequestProto) returns(SetOwnerResponseProto);
-  rpc abandonBlock(AbandonBlockRequestProto) returns(AbandonBlockResponseProto);
-  rpc addBlock(AddBlockRequestProto) returns(AddBlockResponseProto);
-  rpc getAdditionalDatanode(GetAdditionalDatanodeRequestProto)
-      returns(GetAdditionalDatanodeResponseProto);
-  rpc complete(CompleteRequestProto) returns(CompleteResponseProto);
-  rpc reportBadBlocks(ReportBadBlocksRequestProto)
-      returns(ReportBadBlocksResponseProto);
-  rpc concat(ConcatRequestProto) returns(ConcatResponseProto);
-  rpc truncate(TruncateRequestProto) returns(TruncateResponseProto);
-  rpc rename(RenameRequestProto) returns(RenameResponseProto);
-  rpc rename2(Rename2RequestProto) returns(Rename2ResponseProto);
-  rpc delete(DeleteRequestProto) returns(DeleteResponseProto);
-  rpc mkdirs(MkdirsRequestProto) returns(MkdirsResponseProto);
-  rpc getListing(GetListingRequestProto) returns(GetListingResponseProto);
-  rpc renewLease(RenewLeaseRequestProto) returns(RenewLeaseResponseProto);
-  rpc recoverLease(RecoverLeaseRequestProto)
-      returns(RecoverLeaseResponseProto);
-  rpc getFsStats(GetFsStatusRequestProto) returns(GetFsStatsResponseProto);
-  rpc getDatanodeReport(GetDatanodeReportRequestProto)
-      returns(GetDatanodeReportResponseProto);
-  rpc getDatanodeStorageReport(GetDatanodeStorageReportRequestProto)
-      returns(GetDatanodeStorageReportResponseProto);
-  rpc getPreferredBlockSize(GetPreferredBlockSizeRequestProto)
-      returns(GetPreferredBlockSizeResponseProto);
-  rpc setSafeMode(SetSafeModeRequestProto)
-      returns(SetSafeModeResponseProto);
-  rpc saveNamespace(SaveNamespaceRequestProto)
-      returns(SaveNamespaceResponseProto);
-  rpc rollEdits(RollEditsRequestProto)
-      returns(RollEditsResponseProto);
-  rpc restoreFailedStorage(RestoreFailedStorageRequestProto)
-      returns(RestoreFailedStorageResponseProto);
-  rpc refreshNodes(RefreshNodesRequestProto) returns(RefreshNodesResponseProto);
-  rpc finalizeUpgrade(FinalizeUpgradeRequestProto)
-      returns(FinalizeUpgradeResponseProto);
-  rpc rollingUpgrade(RollingUpgradeRequestProto)
-      returns(RollingUpgradeResponseProto);
-  rpc listCorruptFileBlocks(ListCorruptFileBlocksRequestProto)
-      returns(ListCorruptFileBlocksResponseProto);
-  rpc metaSave(MetaSaveRequestProto) returns(MetaSaveResponseProto);
-  rpc getFileInfo(GetFileInfoRequestProto) returns(GetFileInfoResponseProto);
-  rpc addCacheDirective(AddCacheDirectiveRequestProto)
-      returns (AddCacheDirectiveResponseProto);
-  rpc modifyCacheDirective(ModifyCacheDirectiveRequestProto)
-      returns (ModifyCacheDirectiveResponseProto);
-  rpc removeCacheDirective(RemoveCacheDirectiveRequestProto)
-      returns (RemoveCacheDirectiveResponseProto);
-  rpc listCacheDirectives(ListCacheDirectivesRequestProto)
-      returns (ListCacheDirectivesResponseProto);
-  rpc addCachePool(AddCachePoolRequestProto)
-      returns(AddCachePoolResponseProto);
-  rpc modifyCachePool(ModifyCachePoolRequestProto)
-      returns(ModifyCachePoolResponseProto);
-  rpc removeCachePool(RemoveCachePoolRequestProto)
-      returns(RemoveCachePoolResponseProto);
-  rpc listCachePools(ListCachePoolsRequestProto)
-      returns(ListCachePoolsResponseProto);
-  rpc getFileLinkInfo(GetFileLinkInfoRequestProto)
-      returns(GetFileLinkInfoResponseProto);
-  rpc getContentSummary(GetContentSummaryRequestProto)
-      returns(GetContentSummaryResponseProto);
-  rpc setQuota(SetQuotaRequestProto) returns(SetQuotaResponseProto);
-  rpc fsync(FsyncRequestProto) returns(FsyncResponseProto);
-  rpc setTimes(SetTimesRequestProto) returns(SetTimesResponseProto);
-  rpc createSymlink(CreateSymlinkRequestProto)
-      returns(CreateSymlinkResponseProto);
-  rpc getLinkTarget(GetLinkTargetRequestProto)
-      returns(GetLinkTargetResponseProto);
-  rpc updateBlockForPipeline(UpdateBlockForPipelineRequestProto)
-      returns(UpdateBlockForPipelineResponseProto);
-  rpc updatePipeline(UpdatePipelineRequestProto)
-      returns(UpdatePipelineResponseProto);
-  rpc getDelegationToken(hadoop.common.GetDelegationTokenRequestProto)
-      returns(hadoop.common.GetDelegationTokenResponseProto);
-  rpc renewDelegationToken(hadoop.common.RenewDelegationTokenRequestProto)
-      returns(hadoop.common.RenewDelegationTokenResponseProto);
-  rpc cancelDelegationToken(hadoop.common.CancelDelegationTokenRequestProto)
-      returns(hadoop.common.CancelDelegationTokenResponseProto);
-  rpc setBalancerBandwidth(SetBalancerBandwidthRequestProto)
-      returns(SetBalancerBandwidthResponseProto);
-  rpc getDataEncryptionKey(GetDataEncryptionKeyRequestProto)
-      returns(GetDataEncryptionKeyResponseProto);
-  rpc createSnapshot(CreateSnapshotRequestProto)
-      returns(CreateSnapshotResponseProto);
-  rpc renameSnapshot(RenameSnapshotRequestProto)
-      returns(RenameSnapshotResponseProto);
-  rpc allowSnapshot(AllowSnapshotRequestProto)
-      returns(AllowSnapshotResponseProto);
-  rpc disallowSnapshot(DisallowSnapshotRequestProto)
-      returns(DisallowSnapshotResponseProto);   
-  rpc getSnapshottableDirListing(GetSnapshottableDirListingRequestProto)
-      returns(GetSnapshottableDirListingResponseProto);
-  rpc deleteSnapshot(DeleteSnapshotRequestProto)
-      returns(DeleteSnapshotResponseProto);
-  rpc getSnapshotDiffReport(GetSnapshotDiffReportRequestProto)
-      returns(GetSnapshotDiffReportResponseProto);
-  rpc isFileClosed(IsFileClosedRequestProto)
-      returns(IsFileClosedResponseProto);
-  rpc modifyAclEntries(ModifyAclEntriesRequestProto)
-      returns(ModifyAclEntriesResponseProto);
-  rpc removeAclEntries(RemoveAclEntriesRequestProto)
-      returns(RemoveAclEntriesResponseProto);
-  rpc removeDefaultAcl(RemoveDefaultAclRequestProto)
-      returns(RemoveDefaultAclResponseProto);
-  rpc removeAcl(RemoveAclRequestProto)
-      returns(RemoveAclResponseProto);
-  rpc setAcl(SetAclRequestProto)
-      returns(SetAclResponseProto);
-  rpc getAclStatus(GetAclStatusRequestProto)
-      returns(GetAclStatusResponseProto);
-  rpc setXAttr(SetXAttrRequestProto)
-      returns(SetXAttrResponseProto);
-  rpc getXAttrs(GetXAttrsRequestProto)
-      returns(GetXAttrsResponseProto);
-  rpc listXAttrs(ListXAttrsRequestProto)
-      returns(ListXAttrsResponseProto);
-  rpc removeXAttr(RemoveXAttrRequestProto)
-      returns(RemoveXAttrResponseProto);
-  rpc checkAccess(CheckAccessRequestProto)
-      returns(CheckAccessResponseProto);
-  rpc createEncryptionZone(CreateEncryptionZoneRequestProto)
-      returns(CreateEncryptionZoneResponseProto);
-  rpc listEncryptionZones(ListEncryptionZonesRequestProto)
-      returns(ListEncryptionZonesResponseProto);
-  rpc getEZForPath(GetEZForPathRequestProto)
-      returns(GetEZForPathResponseProto);
-  rpc getCurrentEditLogTxid(GetCurrentEditLogTxidRequestProto)
-      returns(GetCurrentEditLogTxidResponseProto);
-  rpc getEditsFromTxid(GetEditsFromTxidRequestProto)
-      returns(GetEditsFromTxidResponseProto);
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/acl.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/acl.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/acl.proto
deleted file mode 100644
index 57cc855..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/acl.proto
+++ /dev/null
@@ -1,113 +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.
- */
-
-option java_package = "org.apache.hadoop.hdfs.protocol.proto";
-option java_outer_classname = "AclProtos";
-option java_generate_equals_and_hash = true;
-package hadoop.hdfs;
-
-import "hdfs.proto";
-
-message AclEntryProto {
-  enum AclEntryScopeProto {
-    ACCESS  = 0x0;
-    DEFAULT = 0x1;
-  }
-
-  enum AclEntryTypeProto {
-    USER  = 0x0;
-    GROUP = 0x1;
-    MASK  = 0x2;
-    OTHER = 0x3;
-  }
-
-  enum FsActionProto {
-    NONE          = 0x0;
-    EXECUTE       = 0x1;
-    WRITE         = 0x2;
-    WRITE_EXECUTE = 0x3;
-    READ          = 0x4;
-    READ_EXECUTE  = 0x5;
-    READ_WRITE    = 0x6;
-    PERM_ALL      = 0x7;
-  }
-
-  required AclEntryTypeProto type    = 1;
-  required AclEntryScopeProto scope  = 2;
-  required FsActionProto permissions = 3;
-  optional string name               = 4;
-}
-
-message AclStatusProto {
-  required string owner          = 1;
-  required string group          = 2;
-  required bool sticky           = 3;
-  repeated AclEntryProto entries = 4;
-  optional FsPermissionProto permission = 5;
-}
-
-message AclEditLogProto {
-  required string src            = 1;
-  repeated AclEntryProto entries = 2;
-}
-
-message ModifyAclEntriesRequestProto {
-  required string src            = 1;
-  repeated AclEntryProto aclSpec = 2;
-}
-
-message ModifyAclEntriesResponseProto {
-}
-
-message RemoveAclRequestProto {
-  required string src = 1;
-}
-
-message RemoveAclResponseProto {
-}
-
-message RemoveAclEntriesRequestProto {
-  required string src            = 1;
-  repeated AclEntryProto aclSpec = 2;
-}
-
-message RemoveAclEntriesResponseProto {
-}
-
-message RemoveDefaultAclRequestProto {
-  required string src = 1;
-}
-
-message RemoveDefaultAclResponseProto {
-}
-
-message SetAclRequestProto {
-  required string src            = 1;
-  repeated AclEntryProto aclSpec = 2;
-}
-
-message SetAclResponseProto {
-}
-
-message GetAclStatusRequestProto {
-  required string src = 1;
-}
-
-message GetAclStatusResponseProto {
-  required AclStatusProto result = 1;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
deleted file mode 100644
index 5071d15..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
+++ /dev/null
@@ -1,304 +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.
- */
-
-/**
- * These .proto interfaces are private and stable.
- * Please see http://wiki.apache.org/hadoop/Compatibility
- * for what changes are allowed for a *stable* .proto interface.
- */
-
-// This file contains protocol buffers that are used to transfer data
-// to and from the datanode, as well as between datanodes.
-
-option java_package = "org.apache.hadoop.hdfs.protocol.proto";
-option java_outer_classname = "DataTransferProtos";
-option java_generate_equals_and_hash = true;
-package hadoop.hdfs;
-
-import "Security.proto";
-import "hdfs.proto";
-
-message DataTransferEncryptorMessageProto {
-  enum DataTransferEncryptorStatus {
-    SUCCESS = 0;
-    ERROR_UNKNOWN_KEY = 1;
-    ERROR = 2;
-  }
-  required DataTransferEncryptorStatus status = 1;
-  optional bytes payload = 2;
-  optional string message = 3;
-  repeated CipherOptionProto cipherOption = 4;
-}
-
-message BaseHeaderProto {
-  required ExtendedBlockProto block = 1;
-  optional hadoop.common.TokenProto token = 2;
-  optional DataTransferTraceInfoProto traceInfo = 3;
-}
-
-message DataTransferTraceInfoProto {
-  required uint64 traceId = 1;
-  required uint64 parentId = 2;
-}
-
-message ClientOperationHeaderProto {
-  required BaseHeaderProto baseHeader = 1;
-  required string clientName = 2;
-}
-
-message CachingStrategyProto {
-  optional bool dropBehind = 1;
-  optional int64 readahead = 2;
-}
-
-message OpReadBlockProto {
-  required ClientOperationHeaderProto header = 1;
-  required uint64 offset = 2;
-  required uint64 len = 3;
-  optional bool sendChecksums = 4 [default = true];
-  optional CachingStrategyProto cachingStrategy = 5;
-}
-
-
-message ChecksumProto {
-  required ChecksumTypeProto type = 1;
-  required uint32 bytesPerChecksum = 2;
-}
-  
-message OpWriteBlockProto {
-  required ClientOperationHeaderProto header = 1;
-  repeated DatanodeInfoProto targets = 2;
-  optional DatanodeInfoProto source = 3;
-  enum BlockConstructionStage {
-    PIPELINE_SETUP_APPEND = 0;
-    // pipeline set up for failed PIPELINE_SETUP_APPEND recovery
-    PIPELINE_SETUP_APPEND_RECOVERY = 1;
-    // data streaming
-    DATA_STREAMING = 2;
-    // pipeline setup for failed data streaming recovery
-    PIPELINE_SETUP_STREAMING_RECOVERY = 3;
-    // close the block and pipeline
-    PIPELINE_CLOSE = 4;
-    // Recover a failed PIPELINE_CLOSE
-    PIPELINE_CLOSE_RECOVERY = 5;
-    // pipeline set up for block creation
-    PIPELINE_SETUP_CREATE = 6;
-    // transfer RBW for adding datanodes
-    TRANSFER_RBW = 7;
-    // transfer Finalized for adding datanodes
-    TRANSFER_FINALIZED = 8;
-  }
-  required BlockConstructionStage stage = 4;
-  required uint32 pipelineSize = 5;
-  required uint64 minBytesRcvd = 6;
-  required uint64 maxBytesRcvd = 7;
-  required uint64 latestGenerationStamp = 8;
-
-  /**
-   * The requested checksum mechanism for this block write.
-   */
-  required ChecksumProto requestedChecksum = 9;
-  optional CachingStrategyProto cachingStrategy = 10;
-  optional StorageTypeProto storageType = 11 [default = DISK];
-  repeated StorageTypeProto targetStorageTypes = 12;
-
-  /**
-   * Hint to the DataNode that the block can be allocated on transient
-   * storage i.e. memory and written to disk lazily. The DataNode is free
-   * to ignore this hint.
-   */
-  optional bool allowLazyPersist = 13 [default = false];
-  //whether to pin the block, so Balancer won't move it.
-  optional bool pinning = 14 [default = false];
-  repeated bool targetPinnings = 15;
-}
-  
-message OpTransferBlockProto {
-  required ClientOperationHeaderProto header = 1;
-  repeated DatanodeInfoProto targets = 2;
-  repeated StorageTypeProto targetStorageTypes = 3;
-}
-
-message OpReplaceBlockProto {
-  required BaseHeaderProto header = 1;
-  required string delHint = 2;
-  required DatanodeInfoProto source = 3;
-  optional StorageTypeProto storageType = 4 [default = DISK];
-}
-
-message OpCopyBlockProto {
-  required BaseHeaderProto header = 1;
-}
-
-message OpBlockChecksumProto { 
-  required BaseHeaderProto header = 1;
-}
-
-/**
- * An ID uniquely identifying a shared memory segment.
- */
-message ShortCircuitShmIdProto { 
-  required int64 hi = 1;
-  required int64 lo = 2;
-}
-
-/**
- * An ID uniquely identifying a slot within a shared memory segment.
- */
-message ShortCircuitShmSlotProto {
-  required ShortCircuitShmIdProto shmId = 1;
-  required int32 slotIdx = 2; 
-}
-
-message OpRequestShortCircuitAccessProto { 
-  required BaseHeaderProto header = 1;
-
-  /** In order to get short-circuit access to block data, clients must set this
-   * to the highest version of the block data that they can understand.
-   * Currently 1 is the only version, but more versions may exist in the future
-   * if the on-disk format changes.
-   */
-  required uint32 maxVersion = 2;
-
-  /**
-   * The shared memory slot to use, if we are using one.
-   */
-  optional ShortCircuitShmSlotProto slotId = 3;
-
-  /**
-   * True if the client supports verifying that the file descriptor has been
-   * sent successfully.
-   */
-  optional bool supportsReceiptVerification = 4 [default = false];
-}
-
-message ReleaseShortCircuitAccessRequestProto {
-  required ShortCircuitShmSlotProto slotId = 1;
-  optional DataTransferTraceInfoProto traceInfo = 2;
-}
-
-message ReleaseShortCircuitAccessResponseProto {
-  required Status status = 1;
-  optional string error = 2;
-}
-
-message ShortCircuitShmRequestProto { 
-  // The name of the client requesting the shared memory segment.  This is
-  // purely for logging / debugging purposes.
-  required string clientName = 1;
-  optional DataTransferTraceInfoProto traceInfo = 2;
-}
-
-message ShortCircuitShmResponseProto { 
-  required Status status = 1;
-  optional string error = 2;
-  optional ShortCircuitShmIdProto id = 3;
-}
-
-message PacketHeaderProto {
-  // All fields must be fixed-length!
-  required sfixed64 offsetInBlock = 1;
-  required sfixed64 seqno = 2;
-  required bool lastPacketInBlock = 3;
-  required sfixed32 dataLen = 4;
-  optional bool syncBlock = 5 [default = false];
-}
-
-// Status is a 4-bit enum
-enum Status {
-  SUCCESS = 0;
-  ERROR = 1;
-  ERROR_CHECKSUM = 2;
-  ERROR_INVALID = 3;
-  ERROR_EXISTS = 4;
-  ERROR_ACCESS_TOKEN = 5;
-  CHECKSUM_OK = 6;
-  ERROR_UNSUPPORTED = 7;
-  OOB_RESTART = 8;            // Quick restart
-  OOB_RESERVED1 = 9;          // Reserved
-  OOB_RESERVED2 = 10;         // Reserved
-  OOB_RESERVED3 = 11;         // Reserved
-  IN_PROGRESS = 12;
-}
-
-enum ShortCircuitFdResponse {
-  DO_NOT_USE_RECEIPT_VERIFICATION = 0;
-  USE_RECEIPT_VERIFICATION = 1;
-}
-
-message PipelineAckProto {
-  required sint64 seqno = 1;
-  repeated Status reply = 2;
-  optional uint64 downstreamAckTimeNanos = 3 [default = 0];
-  repeated uint32 flag = 4 [packed=true];
-}
-
-/**
- * Sent as part of the BlockOpResponseProto
- * for READ_BLOCK and COPY_BLOCK operations.
- */
-message ReadOpChecksumInfoProto {
-  required ChecksumProto checksum = 1;
-
-  /**
-   * The offset into the block at which the first packet
-   * will start. This is necessary since reads will align
-   * backwards to a checksum chunk boundary.
-   */
-  required uint64 chunkOffset = 2;
-}
-
-message BlockOpResponseProto {
-  required Status status = 1;
-
-  optional string firstBadLink = 2;
-  optional OpBlockChecksumResponseProto checksumResponse = 3;
-  optional ReadOpChecksumInfoProto readOpChecksumInfo = 4;
-
-  /** explanatory text which may be useful to log on the client side */
-  optional string message = 5;
-
-  /** If the server chooses to agree to the request of a client for
-   * short-circuit access, it will send a response message with the relevant
-   * file descriptors attached.
-   *
-   * In the body of the message, this version number will be set to the
-   * specific version number of the block data that the client is about to
-   * read.
-   */
-  optional uint32 shortCircuitAccessVersion = 6;
-}
-
-/**
- * Message sent from the client to the DN after reading the entire
- * read request.
- */
-message ClientReadStatusProto {
-  required Status status = 1;
-}
-
-message DNTransferAckProto {
-  required Status status = 1;
-}
-
-message OpBlockChecksumResponseProto {
-  required uint32 bytesPerCrc = 1;
-  required uint64 crcPerBlock = 2;
-  required bytes md5 = 3;
-  optional ChecksumTypeProto crcType = 4;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/editlog.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/editlog.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/editlog.proto
new file mode 100644
index 0000000..f25fe59
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/editlog.proto
@@ -0,0 +1,35 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "EditLogProtos";
+option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
+
+import "acl.proto";
+import "xattr.proto";
+
+message AclEditLogProto {
+  required string src            = 1;
+  repeated AclEntryProto entries = 2;
+}
+
+message XAttrEditLogProto {
+  optional string src = 1;
+  repeated XAttrProto xAttrs = 2;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/encryption.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/encryption.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/encryption.proto
deleted file mode 100644
index 68b2f3a..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/encryption.proto
+++ /dev/null
@@ -1,67 +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.
- */
-
-/**
- * These .proto interfaces are private and stable.
- * Please see http://wiki.apache.org/hadoop/Compatibility
- * for what changes are allowed for a *stable* .proto interface.
- */
-
-// This file contains protocol buffers that are used throughout HDFS -- i.e.
-// by the client, server, and data transfer protocols.
-
-
-option java_package = "org.apache.hadoop.hdfs.protocol.proto";
-option java_outer_classname = "EncryptionZonesProtos";
-option java_generate_equals_and_hash = true;
-package hadoop.hdfs;
-
-import "hdfs.proto";
-
-message CreateEncryptionZoneRequestProto {
-  required string src = 1;
-  optional string keyName = 2;
-}
-
-message CreateEncryptionZoneResponseProto {
-}
-
-message ListEncryptionZonesRequestProto {
-  required int64 id = 1;
-}
-
-message EncryptionZoneProto {
-  required int64 id = 1;
-  required string path = 2;
-  required CipherSuiteProto suite = 3;
-  required CryptoProtocolVersionProto cryptoProtocolVersion = 4;
-  required string keyName = 5;
-}
-
-message ListEncryptionZonesResponseProto {
-  repeated EncryptionZoneProto zones = 1;
-  required bool hasMore = 2;
-}
-
-message GetEZForPathRequestProto {
-    required string src = 1;
-}
-
-message GetEZForPathResponseProto {
-    optional EncryptionZoneProto zone = 1;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
deleted file mode 100644
index 86fb462..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ /dev/null
@@ -1,611 +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.
- */
-
-/**
- * These .proto interfaces are private and stable.
- * Please see http://wiki.apache.org/hadoop/Compatibility
- * for what changes are allowed for a *stable* .proto interface.
- */
-
-// This file contains protocol buffers that are used throughout HDFS -- i.e.
-// by the client, server, and data transfer protocols.
-
-
-option java_package = "org.apache.hadoop.hdfs.protocol.proto";
-option java_outer_classname = "HdfsProtos";
-option java_generate_equals_and_hash = true;
-package hadoop.hdfs;
-
-import "Security.proto";
-
-/**
- * Extended block idenfies a block
- */
-message ExtendedBlockProto {
-  required string poolId = 1;   // Block pool id - gloablly unique across clusters
-  required uint64 blockId = 2;  // the local id within a pool
-  required uint64 generationStamp = 3;
-  optional uint64 numBytes = 4 [default = 0];  // len does not belong in ebid 
-                                               // here for historical reasons
-}
-
-/**
- * Identifies a Datanode
- */
-message DatanodeIDProto {
-  required string ipAddr = 1;    // IP address
-  required string hostName = 2;  // hostname
-  required string datanodeUuid = 3;     // UUID assigned to the Datanode. For
-                                        // upgraded clusters this is the same
-                                        // as the original StorageID of the
-                                        // Datanode.
-  required uint32 xferPort = 4;  // data streaming port
-  required uint32 infoPort = 5;  // datanode http port
-  required uint32 ipcPort = 6;   // ipc server port
-  optional uint32 infoSecurePort = 7 [default = 0]; // datanode https port
-}
-
-/**
- * Datanode local information
- */
-message DatanodeLocalInfoProto {
-  required string softwareVersion = 1;
-  required string configVersion = 2;
-  required uint64 uptime = 3;
-}
-
-/**
- * DatanodeInfo array
- */
-message DatanodeInfosProto {
-  repeated DatanodeInfoProto datanodes = 1;
-}
-
-/**
- * The status of a Datanode
- */
-message DatanodeInfoProto {
-  required DatanodeIDProto id = 1;
-  optional uint64 capacity = 2 [default = 0];
-  optional uint64 dfsUsed = 3 [default = 0];
-  optional uint64 remaining = 4 [default = 0];
-  optional uint64 blockPoolUsed = 5 [default = 0];
-  optional uint64 lastUpdate = 6 [default = 0];
-  optional uint32 xceiverCount = 7 [default = 0];
-  optional string location = 8;
-  enum AdminState {
-    NORMAL = 0;
-    DECOMMISSION_INPROGRESS = 1;
-    DECOMMISSIONED = 2;
-  }
-
-  optional AdminState adminState = 10 [default = NORMAL];
-  optional uint64 cacheCapacity = 11 [default = 0];
-  optional uint64 cacheUsed = 12 [default = 0];
-  optional uint64 lastUpdateMonotonic = 13 [default = 0];
-}
-
-/**
- * Represents a storage available on the datanode
- */
-message DatanodeStorageProto {
-  enum StorageState {
-    NORMAL = 0;
-    READ_ONLY_SHARED = 1;
-  }
-
-  required string storageUuid = 1;
-  optional StorageState state = 2 [default = NORMAL];
-  optional StorageTypeProto storageType = 3 [default = DISK];
-}
-
-message StorageReportProto {
-  required string storageUuid = 1 [ deprecated = true ];
-  optional bool failed = 2 [ default = false ];
-  optional uint64 capacity = 3 [ default = 0 ];
-  optional uint64 dfsUsed = 4 [ default = 0 ];
-  optional uint64 remaining = 5 [ default = 0 ];
-  optional uint64 blockPoolUsed = 6 [ default = 0 ];
-  optional DatanodeStorageProto storage = 7; // supersedes StorageUuid
-}
-
-/**
- * Summary of a file or directory
- */
-message ContentSummaryProto {
-  required uint64 length = 1;
-  required uint64 fileCount = 2;
-  required uint64 directoryCount = 3;
-  required uint64 quota = 4;
-  required uint64 spaceConsumed = 5;
-  required uint64 spaceQuota = 6;
-  optional StorageTypeQuotaInfosProto typeQuotaInfos = 7;
-}
-
-/**
- * Storage type quota and usage information of a file or directory
- */
-message StorageTypeQuotaInfosProto {
-  repeated StorageTypeQuotaInfoProto typeQuotaInfo = 1;
-}
-
-message StorageTypeQuotaInfoProto {
-  required StorageTypeProto type = 1;
-  required uint64 quota = 2;
-  required uint64 consumed = 3;
-}
-
-/**
- * Contains a list of paths corresponding to corrupt files and a cookie
- * used for iterative calls to NameNode.listCorruptFileBlocks.
- *
- */
-message CorruptFileBlocksProto {
- repeated string files = 1;
- required string   cookie = 2;
-}
-
-/**
- * File or Directory permision - same spec as posix
- */
-message FsPermissionProto {
-  required uint32 perm = 1;       // Actually a short - only 16bits used
-}
-
-/**
- * Types of recognized storage media.
- */
-enum StorageTypeProto {
-  DISK = 1;
-  SSD = 2;
-  ARCHIVE = 3;
-  RAM_DISK = 4;
-}
-
-/**
- * A list of storage types. 
- */
-message StorageTypesProto {
-  repeated StorageTypeProto storageTypes = 1;
-}
-
-/**
- * Block replica storage policy.
- */
-message BlockStoragePolicyProto {
-  required uint32 policyId = 1;
-  required string name = 2;
-  // a list of storage types for storing the block replicas when creating a
-  // block.
-  required StorageTypesProto creationPolicy = 3;
-  // A list of storage types for creation fallback storage.
-  optional StorageTypesProto creationFallbackPolicy = 4;
-  optional StorageTypesProto replicationFallbackPolicy = 5;
-}
-
-/**
- * A list of storage IDs. 
- */
-message StorageUuidsProto {
-  repeated string storageUuids = 1;
-}
-
-/**
- * A LocatedBlock gives information about a block and its location.
- */ 
-message LocatedBlockProto {
-  required ExtendedBlockProto b  = 1;
-  required uint64 offset = 2;           // offset of first byte of block in the file
-  repeated DatanodeInfoProto locs = 3;  // Locations ordered by proximity to client ip
-  required bool corrupt = 4;            // true if all replicas of a block are corrupt, else false
-                                        // If block has few corrupt replicas, they are filtered and 
-                                        // their locations are not part of this object
-
-  required hadoop.common.TokenProto blockToken = 5;
-  repeated bool isCached = 6 [packed=true]; // if a location in locs is cached
-  repeated StorageTypeProto storageTypes = 7;
-  repeated string storageIDs = 8;
-}
-
-message DataEncryptionKeyProto {
-  required uint32 keyId = 1;
-  required string blockPoolId = 2;
-  required bytes nonce = 3;
-  required bytes encryptionKey = 4;
-  required uint64 expiryDate = 5;
-  optional string encryptionAlgorithm = 6;
-}
-
-/**
- * Cipher suite.
- */
-enum CipherSuiteProto {
-    UNKNOWN = 1;
-    AES_CTR_NOPADDING = 2;
-}
-
-/**
- * Crypto protocol version used to access encrypted files.
- */
-enum CryptoProtocolVersionProto {
-    UNKNOWN_PROTOCOL_VERSION = 1;
-    ENCRYPTION_ZONES = 2;
-}
-
-/**
- * Encryption information for a file.
- */
-message FileEncryptionInfoProto {
-  required CipherSuiteProto suite = 1;
-  required CryptoProtocolVersionProto cryptoProtocolVersion = 2;
-  required bytes key = 3;
-  required bytes iv = 4;
-  required string keyName = 5;
-  required string ezKeyVersionName = 6;
-}
-
-/**
- * Encryption information for an individual
- * file within an encryption zone
- */
-message PerFileEncryptionInfoProto {
-  required bytes key = 1;
-  required bytes iv = 2;
-  required string ezKeyVersionName = 3;
-}
-
-/**
- * Encryption information for an encryption
- * zone
- */
-message ZoneEncryptionInfoProto {
-  required CipherSuiteProto suite = 1;
-  required CryptoProtocolVersionProto cryptoProtocolVersion = 2;
-  required string keyName = 3;
-}
-
-/**
- * Cipher option
- */
-message CipherOptionProto {
-  required CipherSuiteProto suite = 1;
-  optional bytes inKey = 2;
-  optional bytes inIv = 3;
-  optional bytes outKey = 4;
-  optional bytes outIv = 5;
-}
-
-/**
- * A set of file blocks and their locations.
- */
-message LocatedBlocksProto {
-  required uint64 fileLength = 1;
-  repeated LocatedBlockProto blocks = 2;
-  required bool underConstruction = 3;
-  optional LocatedBlockProto lastBlock = 4;
-  required bool isLastBlockComplete = 5;
-  optional FileEncryptionInfoProto fileEncryptionInfo = 6;
-}
-
-/**
- * Status of a file, directory or symlink
- * Optionally includes a file's block locations if requested by client on the rpc call.
- */
-message HdfsFileStatusProto {
-  enum FileType {
-    IS_DIR = 1;
-    IS_FILE = 2;
-    IS_SYMLINK = 3;
-  }
-  required FileType fileType = 1;
-  required bytes path = 2;          // local name of inode encoded java UTF8
-  required uint64 length = 3;
-  required FsPermissionProto permission = 4;
-  required string owner = 5;
-  required string group = 6;
-  required uint64 modification_time = 7;
-  required uint64 access_time = 8;
-
-  // Optional fields for symlink
-  optional bytes symlink = 9;             // if symlink, target encoded java UTF8 
-
-  // Optional fields for file
-  optional uint32 block_replication = 10 [default = 0]; // only 16bits used
-  optional uint64 blocksize = 11 [default = 0];
-  optional LocatedBlocksProto locations = 12;  // suppled only if asked by client
-
-  // Optional field for fileId
-  optional uint64 fileId = 13 [default = 0]; // default as an invalid id
-  optional int32 childrenNum = 14 [default = -1];
-  // Optional field for file encryption
-  optional FileEncryptionInfoProto fileEncryptionInfo = 15;
-
-  optional uint32 storagePolicy = 16 [default = 0]; // block storage policy id
-} 
-
-/**
- * Checksum algorithms/types used in HDFS
- * Make sure this enum's integer values match enum values' id properties defined
- * in org.apache.hadoop.util.DataChecksum.Type
- */
-enum ChecksumTypeProto {
-  CHECKSUM_NULL = 0;
-  CHECKSUM_CRC32 = 1;
-  CHECKSUM_CRC32C = 2;
-}
-
-/**
- * HDFS Server Defaults
- */
-message FsServerDefaultsProto {
-  required uint64 blockSize = 1;
-  required uint32 bytesPerChecksum = 2;
-  required uint32 writePacketSize = 3;
-  required uint32 replication = 4; // Actually a short - only 16 bits used
-  required uint32 fileBufferSize = 5;
-  optional bool encryptDataTransfer = 6 [default = false];
-  optional uint64 trashInterval = 7 [default = 0];
-  optional ChecksumTypeProto checksumType = 8 [default = CHECKSUM_CRC32];
-}
-
-
-/**
- * Directory listing
- */
-message DirectoryListingProto {
-  repeated HdfsFileStatusProto partialListing = 1;
-  required uint32 remainingEntries  = 2;
-}
-
-/**
- * Status of a snapshottable directory: besides the normal information for 
- * a directory status, also include snapshot quota, number of snapshots, and
- * the full path of the parent directory. 
- */
-message SnapshottableDirectoryStatusProto {
-  required HdfsFileStatusProto dirStatus = 1;
-
-  // Fields specific for snapshottable directory
-  required uint32 snapshot_quota = 2;
-  required uint32 snapshot_number = 3;
-  required bytes parent_fullpath = 4;
-}
-
-/**
- * Snapshottable directory listing
- */
-message SnapshottableDirectoryListingProto {
-  repeated SnapshottableDirectoryStatusProto snapshottableDirListing = 1;
-}
-
-/**
- * Snapshot diff report entry
- */
-message SnapshotDiffReportEntryProto {
-  required bytes fullpath = 1;
-  required string modificationLabel = 2;
-  optional bytes targetPath = 3;
-}
-
-/**
- * Snapshot diff report
- */
-message SnapshotDiffReportProto {
-  // full path of the directory where snapshots were taken
-  required string snapshotRoot = 1;
-  required string fromSnapshot = 2;
-  required string toSnapshot = 3;
-  repeated SnapshotDiffReportEntryProto diffReportEntries = 4;
-}
-
-/**
- * Common node information shared by all the nodes in the cluster
- */
-message StorageInfoProto {
-  required uint32 layoutVersion = 1; // Layout version of the file system
-  required uint32 namespceID = 2;    // File system namespace ID
-  required string clusterID = 3;     // ID of the cluster
-  required uint64 cTime = 4;         // File system creation time
-}
-
-/**
- * Information sent by a namenode to identify itself to the primary namenode.
- */
-message NamenodeRegistrationProto {
-  required string rpcAddress = 1;    // host:port of the namenode RPC address
-  required string httpAddress = 2;   // host:port of the namenode http server
-  enum NamenodeRoleProto {
-    NAMENODE = 1;
-    BACKUP = 2;
-    CHECKPOINT = 3;
-  }
-  required StorageInfoProto storageInfo = 3;  // Node information
-  optional NamenodeRoleProto role = 4 [default = NAMENODE];        // Namenode role
-}
-
-/**
- * Unique signature to identify checkpoint transactions.
- */
-message CheckpointSignatureProto {
-  required string blockPoolId = 1;
-  required uint64 mostRecentCheckpointTxId = 2;
-  required uint64 curSegmentTxId = 3;
-  required StorageInfoProto storageInfo = 4;
-}
-
-/**
- * Command sent from one namenode to another namenode.
- */
-message NamenodeCommandProto {
-  enum Type {
-    NamenodeCommand = 0;      // Base command
-    CheckPointCommand = 1;    // Check point command
-  }
-  required uint32 action = 1;
-  required Type type = 2;
-  optional CheckpointCommandProto checkpointCmd = 3;
-}
-
-/**
- * Command returned from primary to checkpointing namenode.
- * This command has checkpoint signature that identifies
- * checkpoint transaction and is needed for further
- * communication related to checkpointing.
- */
-message CheckpointCommandProto {
-  // Unique signature to identify checkpoint transation
-  required CheckpointSignatureProto signature = 1; 
-
-  // If true, return transfer image to primary upon the completion of checkpoint
-  required bool needToReturnImage = 2;
-}
-
-/**
- * Block information
- *
- * Please be wary of adding additional fields here, since INodeFiles
- * need to fit in PB's default max message size of 64MB.
- * We restrict the max # of blocks per file
- * (dfs.namenode.fs-limits.max-blocks-per-file), but it's better
- * to avoid changing this.
- */
-message BlockProto {
-  required uint64 blockId = 1;
-  required uint64 genStamp = 2;
-  optional uint64 numBytes = 3 [default = 0];
-}
-
-/**
- * Block and datanodes where is it located
- */
-message BlockWithLocationsProto {
-  required BlockProto block = 1;   // Block
-  repeated string datanodeUuids = 2; // Datanodes with replicas of the block
-  repeated string storageUuids = 3;  // Storages with replicas of the block
-  repeated StorageTypeProto storageTypes = 4;
-}
-
-/**
- * List of block with locations
- */
-message BlocksWithLocationsProto {
-  repeated BlockWithLocationsProto blocks = 1;
-}
-
-/**
- * Editlog information with available transactions
- */
-message RemoteEditLogProto {
-  required uint64 startTxId = 1;  // Starting available edit log transaction
-  required uint64 endTxId = 2;    // Ending available edit log transaction
-  optional bool isInProgress = 3 [default = false];
-}
-
-/**
- * Enumeration of editlogs available on a remote namenode
- */
-message RemoteEditLogManifestProto {
-  repeated RemoteEditLogProto logs = 1;
-}
-
-/**
- * Namespace information that describes namespace on a namenode
- */
-message NamespaceInfoProto {
-  required string buildVersion = 1;         // Software revision version (e.g. an svn or git revision)
-  required uint32 unused = 2;               // Retained for backward compatibility
-  required string blockPoolID = 3;          // block pool used by the namespace
-  required StorageInfoProto storageInfo = 4;// Node information
-  required string softwareVersion = 5;      // Software version number (e.g. 2.0.0)
-  optional uint64 capabilities = 6 [default = 0]; // feature flags
-}
-
-/**
- * Block access token information
- */
-message BlockKeyProto {
-  required uint32 keyId = 1;      // Key identifier
-  required uint64 expiryDate = 2; // Expiry time in milliseconds
-  optional bytes keyBytes = 3;    // Key secret
-}
-
-/**
- * Current key and set of block keys at the namenode.
- */
-message ExportedBlockKeysProto {
-  required bool isBlockTokenEnabled = 1;
-  required uint64 keyUpdateInterval = 2;
-  required uint64 tokenLifeTime = 3;
-  required BlockKeyProto currentKey = 4;
-  repeated BlockKeyProto allKeys = 5;
-}
-
-/**
- * State of a block replica at a datanode
- */
-enum ReplicaStateProto {
-  FINALIZED = 0;  // State of a replica when it is not modified
-  RBW = 1;        // State of replica that is being written to
-  RWR = 2;        // State of replica that is waiting to be recovered
-  RUR = 3;        // State of replica that is under recovery
-  TEMPORARY = 4;  // State of replica that is created for replication
-}
-
-/**
- * Block that needs to be recovered with at a given location
- */
-message RecoveringBlockProto {
-  required uint64 newGenStamp = 1;        // New genstamp post recovery
-  required LocatedBlockProto block = 2;   // Block to be recovered
-  optional BlockProto truncateBlock = 3;  // New block for recovery (truncate)
-}
-
-/**
- * void request
- */
-message VersionRequestProto {
-}
-
-/**
- * Version response from namenode.
- */
-message VersionResponseProto {
-  required NamespaceInfoProto info = 1;
-}
-
-/**
- * Information related to a snapshot
- * TODO: add more information
- */
-message SnapshotInfoProto {
-  required string snapshotName = 1;
-  required string snapshotRoot = 2;
-  required FsPermissionProto permission = 3;
-  required string owner = 4;
-  required string group = 5;
-  required string createTime = 6;
-  // TODO: do we need access time?
-}
-
-/**
- * Rolling upgrade status
- */
-message RollingUpgradeStatusProto {
-  required string blockPoolId = 1;
-  optional bool finalized = 2 [default = false];
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/inotify.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/inotify.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/inotify.proto
deleted file mode 100644
index 5b78fe6..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/inotify.proto
+++ /dev/null
@@ -1,126 +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.
- */
-
-/**
- * These .proto interfaces are private and stable.
- * Please see http://wiki.apache.org/hadoop/Compatibility
- * for what changes are allowed for a *stable* .proto interface.
- */
-
-// This file contains protocol buffers used to communicate edits to clients
-// as part of the inotify system.
-
-option java_package = "org.apache.hadoop.hdfs.protocol.proto";
-option java_outer_classname = "InotifyProtos";
-option java_generate_equals_and_hash = true;
-package hadoop.hdfs;
-
-import "acl.proto";
-import "xattr.proto";
-import "hdfs.proto";
-
-enum EventType {
-  EVENT_CREATE = 0x0;
-  EVENT_CLOSE = 0x1;
-  EVENT_APPEND = 0x2;
-  EVENT_RENAME = 0x3;
-  EVENT_METADATA = 0x4;
-  EVENT_UNLINK = 0x5;
-}
-
-message EventProto {
-  required EventType type = 1;
-  required bytes contents = 2;
-}
-
-message EventBatchProto {
-  required int64 txid = 1;
-  repeated EventProto events = 2;
-}
-
-enum INodeType {
-  I_TYPE_FILE = 0x0;
-  I_TYPE_DIRECTORY = 0x1;
-  I_TYPE_SYMLINK = 0x2;
-}
-
-enum MetadataUpdateType {
-  META_TYPE_TIMES = 0x0;
-  META_TYPE_REPLICATION = 0x1;
-  META_TYPE_OWNER = 0x2;
-  META_TYPE_PERMS = 0x3;
-  META_TYPE_ACLS = 0x4;
-  META_TYPE_XATTRS = 0x5;
-}
-
-message CreateEventProto {
-  required INodeType type = 1;
-  required string path = 2;
-  required int64 ctime = 3;
-  required string ownerName = 4;
-  required string groupName = 5;
-  required FsPermissionProto perms = 6;
-  optional int32 replication = 7;
-  optional string symlinkTarget = 8;
-  optional bool overwrite = 9;
-  optional int64 defaultBlockSize = 10 [default=0];
-}
-
-message CloseEventProto {
-  required string path = 1;
-  required int64 fileSize = 2;
-  required int64 timestamp = 3;
-}
-
-message AppendEventProto {
-  required string path = 1;
-  optional bool newBlock = 2 [default = false];
-}
-
-message RenameEventProto {
-  required string srcPath = 1;
-  required string destPath = 2;
-  required int64 timestamp = 3;
-}
-
-message MetadataUpdateEventProto {
-  required string path = 1;
-  required MetadataUpdateType type = 2;
-  optional int64 mtime = 3;
-  optional int64 atime = 4;
-  optional int32 replication = 5;
-  optional string ownerName = 6;
-  optional string groupName = 7;
-  optional FsPermissionProto perms = 8;
-  repeated AclEntryProto acls = 9;
-  repeated XAttrProto xAttrs = 10;
-  optional bool xAttrsRemoved = 11;
-}
-
-message UnlinkEventProto {
-  required string path = 1;
-  required int64 timestamp = 2;
-}
-
-message EventsListProto {
-  repeated EventProto events = 1; // deprecated
-  required int64 firstTxid = 2;
-  required int64 lastTxid = 3;
-  required int64 syncTxid = 4;
-  repeated EventBatchProto batch = 5;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6182d5/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/xattr.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/xattr.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/xattr.proto
deleted file mode 100644
index acdc28e..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/xattr.proto
+++ /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.
- */
-
-option java_package = "org.apache.hadoop.hdfs.protocol.proto";
-option java_outer_classname = "XAttrProtos";
-option java_generate_equals_and_hash = true;
-package hadoop.hdfs;
-  
-message XAttrProto {
-  enum XAttrNamespaceProto {
-    USER      = 0;
-    TRUSTED   = 1;
-    SECURITY  = 2;
-    SYSTEM    = 3;
-    RAW       = 4;
-  }
-  
-  required XAttrNamespaceProto namespace = 1;
-  required string name = 2;
-  optional bytes value = 3;
-}
-  
-message XAttrEditLogProto {
-  optional string src = 1;
-  repeated XAttrProto xAttrs = 2;
-}
-
-enum XAttrSetFlagProto {
-  XATTR_CREATE     = 0x01;
-  XATTR_REPLACE    = 0x02;
-}
-
-message SetXAttrRequestProto {
-  required string src          = 1;
-  optional XAttrProto xAttr    = 2;
-  optional uint32 flag         = 3; //bits set using XAttrSetFlagProto
-}
-
-message SetXAttrResponseProto {
-}
-
-message GetXAttrsRequestProto {
-  required string src = 1;
-  repeated XAttrProto xAttrs = 2;
-}
-
-message GetXAttrsResponseProto {
-  repeated XAttrProto xAttrs = 1;
-}
-
-message ListXAttrsRequestProto {
-  required string src = 1;
-}
-
-message ListXAttrsResponseProto {
-  repeated XAttrProto xAttrs = 1;
-}
-
-message RemoveXAttrRequestProto {
-  required string src        = 1;
-  optional XAttrProto xAttr  = 2;
-}
-
-message RemoveXAttrResponseProto {
-}


[05/50] hadoop git commit: HADOOP-12195. Add annotation to package-info.java file to workaround MCOMPILER-205.

Posted by ar...@apache.org.
HADOOP-12195. Add annotation to package-info.java file to workaround MCOMPILER-205.


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

Branch: refs/heads/HDFS-7240
Commit: 3dc92e84c2a530483c6a7693817b67d86f9c97aa
Parents: aa96a8c
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Jul 7 11:14:12 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue Jul 7 11:14:12 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                 | 3 +++
 .../src/main/java/org/apache/hadoop/jmx/package-info.java       | 5 ++++-
 2 files changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3dc92e84/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 af6e3fe..859e58a 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -686,6 +686,9 @@ Release 2.8.0 - UNRELEASED
 
     HADOOP-12193. Rename Touchz.java to Touch.java. (wang)
 
+    HADOOP-12195. Add annotation to package-info.java file to workaround
+    MCOMPILER-205. (wang)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3dc92e84/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/package-info.java
index e09d993..ef227f0 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/package-info.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/package-info.java
@@ -19,4 +19,7 @@
  * This package provides access to JMX primarily through the
  * {@link org.apache.hadoop.jmx.JMXJsonServlet} class.
  */
-package org.apache.hadoop.jmx;
\ No newline at end of file
+@InterfaceAudience.Private
+package org.apache.hadoop.jmx;
+
+import org.apache.hadoop.classification.InterfaceAudience;
\ No newline at end of file


[04/50] hadoop git commit: HADOOP-12193. Rename Touchz.java to Touch.java.

Posted by ar...@apache.org.
HADOOP-12193. Rename Touchz.java to Touch.java.


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

Branch: refs/heads/HDFS-7240
Commit: aa96a8c0c5d82a21e9b511517bc621c41841bc53
Parents: 7e2fe8c
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Jul 7 11:12:29 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue Jul 7 11:12:56 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  2 +
 .../java/org/apache/hadoop/fs/shell/Touch.java  | 84 ++++++++++++++++++++
 .../java/org/apache/hadoop/fs/shell/Touchz.java | 84 --------------------
 3 files changed, 86 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa96a8c0/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 194e2e3..af6e3fe 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -684,6 +684,8 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11974. Fix FIONREAD #include on Solaris (Alan Burlison via Colin P.
     McCabe)
 
+    HADOOP-12193. Rename Touchz.java to Touch.java. (wang)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa96a8c0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Touch.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Touch.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Touch.java
new file mode 100644
index 0000000..72a463a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Touch.java
@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.shell;
+
+import java.io.IOException;
+import java.util.LinkedList;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.PathIsDirectoryException;
+import org.apache.hadoop.fs.PathNotFoundException;
+
+/**
+ * Unix touch like commands
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+
+class Touch extends FsCommand {
+  public static void registerCommands(CommandFactory factory) {
+    factory.addClass(Touchz.class, "-touchz");
+  }
+
+  /**
+   * (Re)create zero-length file at the specified path.
+   * This will be replaced by a more UNIX-like touch when files may be
+   * modified.
+   */
+  public static class Touchz extends Touch {
+    public static final String NAME = "touchz";
+    public static final String USAGE = "<path> ...";
+    public static final String DESCRIPTION =
+      "Creates a file of zero length " +
+      "at <path> with current time as the timestamp of that <path>. " +
+      "An error is returned if the file exists with non-zero length\n";
+
+    @Override
+    protected void processOptions(LinkedList<String> args) {
+      CommandFormat cf = new CommandFormat(1, Integer.MAX_VALUE);
+      cf.parse(args);
+    }
+
+    @Override
+    protected void processPath(PathData item) throws IOException {
+      if (item.stat.isDirectory()) {
+        // TODO: handle this
+        throw new PathIsDirectoryException(item.toString());
+      }
+      if (item.stat.getLen() != 0) {
+        throw new PathIOException(item.toString(), "Not a zero-length file");
+      }
+      touchz(item);
+    }
+
+    @Override
+    protected void processNonexistentPath(PathData item) throws IOException {
+      if (!item.parentExists()) {
+        throw new PathNotFoundException(item.toString());
+      }
+      touchz(item);
+    }
+
+    private void touchz(PathData item) throws IOException {
+      item.fs.create(item.path).close();
+    }
+  }
+}

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


[06/50] hadoop git commit: HDFS-8711. setSpaceQuota command should print the available storage type when input storage type is wrong. Contributed by Brahma Reddy Battula.

Posted by ar...@apache.org.
HDFS-8711. setSpaceQuota command should print the available storage type when input storage type is wrong. Contributed by Brahma Reddy Battula.


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

Branch: refs/heads/HDFS-7240
Commit: b68701b7b2a9597b4183e0ba19b1551680d543a1
Parents: 3dc92e8
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Tue Jul 7 13:50:49 2015 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Tue Jul 7 13:50:49 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  | 10 ++++++++--
 .../java/org/apache/hadoop/hdfs/TestQuota.java  | 21 ++++++++++++++++++++
 3 files changed, 32 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b68701b7/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 7294cab..1e1e6bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -701,6 +701,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8709. Clarify automatic sync in FSEditLog#logEdit. (wang)
 
+    HDFS-8711. setSpaceQuota command should print the available storage type
+    when input storage type is wrong. (Brahma Reddy Battula via xyao)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b68701b7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
index 4640bb3..014637b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
@@ -290,9 +290,15 @@ public class DFSAdmin extends FsShell {
       String storageTypeString =
           StringUtils.popOptionWithArgument("-storageType", parameters);
       if (storageTypeString != null) {
-        this.type = StorageType.parseStorageType(storageTypeString);
+        try {
+          this.type = StorageType.parseStorageType(storageTypeString);
+        } catch (IllegalArgumentException e) {
+          throw new IllegalArgumentException("Storage type "
+              + storageTypeString
+              + " is not available. Available storage types are "
+              + StorageType.getTypesSupportingQuota());
+        }
       }
-      
       this.args = parameters.toArray(new String[parameters.size()]);
     }
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b68701b7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java
index 4541e69..e339049 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java
@@ -17,18 +17,22 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.ByteArrayOutputStream;
 import java.io.OutputStream;
+import java.io.PrintStream;
 import java.security.PrivilegedExceptionAction;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
@@ -41,6 +45,8 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.google.common.base.Charsets;
+
 /** A class for testing quota-related commands */
 public class TestQuota {
   
@@ -986,4 +992,19 @@ public class TestQuota {
       cluster.shutdown();
     }
   }
+
+  @Test
+  public void testSetSpaceQuotaWhenStorageTypeIsWrong() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    conf.set(FS_DEFAULT_NAME_KEY, "hdfs://127.0.0.1:8020");
+    DFSAdmin admin = new DFSAdmin(conf);
+    ByteArrayOutputStream err = new ByteArrayOutputStream();
+    System.setErr(new PrintStream(err));
+    String[] args = { "-setSpaceQuota", "100", "-storageType", "COLD",
+        "/testDir" };
+    admin.run(args);
+    String errOutput = new String(err.toByteArray(), Charsets.UTF_8);
+    assertTrue(errOutput.contains(StorageType.getTypesSupportingQuota()
+        .toString()));
+  }
 }


[33/50] hadoop git commit: YARN-3116. RM notifies NM whether a container is an AM container or normal task container. Contributed by Giovanni Matteo Fumarola.

Posted by ar...@apache.org.
YARN-3116. RM notifies NM whether a container is an AM container or normal task container. Contributed by Giovanni Matteo Fumarola.


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

Branch: refs/heads/HDFS-7240
Commit: 1ea36299a47af302379ae0750b571ec021eb54ad
Parents: 47f4c54
Author: Zhijie Shen <zj...@apache.org>
Authored: Fri Jul 10 18:58:10 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Jul 10 18:58:10 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../yarn/server/api/ContainerContext.java       | 19 +++++++
 .../api/ContainerInitializationContext.java     |  7 +++
 .../server/api/ContainerTerminationContext.java |  7 +++
 .../hadoop/yarn/server/api/ContainerType.java   | 34 +++++++++++++
 .../src/main/proto/yarn_protos.proto            |  5 ++
 .../yarn/api/records/impl/pb/ProtoUtils.java    | 12 +++++
 .../yarn/security/ContainerTokenIdentifier.java | 43 ++++++++++++++--
 .../main/proto/server/yarn_security_token.proto |  1 +
 .../yarn/security/TestYARNTokenIdentifier.java  | 53 ++++++++++++++++++++
 .../containermanager/AuxServices.java           |  6 ++-
 .../scheduler/SchedulerApplicationAttempt.java  | 17 ++++++-
 .../scheduler/capacity/LeafQueue.java           |  3 +-
 .../security/RMContainerTokenSecretManager.java | 13 +++--
 .../server/resourcemanager/Application.java     | 12 +++++
 .../capacity/TestContainerAllocation.java       |  5 +-
 .../scheduler/fair/FairSchedulerTestBase.java   | 31 ++++++++++--
 .../scheduler/fifo/TestFifoScheduler.java       |  4 ++
 18 files changed, 253 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ea36299/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index db000d7..f78bbfa 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -134,6 +134,9 @@ Release 2.8.0 - UNRELEASED
     YARN-1012. Report NM aggregated container resource utilization in heartbeat. 
     (Inigo Goiri via kasha)
 
+    YARN-3116. RM notifies NM whether a container is an AM container or normal
+    task container. (Giovanni Matteo Fumarola via zjshen)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ea36299/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerContext.java
index d13159b..f7a9b02 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerContext.java
@@ -35,14 +35,23 @@ public class ContainerContext {
   private final String user;
   private final ContainerId containerId;
   private final Resource resource;
+  private final ContainerType containerType;
 
   @Private
   @Unstable
   public ContainerContext(String user, ContainerId containerId,
       Resource resource) {
+    this(user, containerId, resource, ContainerType.TASK);
+  }
+
+  @Private
+  @Unstable
+  public ContainerContext(String user, ContainerId containerId,
+      Resource resource, ContainerType containerType) {
     this.user = user;
     this.containerId = containerId;
     this.resource = resource;
+    this.containerType = containerType;
   }
 
   /**
@@ -72,4 +81,14 @@ public class ContainerContext {
   public Resource getResource() {
     return resource;
   }
+
+  /**
+   * Get {@link ContainerType} the type of the container
+   * being initialized or stopped.
+   *
+   * @return the type of the container
+   */
+  public ContainerType getContainerType() {
+    return containerType;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ea36299/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerInitializationContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerInitializationContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerInitializationContext.java
index 5b5bbda..44428f9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerInitializationContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerInitializationContext.java
@@ -41,4 +41,11 @@ public class ContainerInitializationContext extends ContainerContext {
     super(user, containerId, resource);
   }
 
+  @Private
+  @Unstable
+  public ContainerInitializationContext(String user, ContainerId containerId,
+      Resource resource, ContainerType containerType) {
+    super(user, containerId, resource, containerType);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ea36299/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerTerminationContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerTerminationContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerTerminationContext.java
index 34ba73e..2e4ad3b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerTerminationContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerTerminationContext.java
@@ -41,4 +41,11 @@ public class ContainerTerminationContext extends ContainerContext {
     super(user, containerId, resource);
   }
 
+  @Private
+  @Unstable
+  public ContainerTerminationContext(String user, ContainerId containerId,
+      Resource resource, ContainerType containerType) {
+    super(user, containerId, resource, containerType);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ea36299/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerType.java
new file mode 100644
index 0000000..ffae811
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerType.java
@@ -0,0 +1,34 @@
+/**
+ * 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.api;
+
+/**
+ * Container property encoding allocation and execution semantics.
+ * 
+ * <p>
+ * The container types are the following:
+ * <ul>
+ * <li>{@link #APPLICATION_MASTER}
+ * <li>{@link #TASK}
+ * </ul>
+ * </p>
+ */
+public enum ContainerType {
+  APPLICATION_MASTER, TASK
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ea36299/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 f801409..33cc255 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
@@ -264,6 +264,11 @@ message NodeLabelProto {
   optional bool isExclusive = 2 [default = true]; 
 }
 
+enum ContainerTypeProto {
+  APPLICATION_MASTER = 1;
+  TASK = 2;
+}
+
 ////////////////////////////////////////////////////////////////////////
 ////// From AM_RM_Protocol /////////////////////////////////////////////
 ////////////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ea36299/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
index 4e8a19c..e742f4c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
@@ -53,7 +53,9 @@ import org.apache.hadoop.yarn.proto.YarnProtos.QueueStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestInterpreterProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationAttemptStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationStateProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerTypeProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+import org.apache.hadoop.yarn.server.api.ContainerType;
 
 import com.google.protobuf.ByteString;
 
@@ -270,4 +272,14 @@ public class ProtoUtils {
     return LogAggregationStatus.valueOf(e.name().replace(
       LOG_AGGREGATION_STATUS_PREFIX, ""));
   }
+
+  /*
+   * ContainerType
+   */
+  public static ContainerTypeProto convertToProtoFormat(ContainerType e) {
+    return ContainerTypeProto.valueOf(e.name());
+  }
+  public static ContainerType convertFromProtoFormat(ContainerTypeProto e) {
+    return ContainerType.valueOf(e.name());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ea36299/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java
index 9a60d01..106e6d5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java
@@ -39,13 +39,15 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.LogAggregationContextPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.PriorityPBImpl;
+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.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerTypeProto;
 import org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos.ContainerTokenIdentifierProto;
+import org.apache.hadoop.yarn.server.api.ContainerType;
 
 import com.google.protobuf.TextFormat;
 
-
 /**
  * TokenIdentifier for a container. Encodes {@link ContainerId},
  * {@link Resource} needed by the container and the target NMs host-address.
@@ -66,14 +68,24 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
       int masterKeyId, long rmIdentifier, Priority priority, long creationTime) {
     this(containerID, hostName, appSubmitter, r, expiryTimeStamp, masterKeyId,
         rmIdentifier, priority, creationTime, null,
-        CommonNodeLabelsManager.NO_LABEL);
+        CommonNodeLabelsManager.NO_LABEL, ContainerType.TASK);
   }
 
   public ContainerTokenIdentifier(ContainerId containerID, String hostName,
       String appSubmitter, Resource r, long expiryTimeStamp, int masterKeyId,
       long rmIdentifier, Priority priority, long creationTime,
       LogAggregationContext logAggregationContext, String nodeLabelExpression) {
-    ContainerTokenIdentifierProto.Builder builder = 
+    this(containerID, hostName, appSubmitter, r, expiryTimeStamp, masterKeyId,
+        rmIdentifier, priority, creationTime, logAggregationContext,
+        nodeLabelExpression, ContainerType.TASK);
+  }
+
+  public ContainerTokenIdentifier(ContainerId containerID, String hostName,
+      String appSubmitter, Resource r, long expiryTimeStamp, int masterKeyId,
+      long rmIdentifier, Priority priority, long creationTime,
+      LogAggregationContext logAggregationContext, String nodeLabelExpression,
+      ContainerType containerType) {
+    ContainerTokenIdentifierProto.Builder builder =
         ContainerTokenIdentifierProto.newBuilder();
     if (containerID != null) {
       builder.setContainerId(((ContainerIdPBImpl)containerID).getProto());
@@ -99,7 +111,8 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
     if (nodeLabelExpression != null) {
       builder.setNodeLabelExpression(nodeLabelExpression);
     }
-    
+    builder.setContainerType(convertToProtoFormat(containerType));
+
     proto = builder.build();
   }
 
@@ -156,7 +169,18 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
   public long getRMIdentifier() {
     return proto.getRmIdentifier();
   }
-  
+
+  /**
+   * Get the ContainerType of container to allocate
+   * @return ContainerType
+   */
+  public ContainerType getContainerType(){
+    if (!proto.hasContainerType()) {
+      return null;
+    }
+    return convertFromProtoFormat(proto.getContainerType());
+  }
+
   public ContainerTokenIdentifierProto getProto() {
     return proto;
   }
@@ -232,4 +256,13 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
   public String toString() {
     return TextFormat.shortDebugString(getProto());
   }
+
+  private ContainerTypeProto convertToProtoFormat(ContainerType containerType) {
+    return ProtoUtils.convertToProtoFormat(containerType);
+  }
+
+  private ContainerType convertFromProtoFormat(
+      ContainerTypeProto containerType) {
+    return ProtoUtils.convertFromProtoFormat(containerType);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ea36299/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/proto/server/yarn_security_token.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/proto/server/yarn_security_token.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/proto/server/yarn_security_token.proto
index d1bef21..339e99e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/proto/server/yarn_security_token.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/proto/server/yarn_security_token.proto
@@ -50,6 +50,7 @@ message ContainerTokenIdentifierProto {
   optional int64 creationTime = 9;
   optional LogAggregationContextProto logAggregationContext = 10;
   optional string nodeLabelExpression = 11;
+  optional ContainerTypeProto containerType = 12;
 }
 
 message ClientToAMTokenIdentifierProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ea36299/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestYARNTokenIdentifier.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestYARNTokenIdentifier.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestYARNTokenIdentifier.java
index 5fe75bc..68f0b9d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestYARNTokenIdentifier.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestYARNTokenIdentifier.java
@@ -33,10 +33,12 @@ 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.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos.YARNDelegationTokenIdentifierProto;
 import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.server.api.ContainerType;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -201,6 +203,12 @@ public class TestYARNTokenIdentifier {
         anotherToken.getCreationTime(), creationTime);
     
     Assert.assertNull(anotherToken.getLogAggregationContext());
+
+    Assert.assertEquals(CommonNodeLabelsManager.NO_LABEL,
+        anotherToken.getNodeLabelExpression());
+
+    Assert.assertEquals(ContainerType.TASK,
+        anotherToken.getContainerType());
   }
   
   @Test
@@ -347,4 +355,49 @@ public class TestYARNTokenIdentifier {
     Assert.assertEquals(new Text("yarn"), token.getRenewer());
   }
 
+  @Test
+  public void testAMContainerTokenIdentifier() throws IOException {
+    ContainerId containerID = ContainerId.newContainerId(
+        ApplicationAttemptId.newInstance(ApplicationId.newInstance(
+            1, 1), 1), 1);
+    String hostName = "host0";
+    String appSubmitter = "usr0";
+    Resource r = Resource.newInstance(1024, 1);
+    long expiryTimeStamp = 1000;
+    int masterKeyId = 1;
+    long rmIdentifier = 1;
+    Priority priority = Priority.newInstance(1);
+    long creationTime = 1000;
+
+    ContainerTokenIdentifier token =
+        new ContainerTokenIdentifier(containerID, hostName, appSubmitter, r,
+            expiryTimeStamp, masterKeyId, rmIdentifier, priority, creationTime,
+            null, CommonNodeLabelsManager.NO_LABEL, ContainerType.APPLICATION_MASTER);
+
+    ContainerTokenIdentifier anotherToken = new ContainerTokenIdentifier();
+
+    byte[] tokenContent = token.getBytes();
+    DataInputBuffer dib = new DataInputBuffer();
+    dib.reset(tokenContent, tokenContent.length);
+    anotherToken.readFields(dib);
+
+    Assert.assertEquals(ContainerType.APPLICATION_MASTER,
+        anotherToken.getContainerType());
+
+    token =
+        new ContainerTokenIdentifier(containerID, hostName, appSubmitter, r,
+            expiryTimeStamp, masterKeyId, rmIdentifier, priority, creationTime,
+            null, CommonNodeLabelsManager.NO_LABEL, ContainerType.TASK);
+
+    anotherToken = new ContainerTokenIdentifier();
+
+    tokenContent = token.getBytes();
+    dib = new DataInputBuffer();
+    dib.reset(tokenContent, tokenContent.length);
+    anotherToken.readFields(dib);
+
+    Assert.assertEquals(ContainerType.TASK,
+        anotherToken.getContainerType());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ea36299/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.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/AuxServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
index fb6f79b..cd5ed88 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
@@ -225,7 +225,8 @@ public class AuxServices extends AbstractService
           try {
             serv.initializeContainer(new ContainerInitializationContext(
                 event.getUser(), event.getContainer().getContainerId(),
-                event.getContainer().getResource()));
+                event.getContainer().getResource(), event.getContainer()
+                .getContainerTokenIdentifier().getContainerType()));
           } catch (Throwable th) {
             logWarningWhenAuxServiceThrowExceptions(serv,
                 AuxServicesEventType.CONTAINER_INIT, th);
@@ -237,7 +238,8 @@ public class AuxServices extends AbstractService
           try {
             serv.stopContainer(new ContainerTerminationContext(
                 event.getUser(), event.getContainer().getContainerId(),
-                event.getContainer().getResource()));
+                event.getContainer().getResource(), event.getContainer()
+                .getContainerTokenIdentifier().getContainerType()));
           } catch (Throwable th) {
             logWarningWhenAuxServiceThrowExceptions(serv,
                 AuxServicesEventType.CONTAINER_STOP, th);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ea36299/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.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/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index dbc3cb5..475f2c7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -45,6 +45,7 @@ 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.ResourceRequest;
+import org.apache.hadoop.yarn.server.api.ContainerType;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AggregateAppResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@@ -61,7 +62,6 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.SchedulableEntity;
 
-
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.HashMultiset;
@@ -467,13 +467,26 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
       .hasNext();) {
       RMContainer rmContainer = i.next();
       Container container = rmContainer.getContainer();
+      ContainerType containerType = ContainerType.TASK;
+      // The working knowledge is that masterContainer for AM is null as it
+      // itself is the master container.
+      RMAppAttempt appAttempt =
+          rmContext
+              .getRMApps()
+              .get(
+                  container.getId().getApplicationAttemptId()
+                      .getApplicationId()).getCurrentAppAttempt();
+      if (appAttempt.getMasterContainer() == null
+          && appAttempt.getSubmissionContext().getUnmanagedAM() == false) {
+        containerType = ContainerType.APPLICATION_MASTER;
+      }
       try {
         // create container token and NMToken altogether.
         container.setContainerToken(rmContext.getContainerTokenSecretManager()
             .createContainerToken(container.getId(), container.getNodeId(),
                 getUser(), container.getResource(), container.getPriority(),
                 rmContainer.getCreationTime(), this.logAggregationContext,
-                rmContainer.getNodeLabelExpression()));
+                rmContainer.getNodeLabelExpression(), containerType));
         NMToken nmToken =
             rmContext.getNMTokenSecretManager().createAndGetNMToken(getUser(),
               getApplicationAttemptId(), container);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ea36299/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 e5b44a6..598f279 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
@@ -829,7 +829,8 @@ public class LeafQueue extends AbstractCSQueue {
             RMAppAttempt rmAppAttempt =
                 csContext.getRMContext().getRMApps()
                     .get(application.getApplicationId()).getCurrentAppAttempt();
-            if (null == rmAppAttempt.getMasterContainer()) {
+            if (rmAppAttempt.getSubmissionContext().getUnmanagedAM() == false
+                && null == rmAppAttempt.getMasterContainer()) {
               if (LOG.isDebugEnabled()) {
                 LOG.debug("Skip allocating AM container to app_attempt="
                     + application.getApplicationAttemptId()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ea36299/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.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/security/RMContainerTokenSecretManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
index 1c0533d..6f00615 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
@@ -33,6 +33,7 @@ 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.security.ContainerTokenIdentifier;
+import org.apache.hadoop.yarn.server.api.ContainerType;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.security.BaseContainerTokenSecretManager;
@@ -166,7 +167,7 @@ public class RMContainerTokenSecretManager extends
 
   /**
    * Helper function for creating ContainerTokens
-   * 
+   *
    * @param containerId
    * @param nodeId
    * @param appSubmitter
@@ -179,12 +180,12 @@ public class RMContainerTokenSecretManager extends
       String appSubmitter, Resource capability, Priority priority,
       long createTime) {
     return createContainerToken(containerId, nodeId, appSubmitter, capability,
-      priority, createTime, null, null);
+      priority, createTime, null, null, ContainerType.TASK);
   }
 
   /**
    * Helper function for creating ContainerTokens
-   * 
+   *
    * @param containerId
    * @param nodeId
    * @param appSubmitter
@@ -192,12 +193,14 @@ public class RMContainerTokenSecretManager extends
    * @param priority
    * @param createTime
    * @param logAggregationContext
+   * @param nodeLabelExpression
+   * @param containerType
    * @return the container-token
    */
   public Token createContainerToken(ContainerId containerId, NodeId nodeId,
       String appSubmitter, Resource capability, Priority priority,
       long createTime, LogAggregationContext logAggregationContext,
-      String nodeLabelExpression) {
+      String nodeLabelExpression, ContainerType containerType) {
     byte[] password;
     ContainerTokenIdentifier tokenIdentifier;
     long expiryTimeStamp =
@@ -211,7 +214,7 @@ public class RMContainerTokenSecretManager extends
             appSubmitter, capability, expiryTimeStamp, this.currentMasterKey
               .getMasterKey().getKeyId(),
             ResourceManager.getClusterTimeStamp(), priority, createTime,
-            logAggregationContext, nodeLabelExpression);
+            logAggregationContext, nodeLabelExpression, containerType);
       password = this.createPassword(tokenIdentifier);
 
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ea36299/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.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/Application.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
index 9b7eb84..e62f7d7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
@@ -55,6 +55,8 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.Task.State;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@@ -168,6 +170,16 @@ public class Application {
     
     resourceManager.getClientRMService().submitApplication(request);
 
+    RMAppEvent event =
+        new RMAppEvent(this.applicationId, RMAppEventType.START);
+    resourceManager.getRMContext().getRMApps().get(applicationId).handle(event);
+    event =
+        new RMAppEvent(this.applicationId, RMAppEventType.APP_NEW_SAVED);
+    resourceManager.getRMContext().getRMApps().get(applicationId).handle(event);
+    event =
+        new RMAppEvent(this.applicationId, RMAppEventType.APP_ACCEPTED);
+    resourceManager.getRMContext().getRMApps().get(applicationId).handle(event);
+
     // Notify scheduler
     AppAddedSchedulerEvent addAppEvent =
         new AppAddedSchedulerEvent(this.applicationId, this.queue, "user");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ea36299/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.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/TestContainerAllocation.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/TestContainerAllocation.java
index 0ea993f..6183bf6 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/TestContainerAllocation.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/TestContainerAllocation.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+import org.apache.hadoop.yarn.server.api.ContainerType;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
@@ -286,11 +287,11 @@ public class TestContainerAllocation {
         public Token createContainerToken(ContainerId containerId,
             NodeId nodeId, String appSubmitter, Resource capability,
             Priority priority, long createTime,
-            LogAggregationContext logAggregationContext, String nodeLabelExp) {
+            LogAggregationContext logAggregationContext, String nodeLabelExp, ContainerType containerType) {
           numRetries++;
           return super.createContainerToken(containerId, nodeId, appSubmitter,
               capability, priority, createTime, logAggregationContext,
-              nodeLabelExp);
+              nodeLabelExp, containerType);
         }
       };
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ea36299/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java
index 23b708a..403c8ea 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java
@@ -28,6 +28,7 @@ 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.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -38,6 +39,8 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
@@ -152,6 +155,11 @@ public class FairSchedulerTestBase {
     when(rmApp.getCurrentAppAttempt()).thenReturn(rmAppAttempt);
     when(rmAppAttempt.getRMAppAttemptMetrics()).thenReturn(
         new RMAppAttemptMetrics(id, resourceManager.getRMContext()));
+    ApplicationSubmissionContext submissionContext = mock(ApplicationSubmissionContext.class);
+    when(submissionContext.getUnmanagedAM()).thenReturn(false);
+    when(rmAppAttempt.getSubmissionContext()).thenReturn(submissionContext);
+    Container container = mock(Container.class);
+    when(rmAppAttempt.getMasterContainer()).thenReturn(container);
     resourceManager.getRMContext().getRMApps()
         .put(id.getApplicationId(), rmApp);
 
@@ -175,6 +183,9 @@ public class FairSchedulerTestBase {
     when(rmApp.getCurrentAppAttempt()).thenReturn(rmAppAttempt);
     when(rmAppAttempt.getRMAppAttemptMetrics()).thenReturn(
         new RMAppAttemptMetrics(id,resourceManager.getRMContext()));
+    ApplicationSubmissionContext submissionContext = mock(ApplicationSubmissionContext.class);
+    when(submissionContext.getUnmanagedAM()).thenReturn(false);
+    when(rmAppAttempt.getSubmissionContext()).thenReturn(submissionContext);
     resourceManager.getRMContext().getRMApps()
         .put(id.getApplicationId(), rmApp);
 
@@ -206,13 +217,20 @@ public class FairSchedulerTestBase {
   protected void createApplicationWithAMResource(ApplicationAttemptId attId,
       String queue, String user, Resource amResource) {
     RMContext rmContext = resourceManager.getRMContext();
-    RMApp rmApp = new RMAppImpl(attId.getApplicationId(), rmContext, conf,
-        null, null, null, ApplicationSubmissionContext.newInstance(null, null,
-        null, null, null, false, false, 0, amResource, null), null, null,
+    ApplicationId appId = attId.getApplicationId();
+    RMApp rmApp = new RMAppImpl(appId, rmContext, conf,
+        null, user, null, ApplicationSubmissionContext.newInstance(appId, null,
+        queue, null, null, false, false, 0, amResource, null), null, null,
         0, null, null, null);
-    rmContext.getRMApps().put(attId.getApplicationId(), rmApp);
+    rmContext.getRMApps().put(appId, rmApp);
+    RMAppEvent event = new RMAppEvent(appId, RMAppEventType.START);
+    resourceManager.getRMContext().getRMApps().get(appId).handle(event);
+    event = new RMAppEvent(appId, RMAppEventType.APP_NEW_SAVED);
+    resourceManager.getRMContext().getRMApps().get(appId).handle(event);
+    event = new RMAppEvent(appId, RMAppEventType.APP_ACCEPTED);
+    resourceManager.getRMContext().getRMApps().get(appId).handle(event);
     AppAddedSchedulerEvent appAddedEvent = new AppAddedSchedulerEvent(
-        attId.getApplicationId(), queue, user);
+        appId, queue, user);
     scheduler.handle(appAddedEvent);
     AppAttemptAddedSchedulerEvent attempAddedEvent =
         new AppAttemptAddedSchedulerEvent(attId, false);
@@ -227,6 +245,9 @@ public class FairSchedulerTestBase {
     RMAppAttemptMetrics attemptMetric = mock(RMAppAttemptMetrics.class);
     when(attempt.getRMAppAttemptMetrics()).thenReturn(attemptMetric);
     when(app.getCurrentAppAttempt()).thenReturn(attempt);
+    ApplicationSubmissionContext submissionContext = mock(ApplicationSubmissionContext.class);
+    when(submissionContext.getUnmanagedAM()).thenReturn(false);
+    when(attempt.getSubmissionContext()).thenReturn(submissionContext);
     resourceManager.getRMContext().getRMApps()
         .put(attemptId.getApplicationId(), app);
     return app;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ea36299/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 e4583d1..a454801 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
@@ -39,6 +39,7 @@ import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
@@ -1184,6 +1185,9 @@ public class TestFifoScheduler {
     RMAppAttemptMetrics attemptMetric = mock(RMAppAttemptMetrics.class);
     when(attempt.getRMAppAttemptMetrics()).thenReturn(attemptMetric);
     when(app.getCurrentAppAttempt()).thenReturn(attempt);
+    ApplicationSubmissionContext submissionContext = mock(ApplicationSubmissionContext.class);
+    when(submissionContext.getUnmanagedAM()).thenReturn(false);
+    when(attempt.getSubmissionContext()).thenReturn(submissionContext);
     context.getRMApps().putIfAbsent(attemptId.getApplicationId(), app);
     return app;
   }


[20/50] hadoop git commit: MAPREDUCE-6426. TestShuffleHandler#testGetMapOutputInfo is failing. Contributed by zhihai xu.

Posted by ar...@apache.org.
MAPREDUCE-6426. TestShuffleHandler#testGetMapOutputInfo is failing.
Contributed by zhihai xu.


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

Branch: refs/heads/HDFS-7240
Commit: fffb15bb431fcdd7ca4ce0e249f9d45f4968497b
Parents: 63d0365
Author: Devaraj K <de...@apache.org>
Authored: Thu Jul 9 15:00:26 2015 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Thu Jul 9 15:00:26 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                           | 3 +++
 .../test/java/org/apache/hadoop/mapred/TestShuffleHandler.java | 6 +++++-
 2 files changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fffb15bb/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 874ecea..81b202b 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -548,6 +548,9 @@ Release 2.7.2 - UNRELEASED
     MAPREDUCE-6425. ShuffleHandler passes wrong "base" parameter to getMapOutputInfo
     if mapId is not in the cache. (zhihai xu via devaraj)
 
+    MAPREDUCE-6426. TestShuffleHandler#testGetMapOutputInfo is failing.
+    (zhihai xu via devaraj)
+
 Release 2.7.1 - 2015-07-06 
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fffb15bb/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java
index 746071f..bad9b2d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java
@@ -837,6 +837,9 @@ public class TestShuffleHandler {
     Configuration conf = new Configuration();
     conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0);
     conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3);
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+        "simple");
+    UserGroupInformation.setConfiguration(conf);
     File absLogDir = new File("target", TestShuffleHandler.class.
         getSimpleName() + "LocDir").getAbsoluteFile();
     conf.set(YarnConfiguration.NM_LOCAL_DIRS, absLogDir.getAbsolutePath());
@@ -924,7 +927,8 @@ public class TestShuffleHandler {
       } catch (EOFException e) {
         // ignore
       }
-      Assert.assertEquals(failures.size(), 0);
+      Assert.assertEquals("sendError called due to shuffle error",
+          0, failures.size());
     } finally {
       shuffleHandler.stop();
       FileUtil.fullyDelete(absLogDir);


[10/50] hadoop git commit: HDFS-8712. Remove 'public' and 'abstract' modifiers in FsVolumeSpi and FsDatasetSpi (Contributed by Lei (Eddy) Xu)

Posted by ar...@apache.org.
HDFS-8712. Remove 'public' and 'abstract' modifiers in FsVolumeSpi and FsDatasetSpi (Contributed by Lei (Eddy) Xu)


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

Branch: refs/heads/HDFS-7240
Commit: bd4e10900cc53a2768c31cc29fdb3698684bc2a0
Parents: d632574
Author: Vinayakumar B <vi...@apache.org>
Authored: Wed Jul 8 16:34:54 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Wed Jul 8 16:34:54 2015 +0530

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../server/datanode/fsdataset/FsDatasetSpi.java | 149 ++++++++++---------
 .../server/datanode/fsdataset/FsVolumeSpi.java  |  47 +++---
 3 files changed, 101 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd4e1090/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 db77d0b..b88b42a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -707,6 +707,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8620. Clean up the checkstyle warinings about ClientProtocol.
     (Takanobu Asanuma via wheat9)
 
+    HDFS-8712. Remove 'public' and 'abstracta modifiers in FsVolumeSpi and
+    FsDatasetSpi (Lei (Eddy) Xu via vinayakumarb)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd4e1090/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
index 76c4f02..af6a532 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
@@ -72,7 +72,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
   /**
    * A factory for creating {@link FsDatasetSpi} objects.
    */
-  public static abstract class Factory<D extends FsDatasetSpi<?>> {
+  abstract class Factory<D extends FsDatasetSpi<?>> {
     /** @return the configured factory. */
     public static Factory<?> getFactory(Configuration conf) {
       @SuppressWarnings("rawtypes")
@@ -182,7 +182,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * The caller must release the reference of each volume by calling
    * {@link FsVolumeReferences#close()}.
    */
-  public FsVolumeReferences getFsVolumeReferences();
+  FsVolumeReferences getFsVolumeReferences();
 
   /**
    * Add a new volume to the FsDataset.<p/>
@@ -193,7 +193,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param location      The storage location for the new volume.
    * @param nsInfos       Namespace information for the new volume.
    */
-  public void addVolume(
+  void addVolume(
       final StorageLocation location,
       final List<NamespaceInfo> nsInfos) throws IOException;
 
@@ -207,20 +207,20 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param clearFailure set true to clear the failure information about the
    *                     volumes.
    */
-  public void removeVolumes(Set<File> volumes, boolean clearFailure);
+  void removeVolumes(Set<File> volumes, boolean clearFailure);
 
   /** @return a storage with the given storage ID */
-  public DatanodeStorage getStorage(final String storageUuid);
+  DatanodeStorage getStorage(final String storageUuid);
 
   /** @return one or more storage reports for attached volumes. */
-  public StorageReport[] getStorageReports(String bpid)
+  StorageReport[] getStorageReports(String bpid)
       throws IOException;
 
   /** @return the volume that contains a replica of the block. */
-  public V getVolume(ExtendedBlock b);
+  V getVolume(ExtendedBlock b);
 
   /** @return a volume information map (name => info). */
-  public Map<String, Object> getVolumeInfoMap();
+  Map<String, Object> getVolumeInfoMap();
 
   /**
    * Returns info about volume failures.
@@ -230,17 +230,17 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
   VolumeFailureSummary getVolumeFailureSummary();
 
   /** @return a list of finalized blocks for the given block pool. */
-  public List<FinalizedReplica> getFinalizedBlocks(String bpid);
+  List<FinalizedReplica> getFinalizedBlocks(String bpid);
 
   /** @return a list of finalized blocks for the given block pool. */
-  public List<FinalizedReplica> getFinalizedBlocksOnPersistentStorage(String bpid);
+  List<FinalizedReplica> getFinalizedBlocksOnPersistentStorage(String bpid);
 
   /**
    * Check whether the in-memory block record matches the block on the disk,
    * and, in case that they are not matched, update the record or mark it
    * as corrupted.
    */
-  public void checkAndUpdate(String bpid, long blockId, File diskFile,
+  void checkAndUpdate(String bpid, long blockId, File diskFile,
       File diskMetaFile, FsVolumeSpi vol) throws IOException;
 
   /**
@@ -249,15 +249,15 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    *         otherwise, return null.
    * @throws IOException
    */
-  public LengthInputStream getMetaDataInputStream(ExtendedBlock b
+  LengthInputStream getMetaDataInputStream(ExtendedBlock b
       ) throws IOException;
 
   /**
-   * Returns the specified block's on-disk length (excluding metadata)
+   * Returns the specified block's on-disk length (excluding metadata).
    * @return   the specified block's on-disk length (excluding metadta)
    * @throws IOException on error
    */
-  public long getLength(ExtendedBlock b) throws IOException;
+  long getLength(ExtendedBlock b) throws IOException;
 
   /**
    * Get reference to the replica meta info in the replicasMap. 
@@ -265,47 +265,48 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return replica from the replicas map
    */
   @Deprecated
-  public Replica getReplica(String bpid, long blockId);
+  Replica getReplica(String bpid, long blockId);
 
   /**
    * @return replica meta information
    */
-  public String getReplicaString(String bpid, long blockId);
+  String getReplicaString(String bpid, long blockId);
 
   /**
    * @return the generation stamp stored with the block.
    */
-  public Block getStoredBlock(String bpid, long blkid) throws IOException;
-  
+  Block getStoredBlock(String bpid, long blkid) throws IOException;
+
   /**
-   * Returns an input stream at specified offset of the specified block
+   * Returns an input stream at specified offset of the specified block.
    * @param b block
    * @param seekOffset offset with in the block to seek to
    * @return an input stream to read the contents of the specified block,
    *  starting at the offset
    * @throws IOException
    */
-  public InputStream getBlockInputStream(ExtendedBlock b, long seekOffset)
+  InputStream getBlockInputStream(ExtendedBlock b, long seekOffset)
             throws IOException;
 
   /**
-   * Returns an input stream at specified offset of the specified block
+   * Returns an input stream at specified offset of the specified block.
    * The block is still in the tmp directory and is not finalized
    * @return an input stream to read the contents of the specified block,
    *  starting at the offset
    * @throws IOException
    */
-  public ReplicaInputStreams getTmpInputStreams(ExtendedBlock b, long blkoff,
+  ReplicaInputStreams getTmpInputStreams(ExtendedBlock b, long blkoff,
       long ckoff) throws IOException;
 
   /**
    * Creates a temporary replica and returns the meta information of the replica
+   * .
    * 
    * @param b block
    * @return the meta info of the replica which is being written to
    * @throws IOException if an error occurs
    */
-  public ReplicaHandler createTemporary(StorageType storageType,
+  ReplicaHandler createTemporary(StorageType storageType,
       ExtendedBlock b) throws IOException;
 
   /**
@@ -315,11 +316,11 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return the meta info of the replica which is being written to
    * @throws IOException if an error occurs
    */
-  public ReplicaHandler createRbw(StorageType storageType,
+  ReplicaHandler createRbw(StorageType storageType,
       ExtendedBlock b, boolean allowLazyPersist) throws IOException;
 
   /**
-   * Recovers a RBW replica and returns the meta info of the replica
+   * Recovers a RBW replica and returns the meta info of the replica.
    * 
    * @param b block
    * @param newGS the new generation stamp for the replica
@@ -328,7 +329,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return the meta info of the replica which is being written to
    * @throws IOException if an error occurs
    */
-  public ReplicaHandler recoverRbw(ExtendedBlock b,
+  ReplicaHandler recoverRbw(ExtendedBlock b,
       long newGS, long minBytesRcvd, long maxBytesRcvd) throws IOException;
 
   /**
@@ -336,11 +337,11 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param temporary the temporary replica being converted
    * @return the result RBW
    */
-  public ReplicaInPipelineInterface convertTemporaryToRbw(
+  ReplicaInPipelineInterface convertTemporaryToRbw(
       ExtendedBlock temporary) throws IOException;
 
   /**
-   * Append to a finalized replica and returns the meta info of the replica
+   * Append to a finalized replica and returns the meta info of the replica.
    * 
    * @param b block
    * @param newGS the new generation stamp for the replica
@@ -348,12 +349,12 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return the meata info of the replica which is being written to
    * @throws IOException
    */
-  public ReplicaHandler append(ExtendedBlock b, long newGS,
+  ReplicaHandler append(ExtendedBlock b, long newGS,
       long expectedBlockLen) throws IOException;
 
   /**
-   * Recover a failed append to a finalized replica
-   * and returns the meta info of the replica
+   * Recover a failed append to a finalized replica and returns the meta
+   * info of the replica.
    * 
    * @param b block
    * @param newGS the new generation stamp for the replica
@@ -361,11 +362,11 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return the meta info of the replica which is being written to
    * @throws IOException
    */
-  public ReplicaHandler recoverAppend(
+  ReplicaHandler recoverAppend(
       ExtendedBlock b, long newGS, long expectedBlockLen) throws IOException;
   
   /**
-   * Recover a failed pipeline close
+   * Recover a failed pipeline close.
    * It bumps the replica's generation stamp and finalize it if RBW replica
    * 
    * @param b block
@@ -374,7 +375,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return the storage uuid of the replica.
    * @throws IOException
    */
-  public String recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen
+  String recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen
       ) throws IOException;
   
   /**
@@ -386,21 +387,21 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * block is been finalized. For instance, the block resides on an HDFS volume
    * that has been removed.
    */
-  public void finalizeBlock(ExtendedBlock b) throws IOException;
+  void finalizeBlock(ExtendedBlock b) throws IOException;
 
   /**
    * Unfinalizes the block previously opened for writing using writeToBlock.
    * The temporary file associated with this block is deleted.
    * @throws IOException
    */
-  public void unfinalizeBlock(ExtendedBlock b) throws IOException;
+  void unfinalizeBlock(ExtendedBlock b) throws IOException;
 
   /**
    * Returns one block report per volume.
    * @param bpid Block Pool Id
    * @return - a map of DatanodeStorage to block report for the volume.
    */
-  public Map<DatanodeStorage, BlockListAsLongs> getBlockReports(String bpid);
+  Map<DatanodeStorage, BlockListAsLongs> getBlockReports(String bpid);
 
   /**
    * Returns the cache report - the full list of cached block IDs of a
@@ -408,10 +409,10 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param   bpid Block Pool Id
    * @return  the cache report - the full list of cached block IDs.
    */
-  public List<Long> getCacheReport(String bpid);
+  List<Long> getCacheReport(String bpid);
 
   /** Does the dataset contain the block? */
-  public boolean contains(ExtendedBlock block);
+  boolean contains(ExtendedBlock block);
 
   /**
    * Check if a block is valid.
@@ -431,7 +432,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * 
    * @throws IOException                       May be thrown from the methods called. 
    */
-  public void checkBlock(ExtendedBlock b, long minLength, ReplicaState state)
+  void checkBlock(ExtendedBlock b, long minLength, ReplicaState state)
       throws ReplicaNotFoundException, UnexpectedReplicaStateException,
       FileNotFoundException, EOFException, IOException;
       
@@ -440,13 +441,13 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * Is the block valid?
    * @return - true if the specified block is valid
    */
-  public boolean isValidBlock(ExtendedBlock b);
+  boolean isValidBlock(ExtendedBlock b);
 
   /**
    * Is the block a valid RBW?
    * @return - true if the specified block is a valid RBW
    */
-  public boolean isValidRbw(ExtendedBlock b);
+  boolean isValidRbw(ExtendedBlock b);
 
   /**
    * Invalidates the specified blocks
@@ -454,21 +455,21 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param invalidBlks - the blocks to be invalidated
    * @throws IOException
    */
-  public void invalidate(String bpid, Block invalidBlks[]) throws IOException;
+  void invalidate(String bpid, Block invalidBlks[]) throws IOException;
 
   /**
    * Caches the specified blocks
    * @param bpid Block pool id
    * @param blockIds - block ids to cache
    */
-  public void cache(String bpid, long[] blockIds);
+  void cache(String bpid, long[] blockIds);
 
   /**
    * Uncaches the specified blocks
    * @param bpid Block pool id
    * @param blockIds - blocks ids to uncache
    */
-  public void uncache(String bpid, long[] blockIds);
+  void uncache(String bpid, long[] blockIds);
 
   /**
    * Determine if the specified block is cached.
@@ -476,18 +477,18 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param blockIds - block id
    * @return true if the block is cached
    */
-  public boolean isCached(String bpid, long blockId);
+  boolean isCached(String bpid, long blockId);
 
     /**
      * Check if all the data directories are healthy
      * @return A set of unhealthy data directories.
      */
-  public Set<File> checkDataDir();
+  Set<File> checkDataDir();
 
   /**
    * Shutdown the FSDataset
    */
-  public void shutdown();
+  void shutdown();
 
   /**
    * Sets the file pointer of the checksum stream so that the last checksum
@@ -497,7 +498,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param checksumSize number of bytes each checksum has
    * @throws IOException
    */
-  public void adjustCrcChannelPosition(ExtendedBlock b,
+  void adjustCrcChannelPosition(ExtendedBlock b,
       ReplicaOutputStreams outs, int checksumSize) throws IOException;
 
   /**
@@ -505,7 +506,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return true if more than the minimum number of valid volumes are left 
    * in the FSDataSet.
    */
-  public boolean hasEnoughResource();
+  boolean hasEnoughResource();
 
   /**
    * Get visible length of the specified replica.
@@ -517,14 +518,14 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return actual state of the replica on this data-node or 
    * null if data-node does not have the replica.
    */
-  public ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock
+  ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock
       ) throws IOException;
 
   /**
    * Update replica's generation stamp and length and finalize it.
    * @return the ID of storage that stores the block
    */
-  public String updateReplicaUnderRecovery(ExtendedBlock oldBlock,
+  String updateReplicaUnderRecovery(ExtendedBlock oldBlock,
       long recoveryId, long newBlockId, long newLength) throws IOException;
 
   /**
@@ -532,14 +533,14 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param bpid Block pool Id
    * @param conf Configuration
    */
-  public void addBlockPool(String bpid, Configuration conf) throws IOException;
-  
+  void addBlockPool(String bpid, Configuration conf) throws IOException;
+
   /**
    * Shutdown and remove the block pool from underlying storage.
    * @param bpid Block pool Id to be removed
    */
-  public void shutdownBlockPool(String bpid) ;
-  
+  void shutdownBlockPool(String bpid) ;
+
   /**
    * Deletes the block pool directories. If force is false, directories are 
    * deleted only if no block files exist for the block pool. If force 
@@ -551,12 +552,12 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    *        directory for the blockpool is deleted along with its contents.
    * @throws IOException
    */
-  public void deleteBlockPool(String bpid, boolean force) throws IOException;
-  
+  void deleteBlockPool(String bpid, boolean force) throws IOException;
+
   /**
    * Get {@link BlockLocalPathInfo} for the given block.
    */
-  public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b
+  BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b
       ) throws IOException;
 
   /**
@@ -568,7 +569,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @return metadata Metadata for the list of blocks
    * @throws IOException
    */
-  public HdfsBlocksMetadata getHdfsBlocksMetadata(String bpid,
+  HdfsBlocksMetadata getHdfsBlocksMetadata(String bpid,
       long[] blockIds) throws IOException;
 
   /**
@@ -576,51 +577,51 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * moved to a separate trash directory instead of being deleted immediately.
    * This can be useful for example during rolling upgrades.
    */
-  public void enableTrash(String bpid);
+  void enableTrash(String bpid);
 
   /**
    * Clear trash
    */
-  public void clearTrash(String bpid);
+  void clearTrash(String bpid);
 
   /**
    * @return true when trash is enabled
    */
-  public boolean trashEnabled(String bpid);
+  boolean trashEnabled(String bpid);
 
   /**
    * Create a marker file indicating that a rolling upgrade is in progress.
    */
-  public void setRollingUpgradeMarker(String bpid) throws IOException;
+  void setRollingUpgradeMarker(String bpid) throws IOException;
 
   /**
    * Delete the rolling upgrade marker file if it exists.
    * @param bpid
    */
-  public void clearRollingUpgradeMarker(String bpid) throws IOException;
+  void clearRollingUpgradeMarker(String bpid) throws IOException;
 
   /**
-   * submit a sync_file_range request to AsyncDiskService
+   * submit a sync_file_range request to AsyncDiskService.
    */
-  public void submitBackgroundSyncFileRangeRequest(final ExtendedBlock block,
+  void submitBackgroundSyncFileRangeRequest(final ExtendedBlock block,
       final FileDescriptor fd, final long offset, final long nbytes,
       final int flags);
 
   /**
    * Callback from RamDiskAsyncLazyPersistService upon async lazy persist task end
    */
-   public void onCompleteLazyPersist(String bpId, long blockId,
+  void onCompleteLazyPersist(String bpId, long blockId,
       long creationTime, File[] savedFiles, V targetVolume);
 
    /**
     * Callback from RamDiskAsyncLazyPersistService upon async lazy persist task fail
     */
-   public void onFailLazyPersist(String bpId, long blockId);
+   void onFailLazyPersist(String bpId, long blockId);
 
     /**
      * Move block from one storage to another storage
      */
-   public ReplicaInfo moveBlockAcrossStorage(final ExtendedBlock block,
+   ReplicaInfo moveBlockAcrossStorage(final ExtendedBlock block,
         StorageType targetStorageType) throws IOException;
 
   /**
@@ -629,15 +630,15 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    *
    * It is a no-op when dfs.datanode.block-pinning.enabled is set to false.
    */
-  public void setPinning(ExtendedBlock block) throws IOException;
+  void setPinning(ExtendedBlock block) throws IOException;
 
   /**
    * Check whether the block was pinned
    */
-  public boolean getPinning(ExtendedBlock block) throws IOException;
-  
+  boolean getPinning(ExtendedBlock block) throws IOException;
+
   /**
    * Confirm whether the block is deleting
    */
-  public boolean isDeletingBlock(String bpid, long blockId);
+  boolean isDeletingBlock(String bpid, long blockId);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd4e1090/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
index 8d1bb2a..ee01924 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
@@ -39,38 +39,38 @@ public interface FsVolumeSpi {
   FsVolumeReference obtainReference() throws ClosedChannelException;
 
   /** @return the StorageUuid of the volume */
-  public String getStorageID();
+  String getStorageID();
 
   /** @return a list of block pools. */
-  public String[] getBlockPoolList();
+  String[] getBlockPoolList();
 
   /** @return the available storage space in bytes. */
-  public long getAvailable() throws IOException;
+  long getAvailable() throws IOException;
 
   /** @return the base path to the volume */
-  public String getBasePath();
+  String getBasePath();
 
   /** @return the path to the volume */
-  public String getPath(String bpid) throws IOException;
+  String getPath(String bpid) throws IOException;
 
   /** @return the directory for the finalized blocks in the block pool. */
-  public File getFinalizedDir(String bpid) throws IOException;
+  File getFinalizedDir(String bpid) throws IOException;
   
-  public StorageType getStorageType();
+  StorageType getStorageType();
 
   /** Returns true if the volume is NOT backed by persistent storage. */
-  public boolean isTransientStorage();
+  boolean isTransientStorage();
 
   /**
    * Reserve disk space for an RBW block so a writer does not run out of
    * space before the block is full.
    */
-  public void reserveSpaceForRbw(long bytesToReserve);
+  void reserveSpaceForRbw(long bytesToReserve);
 
   /**
    * Release disk space previously reserved for RBW block.
    */
-  public void releaseReservedSpace(long bytesToRelease);
+  void releaseReservedSpace(long bytesToRelease);
 
   /**
    * Release reserved memory for an RBW block written to transient storage
@@ -78,7 +78,7 @@ public interface FsVolumeSpi {
    * bytesToRelease will be rounded down to the OS page size since locked
    * memory reservation must always be a multiple of the page size.
    */
-  public void releaseLockedMemory(long bytesToRelease);
+  void releaseLockedMemory(long bytesToRelease);
 
   /**
    * BlockIterator will return ExtendedBlock entries from a block pool in
@@ -90,7 +90,7 @@ public interface FsVolumeSpi {
    *
    * Closing the iterator does not save it.  You must call save to save it.
    */
-  public interface BlockIterator extends Closeable {
+  interface BlockIterator extends Closeable {
     /**
      * Get the next block.<p/>
      *
@@ -107,17 +107,17 @@ public interface FsVolumeSpi {
      *                         this volume.  In this case, EOF will be set on
      *                         the iterator.
      */
-    public ExtendedBlock nextBlock() throws IOException;
+    ExtendedBlock nextBlock() throws IOException;
 
     /**
      * Returns true if we got to the end of the block pool.
      */
-    public boolean atEnd();
+    boolean atEnd();
 
     /**
      * Repositions the iterator at the beginning of the block pool.
      */
-    public void rewind();
+    void rewind();
 
     /**
      * Save this block iterator to the underlying volume.
@@ -127,7 +127,7 @@ public interface FsVolumeSpi {
      * @throws IOException   If there was an error when saving the block
      *                         iterator.
      */
-    public void save() throws IOException;
+    void save() throws IOException;
 
     /**
      * Set the maximum staleness of entries that we will return.<p/>
@@ -138,25 +138,25 @@ public interface FsVolumeSpi {
      * to 0, consumers of this API must handle race conditions where block
      * disappear before they can be processed.
      */
-    public void setMaxStalenessMs(long maxStalenessMs);
+    void setMaxStalenessMs(long maxStalenessMs);
 
     /**
      * Get the wall-clock time, measured in milliseconds since the Epoch,
      * when this iterator was created.
      */
-    public long getIterStartMs();
+    long getIterStartMs();
 
     /**
      * Get the wall-clock time, measured in milliseconds since the Epoch,
      * when this iterator was last saved.  Returns iterStartMs if the
      * iterator was never saved.
      */
-    public long getLastSavedMs();
+    long getLastSavedMs();
 
     /**
      * Get the id of the block pool which this iterator traverses.
      */
-    public String getBlockPoolId();
+    String getBlockPoolId();
   }
 
   /**
@@ -168,7 +168,7 @@ public interface FsVolumeSpi {
    *
    * @return                 The new block iterator.
    */
-  public BlockIterator newBlockIterator(String bpid, String name);
+  BlockIterator newBlockIterator(String bpid, String name);
 
   /**
    * Load a saved block iterator.
@@ -180,11 +180,10 @@ public interface FsVolumeSpi {
    * @throws IOException     If there was an IO error loading the saved
    *                           block iterator.
    */
-  public BlockIterator loadBlockIterator(String bpid, String name)
-      throws IOException;
+  BlockIterator loadBlockIterator(String bpid, String name) throws IOException;
 
   /**
    * Get the FSDatasetSpi which this volume is a part of.
    */
-  public FsDatasetSpi getDataset();
+  FsDatasetSpi getDataset();
 }


[29/50] hadoop git commit: HDFS-8749. Fix findbugs warnings in BlockManager.java. Contributed by Brahma Reddy Battula.

Posted by ar...@apache.org.
HDFS-8749. Fix findbugs warnings in BlockManager.java. Contributed by Brahma Reddy Battula.


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

Branch: refs/heads/HDFS-7240
Commit: d66302ed9b2c25b560d8319d6d755aee7cfa4d67
Parents: 5214876
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Jul 10 15:04:06 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Jul 10 15:04:06 2015 +0900

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../apache/hadoop/hdfs/server/blockmanagement/BlockManager.java   | 2 --
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d66302ed/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 e26e061..5c1208d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1026,6 +1026,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8729. Fix TestFileTruncate#testTruncateWithDataNodesRestartImmediately
     which occasionally failed. (Walter Su via jing9)
 
+    HDFS-8749. Fix findbugs warnings in BlockManager.java.
+    (Brahma Reddy Battula via aajisaka)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d66302ed/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 0b60a97..7dce2a8 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
@@ -3596,8 +3596,6 @@ public class BlockManager implements BlockStatsMXBean {
       String src, BlockInfo[] blocks) {
     for (BlockInfo b: blocks) {
       if (!b.isComplete()) {
-        final BlockInfoUnderConstruction uc =
-            (BlockInfoUnderConstruction)b;
         final int numNodes = b.numNodes();
         final int min = getMinStorageNum(b);
         final BlockUCState state = b.getBlockUCState();