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 zh...@apache.org on 2015/09/23 05:40:35 UTC

[01/50] [abbrv] hadoop git commit: YARN-4149. yarn logs -am should provide an option to fetch all the log files. Contributed by Varun Vasudev

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7285 6fc942455 -> 7bff8ca1c (forced update)


YARN-4149. yarn logs -am should provide an option to fetch all the log
files. Contributed by Varun Vasudev


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

Branch: refs/heads/HDFS-7285
Commit: 34ef1a092bcab369bb845481efffb8c47adef53a
Parents: 8c1cdb1
Author: Xuan <xg...@apache.org>
Authored: Tue Sep 15 14:36:30 2015 -0700
Committer: Xuan <xg...@apache.org>
Committed: Tue Sep 15 14:36:30 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../apache/hadoop/yarn/client/cli/LogsCLI.java  | 104 +++++++++++++++++--
 .../hadoop/yarn/client/cli/TestLogsCLI.java     |   3 +-
 .../nodemanager/webapp/NMWebServices.java       |  12 ++-
 .../nodemanager/webapp/dao/ContainerInfo.java   |  46 ++++++--
 .../webapp/TestNMWebServicesContainers.java     |  13 ++-
 6 files changed, 157 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/34ef1a09/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index b2fb780..b044078 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -439,6 +439,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3717. Expose app/am/queue's node-label-expression to RM web UI / 
     CLI / REST-API. (Naganarasimha G R via wangda)
 
+    YARN-4149. yarn logs -am should provide an option to fetch all the log files
+    (Varun Vasudev via xgong)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/34ef1a09/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
index 25481f8..edee8ee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
@@ -19,12 +19,15 @@
 package org.apache.hadoop.yarn.client.cli;
 
 import java.io.IOException;
+import java.io.StringReader;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
 import javax.ws.rs.core.MediaType;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
@@ -61,6 +64,9 @@ import com.sun.jersey.api.client.ClientHandlerException;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.UniformInterfaceException;
 import com.sun.jersey.api.client.WebResource;
+import org.w3c.dom.Document;
+import org.w3c.dom.NodeList;
+import org.xml.sax.InputSource;
 
 @Public
 @Evolving
@@ -105,7 +111,8 @@ public class LogsCLI extends Configured implements Tool {
     opts.addOption(amOption);
     Option logFileOpt = new Option(CONTAINER_LOG_FILES, true,
       "Work with -am/-containerId and specify comma-separated value "
-      + "to get specified Container log files");
+        + "to get specified container log files. Use \"ALL\" to fetch all the "
+        + "log files for the container.");
     logFileOpt.setValueSeparator(',');
     logFileOpt.setArgs(Option.UNLIMITED_VALUES);
     logFileOpt.setArgName("Log File Name");
@@ -248,8 +255,8 @@ public class LogsCLI extends Configured implements Tool {
             logFiles, logCliHelper, appOwner, true);
         } else {
           System.out
-            .println("Can not get AMContainers logs for the application:"
-                + appId);
+            .println(
+                "Can not get AMContainers logs for the application:" + appId);
           System.out.println("This application:" + appId + " is finished."
               + " Please enable the application history service. Or Using "
               + "yarn logs -applicationId <appId> -containerId <containerId> "
@@ -264,9 +271,18 @@ public class LogsCLI extends Configured implements Tool {
       // if we provide the node address and the application is in the final
       // state, we could directly get logs from HDFS.
       if (nodeAddress != null && isApplicationFinished(appState)) {
+        // if user specified "ALL" as the logFiles param, pass null
+        // to logCliHelper so that it fetches all the logs
+        List<String> logs;
+        if (logFiles == null) {
+          logs = null;
+        } else if (fetchAllLogFiles(logFiles)) {
+          logs = null;
+        } else {
+          logs = Arrays.asList(logFiles);
+        }
         return logCliHelper.dumpAContainersLogsForALogType(appIdStr,
-            containerIdStr, nodeAddress, appOwner, logFiles == null ? null
-                : Arrays.asList(logFiles));
+            containerIdStr, nodeAddress, appOwner, logs);
       }
       try {
         // If the nodeAddress is not provided, we will try to get
@@ -288,10 +304,14 @@ public class LogsCLI extends Configured implements Tool {
             containerIdStr, nodeHttpAddress, nodeId, logFiles, logCliHelper,
             appOwner);
         } else {
+          String [] requestedLogFiles = logFiles;
+          if(fetchAllLogFiles(logFiles)) {
+            requestedLogFiles = null;
+          }
           // If the application is in the final state, we will directly
           // get the container logs from HDFS.
           printContainerLogsForFinishedApplication(appIdStr, containerIdStr,
-            nodeId, logFiles, logCliHelper, appOwner);
+            nodeId, requestedLogFiles, logCliHelper, appOwner);
         }
         return resultCode;
       } catch (IOException | YarnException ex) {
@@ -401,15 +421,69 @@ public class LogsCLI extends Configured implements Tool {
     return amContainersList;
   }
 
+  private boolean fetchAllLogFiles(String[] logFiles) {
+    if(logFiles != null) {
+      List<String> logs = Arrays.asList(logFiles);
+      if(logs.contains("ALL")) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private String[] getContainerLogFiles(Configuration conf,
+      String containerIdStr, String nodeHttpAddress) throws IOException {
+    List<String> logFiles = new ArrayList<>();
+    Client webServiceClient = Client.create();
+    try {
+      WebResource webResource = webServiceClient
+          .resource(WebAppUtils.getHttpSchemePrefix(conf) + nodeHttpAddress);
+      ClientResponse response =
+          webResource.path("ws").path("v1").path("node").path("containers")
+              .path(containerIdStr).accept(MediaType.APPLICATION_XML)
+              .get(ClientResponse.class);
+      if (response.getClientResponseStatus().equals(ClientResponse.Status.OK)) {
+        try {
+          String xml = response.getEntity(String.class);
+          DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
+          DocumentBuilder db = dbf.newDocumentBuilder();
+          InputSource is = new InputSource();
+          is.setCharacterStream(new StringReader(xml));
+          Document dom = db.parse(is);
+          NodeList elements = dom.getElementsByTagName("containerLogFiles");
+          for (int i = 0; i < elements.getLength(); i++) {
+            logFiles.add(elements.item(i).getTextContent());
+          }
+        } catch (Exception e) {
+          System.out.println("Unable to parse xml from webservice. Error:");
+          System.out.println(e.getMessage());
+          throw new IOException(e);
+        }
+      }
+
+    } catch (ClientHandlerException | UniformInterfaceException ex) {
+      System.out.println("Unable to fetch log files list");
+      throw new IOException(ex);
+    }
+    return logFiles.toArray(new String[0]);
+  }
+
   private void printContainerLogsFromRunningApplication(Configuration conf,
       String appId, String containerIdStr, String nodeHttpAddress,
       String nodeId, String[] logFiles, LogCLIHelpers logCliHelper,
       String appOwner) throws IOException {
+    String [] requestedLogFiles = logFiles;
+    // fetch all the log files for the container
+    if (fetchAllLogFiles(logFiles)) {
+      requestedLogFiles =
+          getContainerLogFiles(getConf(), containerIdStr, nodeHttpAddress);
+    }
     Client webServiceClient = Client.create();
     String containerString = "\n\nContainer: " + containerIdStr;
     System.out.println(containerString);
     System.out.println(StringUtils.repeat("=", containerString.length()));
-    for (String logFile : logFiles) {
+
+    for (String logFile : requestedLogFiles) {
       System.out.println("LogType:" + logFile);
       System.out.println("Log Upload Time:"
           + Times.format(System.currentTimeMillis()));
@@ -432,7 +506,7 @@ public class LogsCLI extends Configured implements Tool {
     }
     // for the case, we have already uploaded partial logs in HDFS
     logCliHelper.dumpAContainersLogsForALogType(appId, containerIdStr, nodeId,
-      appOwner, Arrays.asList(logFiles));
+      appOwner, Arrays.asList(requestedLogFiles));
   }
 
   private void printContainerLogsForFinishedApplication(String appId,
@@ -551,15 +625,25 @@ public class LogsCLI extends Configured implements Tool {
           }
         }
         if (nodeId != null && !nodeId.isEmpty()) {
+          String [] requestedLogFilesList = null;
+          if(!fetchAllLogFiles(logFiles)) {
+            requestedLogFilesList = logFiles;
+          }
           printContainerLogsForFinishedApplication(appId, containerId, nodeId,
-            logFiles, logCliHelper, appOwner);
+            requestedLogFilesList, logCliHelper, appOwner);
         }
       }
     } else {
       if (nodeHttpAddress != null && containerId != null
           && !nodeHttpAddress.isEmpty() && !containerId.isEmpty()) {
+        String [] requestedLogFiles = logFiles;
+        // fetch all the log files for the AM
+        if (fetchAllLogFiles(logFiles)) {
+          requestedLogFiles =
+              getContainerLogFiles(getConf(), containerId, nodeHttpAddress);
+        }
         printContainerLogsFromRunningApplication(conf, appId, containerId,
-          nodeHttpAddress, nodeId, logFiles, logCliHelper, appOwner);
+          nodeHttpAddress, nodeId, requestedLogFiles, logCliHelper, appOwner);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/34ef1a09/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
index a353811..aec7cae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
@@ -168,7 +168,8 @@ public class TestLogsCLI {
     pw.println(" -help                           Displays help for all commands.");
     pw.println(" -logFiles <Log File Name>       Work with -am/-containerId and specify");
     pw.println("                                 comma-separated value to get specified");
-    pw.println("                                 Container log files");
+    pw.println("                                 container log files. Use \"ALL\" to fetch");
+    pw.println("                                 all the log files for the container.");
     pw.println(" -nodeAddress <Node Address>     NodeAddress in the format nodename:port");
     pw.close();
     String appReportStr = baos.toString("UTF-8");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/34ef1a09/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
index 16f2c68..de6d219 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
@@ -129,7 +129,7 @@ public class NMWebServices {
           String msg = "Error: You must specify a non-empty string for the user";
           throw new BadRequestException(msg);
         }
-        if (!appInfo.getUser().toString().equals(userQuery)) {
+        if (!appInfo.getUser().equals(userQuery)) {
           continue;
         }
       }
@@ -158,7 +158,8 @@ public class NMWebServices {
   @GET
   @Path("/containers")
   @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
-  public ContainersInfo getNodeContainers() {
+  public ContainersInfo getNodeContainers(@javax.ws.rs.core.Context
+      HttpServletRequest hsr) {
     init();
     ContainersInfo allContainers = new ContainersInfo();
     for (Entry<ContainerId, Container> entry : this.nmContext.getContainers()
@@ -168,7 +169,7 @@ public class NMWebServices {
         continue;
       }
       ContainerInfo info = new ContainerInfo(this.nmContext, entry.getValue(),
-          uriInfo.getBaseUri().toString(), webapp.name());
+          uriInfo.getBaseUri().toString(), webapp.name(), hsr.getRemoteUser());
       allContainers.add(info);
     }
     return allContainers;
@@ -177,7 +178,8 @@ public class NMWebServices {
   @GET
   @Path("/containers/{containerid}")
   @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
-  public ContainerInfo getNodeContainer(@PathParam("containerid") String id) {
+  public ContainerInfo getNodeContainer(@javax.ws.rs.core.Context
+      HttpServletRequest hsr, @PathParam("containerid") String id) {
     ContainerId containerId = null;
     init();
     try {
@@ -191,7 +193,7 @@ public class NMWebServices {
       throw new NotFoundException("container with id, " + id + ", not found");
     }
     return new ContainerInfo(this.nmContext, container, uriInfo.getBaseUri()
-        .toString(), webapp.name());
+        .toString(), webapp.name(), hsr.getRemoteUser());
 
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/34ef1a09/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/ContainerInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/ContainerInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/ContainerInfo.java
index 78982de..e462eba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/ContainerInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/ContainerInfo.java
@@ -21,16 +21,19 @@ package org.apache.hadoop.yarn.server.nodemanager.webapp.dao;
 import static org.apache.hadoop.yarn.util.StringHelper.join;
 import static org.apache.hadoop.yarn.util.StringHelper.ujoin;
 
-import javax.xml.bind.annotation.XmlAccessType;
-import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlRootElement;
-import javax.xml.bind.annotation.XmlTransient;
+import javax.xml.bind.annotation.*;
 
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.webapp.ContainerLogsUtils;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
 
 @XmlRootElement(name = "container")
 @XmlAccessorType(XmlAccessType.FIELD)
@@ -50,15 +53,18 @@ public class ContainerInfo {
   @XmlTransient
   protected String exitStatus;
 
+  @XmlElementWrapper
+  protected List<String> containerLogFiles;
+
   public ContainerInfo() {
   } // JAXB needs this
 
   public ContainerInfo(final Context nmContext, final Container container) {
-    this(nmContext, container, "", "");
+    this(nmContext, container, "", "", "");
   }
 
   public ContainerInfo(final Context nmContext, final Container container,
-       String requestUri, String pathPrefix) {
+       String requestUri, String pathPrefix, String remoteUser) {
 
     this.id = container.getContainerId().toString();
     this.nodeId = nmContext.getNodeId().toString();
@@ -90,6 +96,8 @@ public class ContainerInfo {
     }
     this.containerLogsLink = join(requestUri, pathPrefix,
         this.containerLogsShortLink);
+    this.containerLogFiles =
+        getContainerLogFiles(container.getContainerId(), remoteUser, nmContext);
   }
 
   public String getId() {
@@ -136,4 +144,30 @@ public class ContainerInfo {
     return this.totalVCoresNeeded;
   }
 
+  public List<String> getContainerLogFiles() {
+    return this.containerLogFiles;
+  }
+
+  private List<String> getContainerLogFiles(ContainerId id, String remoteUser,
+      Context nmContext) {
+    List<String> logFiles = new ArrayList<>();
+    try {
+      List<File> logDirs =
+          ContainerLogsUtils.getContainerLogDirs(id, remoteUser, nmContext);
+      for (File containerLogsDir : logDirs) {
+        File[] logs = containerLogsDir.listFiles();
+        if (logs != null) {
+          for (File log : logs) {
+            if (log.isFile()) {
+              logFiles.add(log.getName());
+            }
+          }
+        }
+      }
+    } catch (Exception ye) {
+      return logFiles;
+    }
+    return logFiles;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/34ef1a09/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesContainers.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesContainers.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesContainers.java
index 3bfd440..0ed56d3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesContainers.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesContainers.java
@@ -32,6 +32,7 @@ import javax.ws.rs.core.MediaType;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
+import com.sun.jersey.api.client.filter.LoggingFilter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.util.NodeHealthScriptRunner;
@@ -245,6 +246,7 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
     Application app2 = new MockApp(2);
     nmContext.getApplications().put(app2.getAppId(), app2);
     addAppContainers(app2);
+    client().addFilter(new LoggingFilter());
 
     ClientResponse response = r.path("ws").path("v1").path("node").path(path)
         .accept(media).get(ClientResponse.class);
@@ -412,6 +414,7 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
     Application app2 = new MockApp(2);
     nmContext.getApplications().put(app2.getAppId(), app2);
     addAppContainers(app2);
+    client().addFilter(new LoggingFilter(System.out));
 
     for (String id : hash.keySet()) {
       ClientResponse response = r.path("ws").path("v1").path("node")
@@ -471,12 +474,16 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
           WebServicesTestUtils.getXmlInt(element, "totalMemoryNeededMB"),
           WebServicesTestUtils.getXmlInt(element, "totalVCoresNeeded"),
           WebServicesTestUtils.getXmlString(element, "containerLogsLink"));
+      // verify that the container log files element exists
+      assertTrue("containerLogFiles missing",
+          WebServicesTestUtils.getXmlString(element, "containerLogFiles")
+              != null);
     }
   }
 
   public void verifyNodeContainerInfo(JSONObject info, Container cont)
       throws JSONException, Exception {
-    assertEquals("incorrect number of elements", 9, info.length());
+    assertEquals("incorrect number of elements", 10, info.length());
 
     verifyNodeContainerInfoGeneric(cont, info.getString("id"),
         info.getString("state"), info.getString("user"),
@@ -484,6 +491,9 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
         info.getString("nodeId"), info.getInt("totalMemoryNeededMB"),
         info.getInt("totalVCoresNeeded"),
         info.getString("containerLogsLink"));
+    // verify that the container log files element exists
+    assertTrue("containerLogFiles missing",
+        info.getJSONArray("containerLogFiles") != null);
   }
 
   public void verifyNodeContainerInfoGeneric(Container cont, String id,
@@ -514,5 +524,4 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
             cont.getUser());
     assertTrue("containerLogsLink wrong", logsLink.contains(shortLink));
   }
-
 }


[25/50] [abbrv] hadoop git commit: HDFS-9063. Correctly handle snapshot path for getContentSummary. Contributed by Jing Zhao.

Posted by zh...@apache.org.
HDFS-9063. Correctly handle snapshot path for getContentSummary. Contributed by Jing Zhao.


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

Branch: refs/heads/HDFS-7285
Commit: 3f4275310203de4ccfb15337f3c503e25408a265
Parents: e690a32
Author: Jing Zhao <ji...@apache.org>
Authored: Fri Sep 18 09:26:33 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Fri Sep 18 09:26:33 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../server/namenode/FSDirStatAndListingOp.java  |   3 +-
 .../hadoop/hdfs/server/namenode/INode.java      |  14 ++-
 .../hdfs/server/namenode/INodeDirectory.java    |  12 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  19 +--
 .../hadoop/hdfs/server/namenode/INodeMap.java   |   2 +-
 .../hdfs/server/namenode/INodeReference.java    |  10 +-
 .../hdfs/server/namenode/INodeSymlink.java      |   2 +-
 .../snapshot/DirectorySnapshottableFeature.java |  16 ++-
 .../snapshot/DirectoryWithSnapshotFeature.java  |   5 +-
 .../hdfs/server/namenode/snapshot/Snapshot.java |   3 +-
 .../TestGetContentSummaryWithSnapshot.java      | 126 +++++++++++++++++++
 12 files changed, 166 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f427531/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 f9837f3..b905d42 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1367,6 +1367,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-9073. Fix failures in TestLazyPersistLockedMemory
     testReleaseOnEviction(). (J.Andreina via stevel)
 
+    HDFS-9063. Correctly handle snapshot path for getContentSummary. (jing9)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f427531/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index 4a45074..f737cc3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@ -571,7 +571,8 @@ class FSDirStatAndListingOp {
         ContentSummaryComputationContext cscc =
             new ContentSummaryComputationContext(fsd, fsd.getFSNamesystem(),
                 fsd.getContentCountLimit(), fsd.getContentSleepMicroSec());
-        ContentSummary cs = targetNode.computeAndConvertContentSummary(cscc);
+        ContentSummary cs = targetNode.computeAndConvertContentSummary(
+            iip.getPathSnapshotId(), cscc);
         fsd.addYieldCount(cscc.getYieldCount());
         return cs;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f427531/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
index 2018844..64442fd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
@@ -419,16 +419,17 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
 
   /** Compute {@link ContentSummary}. Blocking call */
   public final ContentSummary computeContentSummary(BlockStoragePolicySuite bsps) {
-    return computeAndConvertContentSummary(
+    return computeAndConvertContentSummary(Snapshot.CURRENT_STATE_ID,
         new ContentSummaryComputationContext(bsps));
   }
 
   /**
    * Compute {@link ContentSummary}. 
    */
-  public final ContentSummary computeAndConvertContentSummary(
+  public final ContentSummary computeAndConvertContentSummary(int snapshotId,
       ContentSummaryComputationContext summary) {
-    ContentCounts counts = computeContentSummary(summary).getCounts();
+    ContentCounts counts = computeContentSummary(snapshotId, summary)
+        .getCounts();
     final QuotaCounts q = getQuotaCounts();
     return new ContentSummary.Builder().
         length(counts.getLength()).
@@ -445,11 +446,16 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   /**
    * Count subtree content summary with a {@link ContentCounts}.
    *
+   * @param snapshotId Specify the time range for the calculation. If this
+   *                   parameter equals to {@link Snapshot#CURRENT_STATE_ID},
+   *                   the result covers both the current states and all the
+   *                   snapshots. Otherwise the result only covers all the
+   *                   files/directories contained in the specific snapshot.
    * @param summary the context object holding counts for the subtree.
    * @return The same objects as summary.
    */
   public abstract ContentSummaryComputationContext computeContentSummary(
-      ContentSummaryComputationContext summary);
+      int snapshotId, ContentSummaryComputationContext summary);
 
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f427531/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
index 21fe313..d00c136 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
@@ -626,18 +626,20 @@ public class INodeDirectory extends INodeWithAdditionalFields
   }
 
   @Override
-  public ContentSummaryComputationContext computeContentSummary(
+  public ContentSummaryComputationContext computeContentSummary(int snapshotId,
       ContentSummaryComputationContext summary) {
     final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
-    if (sf != null) {
+    if (sf != null && snapshotId == Snapshot.CURRENT_STATE_ID) {
+      // if the getContentSummary call is against a non-snapshot path, the
+      // computation should include all the deleted files/directories
       sf.computeContentSummary4Snapshot(summary.getBlockStoragePolicySuite(),
           summary.getCounts());
     }
     final DirectoryWithQuotaFeature q = getDirectoryWithQuotaFeature();
-    if (q != null) {
+    if (q != null && snapshotId == Snapshot.CURRENT_STATE_ID) {
       return q.computeContentSummary(this, summary);
     } else {
-      return computeDirectoryContentSummary(summary, Snapshot.CURRENT_STATE_ID);
+      return computeDirectoryContentSummary(summary, snapshotId);
     }
   }
 
@@ -651,7 +653,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
       byte[] childName = child.getLocalNameBytes();
 
       long lastYieldCount = summary.getYieldCount();
-      child.computeContentSummary(summary);
+      child.computeContentSummary(snapshotId, summary);
 
       // Check whether the computation was paused in the subtree.
       // The counts may be off, but traversing the rest of children

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f427531/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 d546905..8565522 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
@@ -610,23 +610,10 @@ public class INodeFile extends INodeWithAdditionalFields
 
   @Override
   public final ContentSummaryComputationContext computeContentSummary(
-      final ContentSummaryComputationContext summary) {
+      int snapshotId, final ContentSummaryComputationContext summary) {
     final ContentCounts counts = summary.getCounts();
-    FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
-    final long fileLen;
-    if (sf == null) {
-      fileLen = computeFileSize();
-      counts.addContent(Content.FILE, 1);
-    } else {
-      final FileDiffList diffs = sf.getDiffs();
-      final int n = diffs.asList().size();
-      counts.addContent(Content.FILE, n);
-      if (n > 0 && sf.isCurrentFileDeleted()) {
-        fileLen =  diffs.getLast().getFileSize();
-      } else {
-        fileLen = computeFileSize();
-      }
-    }
+    counts.addContent(Content.FILE, 1);
+    final long fileLen = computeFileSize(snapshotId);
     counts.addContent(Content.LENGTH, fileLen);
     counts.addContent(Content.DISKSPACE, storagespaceConsumed(null)
         .getStorageSpace());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f427531/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
index 795f474..bc273d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
@@ -109,7 +109,7 @@ public class INodeMap {
 
       @Override
       public ContentSummaryComputationContext computeContentSummary(
-          ContentSummaryComputationContext summary) {
+          int snapshotId, ContentSummaryComputationContext summary) {
         return null;
       }
       

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f427531/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
index 2fea903..8734956 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
@@ -313,9 +313,9 @@ public abstract class INodeReference extends INode {
   }
 
   @Override
-  public ContentSummaryComputationContext computeContentSummary(
+  public ContentSummaryComputationContext computeContentSummary(int snapshotId,
       ContentSummaryComputationContext summary) {
-    return referred.computeContentSummary(summary);
+    return referred.computeContentSummary(snapshotId, summary);
   }
 
   @Override
@@ -502,11 +502,11 @@ public abstract class INodeReference extends INode {
     
     @Override
     public final ContentSummaryComputationContext computeContentSummary(
-        ContentSummaryComputationContext summary) {
+        int snapshotId, ContentSummaryComputationContext summary) {
+      final int s = snapshotId < lastSnapshotId ? snapshotId : lastSnapshotId;
       // only count storagespace for WithName
       final QuotaCounts q = computeQuotaUsage(
-          summary.getBlockStoragePolicySuite(), getStoragePolicyID(), false,
-          lastSnapshotId);
+          summary.getBlockStoragePolicySuite(), getStoragePolicyID(), false, s);
       summary.getCounts().addContent(Content.DISKSPACE, q.getStorageSpace());
       summary.getCounts().addTypeSpaces(q.getTypeSpaces());
       return summary;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f427531/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
index 8ad3aa8..c76bea0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
@@ -94,7 +94,7 @@ public class INodeSymlink extends INodeWithAdditionalFields {
   }
 
   @Override
-  public ContentSummaryComputationContext computeContentSummary(
+  public ContentSummaryComputationContext computeContentSummary(int snapshotId,
       final ContentSummaryComputationContext summary) {
     summary.getCounts().addContent(Content.SYMLINK, 1);
     return summary;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f427531/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
index 4d6ca27..39db979 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.Content;
-import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
+import org.apache.hadoop.hdfs.server.namenode.ContentCounts;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.SnapshotAndINode;
@@ -219,14 +219,12 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
     }
   }
 
-  public ContentSummaryComputationContext computeContentSummary(
-      final BlockStoragePolicySuite bsps,
-      final INodeDirectory snapshotRoot,
-      final ContentSummaryComputationContext summary) {
-    snapshotRoot.computeContentSummary(summary);
-    summary.getCounts().addContent(Content.SNAPSHOT, snapshotsByNames.size());
-    summary.getCounts().addContent(Content.SNAPSHOTTABLE_DIRECTORY, 1);
-    return summary;
+  @Override
+  public void computeContentSummary4Snapshot(final BlockStoragePolicySuite bsps,
+      final ContentCounts counts) {
+    counts.addContent(Content.SNAPSHOT, snapshotsByNames.size());
+    counts.addContent(Content.SNAPSHOTTABLE_DIRECTORY, 1);
+    super.computeContentSummary4Snapshot(bsps, counts);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f427531/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
index e0b4218..5c5b259 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.AclStorage;
-import org.apache.hadoop.hdfs.server.namenode.Content;
 import org.apache.hadoop.hdfs.server.namenode.ContentCounts;
 import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
@@ -628,13 +627,11 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
         new ContentSummaryComputationContext(bsps);
     for(DirectoryDiff d : diffs) {
       for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
-        deleted.computeContentSummary(summary);
+        deleted.computeContentSummary(Snapshot.CURRENT_STATE_ID, summary);
       }
     }
     // Add the counts from deleted trees.
     counts.addContents(summary.getCounts());
-    // Add the deleted directory count.
-    counts.addContent(Content.DIRECTORY, diffs.asList().size());
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f427531/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java
index 59e618a..5313142 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java
@@ -176,8 +176,7 @@ public class Snapshot implements Comparable<byte[]> {
 
     @Override
     public ContentSummaryComputationContext computeContentSummary(
-        ContentSummaryComputationContext summary) {
-      int snapshotId = getParent().getSnapshot(getLocalNameBytes()).getId();
+        int snapshotId, ContentSummaryComputationContext summary) {
       return computeDirectoryContentSummary(summary, snapshotId);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f427531/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestGetContentSummaryWithSnapshot.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestGetContentSummaryWithSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestGetContentSummaryWithSnapshot.java
new file mode 100644
index 0000000..21f2db5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestGetContentSummaryWithSnapshot.java
@@ -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.
+ */
+package org.apache.hadoop.hdfs.server.namenode.snapshot;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+/**
+ * Verify content summary is computed correctly when
+ * 1. There are snapshots taken under the directory
+ * 2. The given path is a snapshot path
+ */
+public class TestGetContentSummaryWithSnapshot {
+  protected static final short REPLICATION = 3;
+  protected static final long BLOCKSIZE = 1024;
+
+  protected Configuration conf;
+  protected MiniDFSCluster cluster;
+  protected FSNamesystem fsn;
+  protected FSDirectory fsdir;
+  protected DistributedFileSystem dfs;
+
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION).build();
+    cluster.waitActive();
+
+    fsn = cluster.getNamesystem();
+    fsdir = fsn.getFSDirectory();
+    dfs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Calculate against a snapshot path.
+   * 1. create dirs /foo/bar
+   * 2. take snapshot s1 on /foo
+   * 3. create a 10 byte file /foo/bar/baz
+   * Make sure for "/foo/bar" and "/foo/.snapshot/s1/bar" have correct results:
+   * the 1 byte file is not included in snapshot s1.
+   */
+  @Test
+  public void testGetContentSummary() throws IOException {
+    final Path foo = new Path("/foo");
+    final Path bar = new Path(foo, "bar");
+    final Path baz = new Path(bar, "baz");
+
+    dfs.mkdirs(bar);
+    dfs.allowSnapshot(foo);
+    dfs.createSnapshot(foo, "s1");
+
+    DFSTestUtil.createFile(dfs, baz, 10, REPLICATION, 0L);
+
+    ContentSummary summary = cluster.getNameNodeRpc().getContentSummary(
+        bar.toString());
+    Assert.assertEquals(1, summary.getDirectoryCount());
+    Assert.assertEquals(1, summary.getFileCount());
+    Assert.assertEquals(10, summary.getLength());
+
+    final Path barS1 = SnapshotTestHelper.getSnapshotPath(foo, "s1", "bar");
+    summary = cluster.getNameNodeRpc().getContentSummary(barS1.toString());
+    Assert.assertEquals(1, summary.getDirectoryCount());
+    Assert.assertEquals(0, summary.getFileCount());
+    Assert.assertEquals(0, summary.getLength());
+
+    // also check /foo and /foo/.snapshot/s1
+    summary = cluster.getNameNodeRpc().getContentSummary(foo.toString());
+    Assert.assertEquals(2, summary.getDirectoryCount());
+    Assert.assertEquals(1, summary.getFileCount());
+    Assert.assertEquals(10, summary.getLength());
+
+    final Path fooS1 = SnapshotTestHelper.getSnapshotRoot(foo, "s1");
+    summary = cluster.getNameNodeRpc().getContentSummary(fooS1.toString());
+    Assert.assertEquals(2, summary.getDirectoryCount());
+    Assert.assertEquals(0, summary.getFileCount());
+    Assert.assertEquals(0, summary.getLength());
+
+    final Path bazS1 = SnapshotTestHelper.getSnapshotPath(foo, "s1", "bar/baz");
+    try {
+      cluster.getNameNodeRpc().getContentSummary(bazS1.toString());
+      Assert.fail("should get FileNotFoundException");
+    } catch (FileNotFoundException ignored) {}
+  }
+}


[29/50] [abbrv] hadoop git commit: HADOOP-12404. Disable caching for JarURLConnection to avoid sharing JarFile with other users when loading resource from URL in Configuration class. Contributed by Zhihai Xu

Posted by zh...@apache.org.
HADOOP-12404. Disable caching for JarURLConnection to avoid sharing JarFile with other users when loading resource from URL in Configuration class. 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/88d89267
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/88d89267
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/88d89267

Branch: refs/heads/HDFS-7285
Commit: 88d89267ff6b66e144bfcceb09532191975f2a4a
Parents: 3732d52
Author: Zhihai Xu <zx...@apache.org>
Authored: Fri Sep 18 10:40:04 2015 -0700
Committer: Zhihai Xu <zx...@apache.org>
Committed: Fri Sep 18 10:40:04 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt          |  4 ++++
 .../main/java/org/apache/hadoop/conf/Configuration.java  | 11 ++++++++++-
 2 files changed, 14 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/88d89267/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 54d7b6b..2bf5c9d 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -782,6 +782,10 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12413. AccessControlList should avoid calling getGroupNames in
     isUserInList with empty groups. (Zhihai Xu via cnauroth)
 
+    HADOOP-12404. Disable caching for JarURLConnection to avoid sharing
+    JarFile with other users when loading resource from URL in Configuration
+    class. (zxu)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/88d89267/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
index 0b45429..8801c6c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
@@ -34,7 +34,9 @@ import java.io.Reader;
 import java.io.Writer;
 import java.lang.ref.WeakReference;
 import java.net.InetSocketAddress;
+import java.net.JarURLConnection;
 import java.net.URL;
+import java.net.URLConnection;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -2531,7 +2533,14 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     if (url == null) {
       return null;
     }
-    return parse(builder, url.openStream(), url.toString());
+
+    URLConnection connection = url.openConnection();
+    if (connection instanceof JarURLConnection) {
+      // Disable caching for JarURLConnection to avoid sharing JarFile
+      // with other users.
+      connection.setUseCaches(false);
+    }
+    return parse(builder, connection.getInputStream(), url.toString());
   }
 
   private Document parse(DocumentBuilder builder, InputStream is,


[15/50] [abbrv] hadoop git commit: MAPREDUCE-5002. AM could potentially allocate a reduce container to a map attempt. Contributed by Chang Li

Posted by zh...@apache.org.
MAPREDUCE-5002. AM could potentially allocate a reduce container to a map attempt. Contributed by Chang Li


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

Branch: refs/heads/HDFS-7285
Commit: 3f82f582e51c514cc0f052c828c2f58c6e8927ad
Parents: 58d1a02
Author: Jason Lowe <jl...@apache.org>
Authored: Thu Sep 17 18:17:29 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Thu Sep 17 18:17:29 2015 +0000

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |   3 +
 .../v2/app/rm/RMContainerAllocator.java         |  10 +-
 .../v2/app/rm/TestRMContainerAllocator.java     | 110 +++++++++++++++++++
 3 files changed, 119 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f82f582/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index cde6d92..6cf7abb 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -570,6 +570,9 @@ Release 2.8.0 - UNRELEASED
     position/key information for uncompressed input sometimes. (Zhihai Xu via
     jlowe)
 
+    MAPREDUCE-5002. AM could potentially allocate a reduce container to a map
+    attempt (Chang Li via jlowe)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f82f582/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
index ac4c586..78b0dc4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
@@ -1004,6 +1004,7 @@ public class RMContainerAllocator extends RMContainerRequestor
       Iterator<Container> it = allocatedContainers.iterator();
       LOG.info("Got allocated containers " + allocatedContainers.size());
       containersAllocated += allocatedContainers.size();
+      int reducePending = reduces.size();
       while (it.hasNext()) {
         Container allocated = it.next();
         if (LOG.isDebugEnabled()) {
@@ -1034,13 +1035,14 @@ public class RMContainerAllocator extends RMContainerRequestor
         else if (PRIORITY_REDUCE.equals(priority)) {
           if (ResourceCalculatorUtils.computeAvailableContainers(allocatedResource,
               reduceResourceRequest, getSchedulerResourceTypes()) <= 0
-              || reduces.isEmpty()) {
-            LOG.info("Cannot assign container " + allocated 
+              || (reducePending <= 0)) {
+            LOG.info("Cannot assign container " + allocated
                 + " for a reduce as either "
                 + " container memory less than required " + reduceResourceRequest
-                + " or no pending reduce tasks - reduces.isEmpty=" 
-                + reduces.isEmpty()); 
+                + " or no pending reduce tasks.");
             isAssignable = false;
+          } else {
+            reducePending--;
           }
         } else {
           LOG.warn("Container allocated at unwanted priority: " + priority + 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f82f582/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
index e148c32..c98ccd3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
@@ -127,6 +127,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
@@ -562,6 +563,52 @@ public class TestRMContainerAllocator {
         assignedRequests.preemptionWaitingReduces.size());
   }
 
+  @Test(timeout = 30000)
+  public void testExcessReduceContainerAssign() throws Exception {
+  final Configuration conf = new Configuration();
+    conf.setFloat(MRJobConfig.COMPLETED_MAPS_FOR_REDUCE_SLOWSTART, 0.0f);
+    final MyResourceManager2 rm = new MyResourceManager2(conf);
+    rm.start();
+    final DrainDispatcher dispatcher = (DrainDispatcher)rm.getRMContext()
+            .getDispatcher();
+    final RMApp app = rm.submitApp(2048);
+    dispatcher.await();
+    final String host = "host1";
+    final MockNM nm = rm.registerNode(String.format("%s:1234", host), 4096);
+    nm.nodeHeartbeat(true);
+    dispatcher.await();
+    final ApplicationAttemptId appAttemptId = app.getCurrentAppAttempt()
+          .getAppAttemptId();
+    rm.sendAMLaunched(appAttemptId);
+    dispatcher.await();
+    final JobId jobId = MRBuilderUtils
+                 .newJobId(appAttemptId.getApplicationId(), 0);
+    final Job mockJob = mock(Job.class);
+    when(mockJob.getReport()).thenReturn(
+        MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, 0,
+            0, 0, 0, 0, 0, 0, "jobfile", null, false, ""));
+    final MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
+        appAttemptId, mockJob);
+
+    // request to allocate two reduce priority containers
+    final String[] locations = new String[] { host };
+    allocator.sendRequest(createReq(jobId, 0, 1024, locations, false, true));
+    allocator.scheduleAllReduces();
+    allocator.makeRemoteRequest();
+    nm.nodeHeartbeat(true);
+    dispatcher.await();
+    allocator.sendRequest(createReq(jobId, 1, 1024, locations, false, false));
+
+    int assignedContainer;
+    for (assignedContainer = 0; assignedContainer < 1;) {
+      assignedContainer += allocator.schedule().size();
+      nm.nodeHeartbeat(true);
+      dispatcher.await();
+    }
+    // only 1 allocated container should be assigned
+    Assert.assertEquals(assignedContainer, 1);
+  }
+
   @Test
   public void testMapReduceAllocationWithNodeLabelExpression() throws Exception {
 
@@ -770,6 +817,17 @@ public class TestRMContainerAllocator {
     }
   }
 
+  private static class MyResourceManager2 extends MyResourceManager {
+    public MyResourceManager2(Configuration conf) {
+      super(conf);
+    }
+
+    @Override
+    protected ResourceScheduler createScheduler() {
+      return new ExcessReduceContainerAllocateScheduler(this.getRMContext());
+    }
+  }
+
   @Test
   public void testReportedAppProgress() throws Exception {
 
@@ -1595,6 +1653,58 @@ public class TestRMContainerAllocator {
     }
   }
 
+  private static class ExcessReduceContainerAllocateScheduler extends FifoScheduler {
+
+    public ExcessReduceContainerAllocateScheduler(RMContext rmContext) {
+      super();
+      try {
+        Configuration conf = new Configuration();
+        reinitialize(conf, rmContext);
+      } catch (IOException ie) {
+        LOG.info("add application failed with ", ie);
+        assert (false);
+      }
+    }
+
+    @Override
+    public synchronized Allocation allocate(
+        ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
+        List<ContainerId> release,
+        List<String> blacklistAdditions, List<String> blacklistRemovals) {
+      List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
+      for (ResourceRequest req : ask) {
+        ResourceRequest reqCopy = ResourceRequest.newInstance(req
+            .getPriority(), req.getResourceName(), req.getCapability(), req
+            .getNumContainers(), req.getRelaxLocality());
+        askCopy.add(reqCopy);
+      }
+      SecurityUtil.setTokenServiceUseIp(false);
+      Allocation normalAlloc = super.allocate(
+          applicationAttemptId, askCopy, release,
+          blacklistAdditions, blacklistRemovals);
+      List<Container> containers = normalAlloc.getContainers();
+      if(containers.size() > 0) {
+        // allocate excess container
+        FiCaSchedulerApp application = super.getApplicationAttempt(applicationAttemptId);
+        ContainerId containerId = BuilderUtils.newContainerId(application
+            .getApplicationAttemptId(), application.getNewContainerId());
+        Container excessC = mock(Container.class);
+        when(excessC.getId()).thenReturn(containerId);
+        when(excessC.getPriority()).thenReturn(RMContainerAllocator.PRIORITY_REDUCE);
+        Resource mockR = mock(Resource.class);
+        when(mockR.getMemory()).thenReturn(2048);
+        when(excessC.getResource()).thenReturn(mockR);
+        NodeId nId = mock(NodeId.class);
+        when(nId.getHost()).thenReturn("local");
+        when(excessC.getNodeId()).thenReturn(nId);
+        containers.add(excessC);
+      }
+      Allocation excessAlloc = mock(Allocation.class);
+      when(excessAlloc.getContainers()).thenReturn(containers);
+      return excessAlloc;
+    }
+  }
+
   private ContainerRequestEvent createReq(JobId jobId, int taskAttemptId,
       int memory, String[] hosts) {
     return createReq(jobId, taskAttemptId, memory, hosts, false, false);


[18/50] [abbrv] hadoop git commit: HADOOP-12386. RetryPolicies.RETRY_FOREVER should be able to specify a retry interval. (Sunil G via wangda)

Posted by zh...@apache.org.
HADOOP-12386. RetryPolicies.RETRY_FOREVER should be able to specify a retry interval. (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/6b97fa66
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6b97fa66
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6b97fa66

Branch: refs/heads/HDFS-7285
Commit: 6b97fa6652da29a203f1537508b43333633eb760
Parents: ee4ee6a
Author: Wangda Tan <wa...@apache.org>
Authored: Thu Sep 17 17:46:28 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Thu Sep 17 17:46:28 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt        |  3 +++
 .../java/org/apache/hadoop/io/retry/RetryPolicies.java | 13 ++++++++++++-
 .../org/apache/hadoop/io/retry/TestRetryProxy.java     | 13 ++++++++++++-
 3 files changed, 27 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b97fa66/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 ecd8833..54d7b6b 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1072,6 +1072,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12087. [JDK8] Fix javadoc errors caused by incorrect or illegal
     tags. (Akira AJISAKA via stevel).
 
+    HADOOP-12386. RetryPolicies.RETRY_FOREVER should be able to specify a 
+    retry interval. (Sunil G via wangda)
+
   OPTIMIZATIONS
 
     HADOOP-12051. ProtobufRpcEngine.invoke() should use Exception.toString()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b97fa66/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
index d27096f..871a8ba 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
@@ -65,6 +65,17 @@ public class RetryPolicies {
 
   /**
    * <p>
+   * Keep trying forever with a fixed time between attempts.
+   * </p>
+   */
+  public static final RetryPolicy retryForeverWithFixedSleep(long sleepTime,
+      TimeUnit timeUnit) {
+    return new RetryUpToMaximumCountWithFixedSleep(Integer.MAX_VALUE,
+        sleepTime, timeUnit);
+  }
+
+  /**
+   * <p>
    * Keep trying a limited number of times, waiting a fixed time between attempts,
    * and then fail by re-throwing the exception.
    * </p>
@@ -151,7 +162,7 @@ public class RetryPolicies {
     return new FailoverOnNetworkExceptionRetry(fallbackPolicy, maxFailovers,
         maxRetries, delayMillis, maxDelayBase);
   }
-  
+
   static class TryOnceThenFail implements RetryPolicy {
     @Override
     public RetryAction shouldRetry(Exception e, int retries, int failovers,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b97fa66/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestRetryProxy.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestRetryProxy.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestRetryProxy.java
index 79ea1b9..81f3a9b 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestRetryProxy.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestRetryProxy.java
@@ -25,6 +25,7 @@ import static org.apache.hadoop.io.retry.RetryPolicies.retryByRemoteException;
 import static org.apache.hadoop.io.retry.RetryPolicies.retryUpToMaximumCountWithFixedSleep;
 import static org.apache.hadoop.io.retry.RetryPolicies.retryUpToMaximumCountWithProportionalSleep;
 import static org.apache.hadoop.io.retry.RetryPolicies.retryUpToMaximumTimeWithFixedSleep;
+import static org.apache.hadoop.io.retry.RetryPolicies.retryForeverWithFixedSleep;
 import static org.apache.hadoop.io.retry.RetryPolicies.exponentialBackoffRetry;
 import static org.junit.Assert.*;
 
@@ -110,7 +111,17 @@ public class TestRetryProxy {
     unreliable.failsOnceThenSucceeds();
     unreliable.failsTenTimesThenSucceeds();
   }
-  
+
+  @Test
+  public void testRetryForeverWithFixedSleep() throws UnreliableException {
+    UnreliableInterface unreliable = (UnreliableInterface) RetryProxy.create(
+        UnreliableInterface.class, unreliableImpl,
+        retryForeverWithFixedSleep(1, TimeUnit.MILLISECONDS));
+    unreliable.alwaysSucceeds();
+    unreliable.failsOnceThenSucceeds();
+    unreliable.failsTenTimesThenSucceeds();
+  }
+
   @Test
   public void testRetryUpToMaximumCountWithFixedSleep() throws UnreliableException {
     UnreliableInterface unreliable = (UnreliableInterface)


[36/50] [abbrv] hadoop git commit: HDFS-9004. Add upgrade domain to DatanodeInfo. Contributed by Ming Ma (via Lei (Eddy) Xu).

Posted by zh...@apache.org.
HDFS-9004. Add upgrade domain to DatanodeInfo. Contributed by Ming Ma (via Lei (Eddy) Xu).

Change-Id: I887c66578eebd61acc34b94f18da6e6851c609f4


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

Branch: refs/heads/HDFS-7285
Commit: 3a9c7076e81c1cc47c0ecf30c60abd9a65d8a501
Parents: c39ddc3
Author: Lei Xu <le...@apache.org>
Authored: Sat Sep 19 18:08:09 2015 -0700
Committer: Lei Xu <le...@apache.org>
Committed: Sat Sep 19 18:08:09 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/DatanodeInfo.java      | 41 ++++++++++++++++++--
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |  3 ++
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  |  3 +-
 .../src/main/proto/hdfs.proto                   |  1 +
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  3 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  3 ++
 .../org/apache/hadoop/hdfs/web/JsonUtil.java    |  3 ++
 .../server/namenode/TestNameNodeMXBean.java     | 20 ++++++++++
 8 files changed, 72 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a9c7076/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
index 2ef40d2..c895a1a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
@@ -53,7 +53,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
   private String location = NetworkTopology.DEFAULT_RACK;
   private String softwareVersion;
   private List<String> dependentHostNames = new LinkedList<String>();
-
+  private String upgradeDomain;
 
   // Datanode administrative states
   public enum AdminStates {
@@ -95,6 +95,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.xceiverCount = from.getXceiverCount();
     this.location = from.getNetworkLocation();
     this.adminState = from.getAdminState();
+    this.upgradeDomain = from.getUpgradeDomain();
   }
 
   public DatanodeInfo(DatanodeID nodeID) {
@@ -120,12 +121,13 @@ public class DatanodeInfo extends DatanodeID implements Node {
       final long capacity, final long dfsUsed, final long remaining,
       final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
       final long lastUpdate, final long lastUpdateMonotonic,
-      final int xceiverCount, final AdminStates adminState) {
+      final int xceiverCount, final AdminStates adminState,
+      final String upgradeDomain) {
     this(nodeID.getIpAddr(), nodeID.getHostName(), nodeID.getDatanodeUuid(),
         nodeID.getXferPort(), nodeID.getInfoPort(), nodeID.getInfoSecurePort(),
         nodeID.getIpcPort(), capacity, dfsUsed, remaining, blockPoolUsed,
         cacheCapacity, cacheUsed, lastUpdate, lastUpdateMonotonic,
-        xceiverCount, location, adminState);
+        xceiverCount, location, adminState, upgradeDomain);
   }
 
   /** Constructor */
@@ -137,6 +139,22 @@ public class DatanodeInfo extends DatanodeID implements Node {
       final long lastUpdate, final long lastUpdateMonotonic,
       final int xceiverCount, final String networkLocation,
       final AdminStates adminState) {
+    this(ipAddr, hostName, datanodeUuid, xferPort, infoPort, infoSecurePort,
+        ipcPort, capacity, dfsUsed, remaining, blockPoolUsed, cacheCapacity,
+        cacheUsed, lastUpdate, lastUpdateMonotonic, xceiverCount,
+        networkLocation, adminState, null);
+  }
+
+  /** Constructor */
+  public DatanodeInfo(final String ipAddr, final String hostName,
+      final String datanodeUuid, final int xferPort, final int infoPort,
+      final int infoSecurePort, final int ipcPort,
+      final long capacity, final long dfsUsed, final long remaining,
+      final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
+      final long lastUpdate, final long lastUpdateMonotonic,
+      final int xceiverCount, final String networkLocation,
+      final AdminStates adminState,
+      final String upgradeDomain) {
     super(ipAddr, hostName, datanodeUuid, xferPort, infoPort,
             infoSecurePort, ipcPort);
     this.capacity = capacity;
@@ -150,6 +168,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.xceiverCount = xceiverCount;
     this.location = networkLocation;
     this.adminState = adminState;
+    this.upgradeDomain = upgradeDomain;
   }
 
   /** Network location name */
@@ -300,6 +319,16 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.location = NodeBase.normalize(location);
   }
 
+  /** Sets the upgrade domain */
+  public void setUpgradeDomain(String upgradeDomain) {
+    this.upgradeDomain = upgradeDomain;
+  }
+
+  /** upgrade domain */
+  public String getUpgradeDomain() {
+    return upgradeDomain;
+  }
+
   /** Add a hostname to a list of network dependencies */
   public void addDependentHostName(String hostname) {
     dependentHostNames.add(hostname);
@@ -341,6 +370,9 @@ public class DatanodeInfo extends DatanodeID implements Node {
     if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
       buffer.append("Rack: "+location+"\n");
     }
+    if (upgradeDomain != null) {
+      buffer.append("Upgrade domain: "+ upgradeDomain +"\n");
+    }
     buffer.append("Decommission Status : ");
     if (isDecommissioned()) {
       buffer.append("Decommissioned\n");
@@ -380,6 +412,9 @@ public class DatanodeInfo extends DatanodeID implements Node {
     if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
       buffer.append(" "+location);
     }
+    if (upgradeDomain != null) {
+      buffer.append(" " + upgradeDomain);
+    }
     if (isDecommissioned()) {
       buffer.append(" DD");
     } else if (isDecommissionInProgress()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a9c7076/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index 1e561cc..98de2e9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -140,6 +140,9 @@ public class PBHelperClient {
     if (info.getNetworkLocation() != null) {
       builder.setLocation(info.getNetworkLocation());
     }
+    if (info.getUpgradeDomain() != null) {
+      builder.setUpgradeDomain(info.getUpgradeDomain());
+    }
     builder
       .setId(convert((DatanodeID) info))
       .setCapacity(info.getCapacity())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a9c7076/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
index 713836c..3f85814 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
@@ -241,7 +241,8 @@ class JsonUtilClient {
         getLong(m, "lastUpdateMonotonic", 0l),
         getInt(m, "xceiverCount", 0),
         getString(m, "networkLocation", ""),
-        DatanodeInfo.AdminStates.valueOf(getString(m, "adminState", "NORMAL")));
+        DatanodeInfo.AdminStates.valueOf(getString(m, "adminState", "NORMAL")),
+        getString(m, "upgradeDomain", ""));
   }
 
   /** Convert an Object[] to a DatanodeInfo[]. */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a9c7076/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
index 86fb462..ee77dc0 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
@@ -98,6 +98,7 @@ message DatanodeInfoProto {
   optional uint64 cacheCapacity = 11 [default = 0];
   optional uint64 cacheUsed = 12 [default = 0];
   optional uint64 lastUpdateMonotonic = 13 [default = 0];
+  optional string upgradeDomain = 14;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a9c7076/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 5b60307..cf55445 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
@@ -594,7 +594,8 @@ public class PBHelper {
         di.getCapacity(),  di.getDfsUsed(),  di.getRemaining(),
         di.getBlockPoolUsed(), di.getCacheCapacity(), di.getCacheUsed(),
         di.getLastUpdate(), di.getLastUpdateMonotonic(),
-        di.getXceiverCount(), PBHelper.convert(di.getAdminState()));
+        di.getXceiverCount(), PBHelper.convert(di.getAdminState()),
+        di.hasUpgradeDomain() ? di.getUpgradeDomain() : null);
   }
   
   static public DatanodeInfo[] convert(DatanodeInfoProto di[]) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a9c7076/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 328c29d..75b6be9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -5922,6 +5922,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             .put("estimatedCapacityLostTotal",
                 volumeFailureSummary.getEstimatedCapacityLostTotal());
       }
+      if (node.getUpgradeDomain() != null) {
+        innerinfo.put("upgradeDomain", node.getUpgradeDomain());
+      }
       info.put(node.getHostName() + ":" + node.getXferPort(), innerinfo.build());
     }
     return JSON.toString(info);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a9c7076/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
index 0de89cc..4b0e63e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
@@ -169,6 +169,9 @@ public class JsonUtil {
     m.put("xceiverCount", datanodeinfo.getXceiverCount());
     m.put("networkLocation", datanodeinfo.getNetworkLocation());
     m.put("adminState", datanodeinfo.getAdminState().name());
+    if (datanodeinfo.getUpgradeDomain() != null) {
+      m.put("upgradeDomain", datanodeinfo.getUpgradeDomain());
+    }
     return m;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a9c7076/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
index f8aa317..f10d57e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
@@ -26,6 +26,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.top.TopConf;
 import org.apache.hadoop.io.nativeio.NativeIO;
@@ -76,6 +78,15 @@ public class TestNameNodeMXBean {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       cluster.waitActive();
 
+      // Set upgrade domain on the first DN.
+      String upgradeDomain = "abcd";
+      DatanodeManager dm = cluster.getNameNode().getNamesystem().
+          getBlockManager().getDatanodeManager();
+      DatanodeDescriptor dd = dm.getDatanode(
+          cluster.getDataNodes().get(0).getDatanodeId());
+      dd.setUpgradeDomain(upgradeDomain);
+      String dnXferAddrWithUpgradeDomainSet = dd.getXferAddr();
+
       FSNamesystem fsn = cluster.getNameNode().namesystem;
 
       MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
@@ -125,6 +136,15 @@ public class TestNameNodeMXBean {
         assertTrue(((Long)liveNode.get("capacity")) > 0);
         assertTrue(liveNode.containsKey("numBlocks"));
         assertTrue(((Long)liveNode.get("numBlocks")) == 0);
+        // a. By default the upgrade domain isn't defined on any DN.
+        // b. If the upgrade domain is set on a DN, JMX should have the same
+        // value.
+        String xferAddr = (String)liveNode.get("xferaddr");
+        if (!xferAddr.equals(dnXferAddrWithUpgradeDomainSet)) {
+          assertTrue(!liveNode.containsKey("upgradeDomain"));
+        } else {
+          assertTrue(liveNode.get("upgradeDomain").equals(upgradeDomain));
+        }
       }
       assertEquals(fsn.getLiveNodes(), alivenodeinfo);
       // get attribute deadnodeinfo


[50/50] [abbrv] hadoop git commit: HDFS-8920. Erasure Coding: when recovering lost blocks, logs can be too verbose and hurt performance. Contributed by Rui Li

Posted by zh...@apache.org.
HDFS-8920. Erasure Coding: when recovering lost blocks, logs can be too verbose and hurt performance. Contributed by Rui Li


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

Branch: refs/heads/HDFS-7285
Commit: 7bff8ca1c872ea534a96cbbc5f70134574e289ce
Parents: 1080c37
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Sep 23 14:13:26 2015 +0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue Sep 22 13:27:24 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 ++
 .../org/apache/hadoop/hdfs/DFSInputStream.java  | 15 +++++++--
 .../hadoop/hdfs/DFSStripedInputStream.java      | 34 ++++++++++++++++++++
 3 files changed, 49 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7bff8ca1/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 0e21d22..b79ce64 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -444,3 +444,6 @@
 
     HDFS-9091. Erasure Coding: Provide DistributedFilesystem API to 
     getAllErasureCodingPolicies. (Rakesh R via zhz)
+
+    HDFS-8920. Erasure Coding: when recovering lost blocks, logs can be too
+    verbose and hurt performance. (Rui Li via Kai Zheng)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7bff8ca1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index a5911cc..385ba4b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -1057,9 +1057,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       }
     }
     if (chosenNode == null) {
-      DFSClient.LOG.warn("No live nodes contain block " + block.getBlock() +
-          " after checking nodes = " + Arrays.toString(nodes) +
-          ", ignoredNodes = " + ignoredNodes);
+      reportLostBlock(block, ignoredNodes);
       return null;
     }
     final String dnAddr =
@@ -1071,6 +1069,17 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     return new DNAddrPair(chosenNode, targetAddr, storageType);
   }
 
+  /**
+   * Warn the user of a lost block
+   */
+  protected void reportLostBlock(LocatedBlock lostBlock,
+      Collection<DatanodeInfo> ignoredNodes) {
+    DatanodeInfo[] nodes = lostBlock.getLocations();
+    DFSClient.LOG.warn("No live nodes contain block " + lostBlock.getBlock() +
+        " after checking nodes = " + Arrays.toString(nodes) +
+        ", ignoredNodes = " + ignoredNodes);
+  }
+
   private static String getBestNodeDNAddrPairErrorString(
       DatanodeInfo nodes[], AbstractMap<DatanodeInfo,
       DatanodeInfo> deadNodes, Collection<DatanodeInfo> ignoredNodes) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7bff8ca1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
index b7c22c4..131a552 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
@@ -45,8 +45,11 @@ import java.io.EOFException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.EnumSet;
+import java.util.List;
 import java.util.Set;
 import java.util.Collection;
 import java.util.Map;
@@ -154,6 +157,17 @@ public class DFSStripedInputStream extends DFSInputStream {
   private StripeRange curStripeRange;
   private final CompletionService<Void> readingService;
 
+  /**
+   * When warning the user of a lost block in striping mode, we remember the
+   * dead nodes we've logged. All other striping blocks on these nodes can be
+   * considered lost too, and we don't want to log a warning for each of them.
+   * This is to prevent the log from being too verbose. Refer to HDFS-8920.
+   *
+   * To minimize the overhead, we only store the datanodeUuid in this set
+   */
+  private final Set<String> warnedNodes = Collections.newSetFromMap(
+      new ConcurrentHashMap<String, Boolean>());
+
   DFSStripedInputStream(DFSClient dfsClient, String src,
       boolean verifyChecksum, ErasureCodingPolicy ecPolicy,
       LocatedBlocks locatedBlocks) throws IOException {
@@ -527,6 +541,26 @@ public class DFSStripedInputStream extends DFSInputStream {
     }
   }
 
+  @Override
+  protected void reportLostBlock(LocatedBlock lostBlock,
+      Collection<DatanodeInfo> ignoredNodes) {
+    DatanodeInfo[] nodes = lostBlock.getLocations();
+    if (nodes != null && nodes.length > 0) {
+      List<String> dnUUIDs = new ArrayList<>();
+      for (DatanodeInfo node : nodes) {
+        dnUUIDs.add(node.getDatanodeUuid());
+      }
+      if (!warnedNodes.containsAll(dnUUIDs)) {
+        DFSClient.LOG.warn(Arrays.toString(nodes) + " are unavailable and " +
+            "all striping blocks on them are lost. " +
+            "IgnoredNodes = " + ignoredNodes);
+        warnedNodes.addAll(dnUUIDs);
+      }
+    } else {
+      super.reportLostBlock(lostBlock, ignoredNodes);
+    }
+  }
+
   /**
    * The reader for reading a complete {@link AlignedStripe}. Note that an
    * {@link AlignedStripe} may cross multiple stripes with cellSize width.


[37/50] [abbrv] hadoop git commit: YARN-4167. NPE on RMActiveServices#serviceStop when store is null. (Bibin A Chundatt via rohithsharmaks)

Posted by zh...@apache.org.
YARN-4167. NPE on RMActiveServices#serviceStop when store is null. (Bibin A Chundatt via rohithsharmaks)


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

Branch: refs/heads/HDFS-7285
Commit: c9cb6a5960ad335a3ee93a6ee219eae5aad372f9
Parents: 3a9c707
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Mon Sep 21 09:59:30 2015 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Mon Sep 21 09:59:30 2015 +0530

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9cb6a59/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 7487f71..a3dfb85 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -863,6 +863,8 @@ Release 2.8.0 - UNRELEASED
     YARN-4135. Improve the assertion message in MockRM while failing after waiting for the state.
     (Nijel S F via rohithsharmaks)
 
+    YARN-4167. NPE on RMActiveServices#serviceStop when store is null. (Bibin A Chundatt via rohithsharmaks)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9cb6a59/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index d6d9629..d1f339a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -605,7 +605,9 @@ public class ResourceManager extends CompositeService implements Recoverable {
       if (rmContext != null) {
         RMStateStore store = rmContext.getStateStore();
         try {
-          store.close();
+          if (null != store) {
+            store.close();
+          }
         } catch (Exception e) {
           LOG.error("Error closing store.", e);
         }


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

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
index 21e60c8,a655d66..25012e7
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
@@@ -74,10 -74,9 +74,10 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.protocol.HdfsConstants;
  import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
  import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 +import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
  import org.apache.hadoop.hdfs.server.balancer.Balancer.Cli;
- import org.apache.hadoop.hdfs.server.balancer.Balancer.Parameters;
  import org.apache.hadoop.hdfs.server.balancer.Balancer.Result;
+ import org.apache.hadoop.hdfs.server.balancer.BalancerParameters;
  import org.apache.hadoop.hdfs.server.datanode.DataNode;
  import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
  import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.LazyPersistTestCase;
@@@ -1667,74 -1680,7 +1698,74 @@@ public class TestBalancer 
        cluster.shutdown();
      }
    }
 -  
 +  public void integrationTestWithStripedFile(Configuration conf) throws Exception {
 +    initConfWithStripe(conf);
 +    doTestBalancerWithStripedFile(conf);
 +  }
 +
 +  @Test(timeout = 100000)
 +  public void testBalancerWithStripedFile() throws Exception {
 +    Configuration conf = new Configuration();
 +    initConfWithStripe(conf);
 +    doTestBalancerWithStripedFile(conf);
 +  }
 +
 +  private void doTestBalancerWithStripedFile(Configuration conf) throws Exception {
 +    int numOfDatanodes = dataBlocks + parityBlocks + 2;
 +    int numOfRacks = dataBlocks;
 +    long capacity = 20 * DEFAULT_STRIPE_BLOCK_SIZE;
 +    long[] capacities = new long[numOfDatanodes];
 +    for (int i = 0; i < capacities.length; i++) {
 +      capacities[i] = capacity;
 +    }
 +    String[] racks = new String[numOfDatanodes];
 +    for (int i = 0; i < numOfDatanodes; i++) {
 +      racks[i] = "/rack" + (i % numOfRacks);
 +    }
 +    cluster = new MiniDFSCluster.Builder(conf)
 +        .numDataNodes(numOfDatanodes)
 +        .racks(racks)
 +        .simulatedCapacities(capacities)
 +        .build();
 +
 +    try {
 +      cluster.waitActive();
 +      client = NameNodeProxies.createProxy(conf, cluster.getFileSystem(0).getUri(),
 +          ClientProtocol.class).getProxy();
 +      client.setErasureCodingPolicy("/", null);
 +
 +      long totalCapacity = sum(capacities);
 +
 +      // fill up the cluster with 30% data. It'll be 45% full plus parity.
 +      long fileLen = totalCapacity * 3 / 10;
 +      long totalUsedSpace = fileLen * (dataBlocks + parityBlocks) / dataBlocks;
 +      FileSystem fs = cluster.getFileSystem(0);
 +      DFSTestUtil.createFile(fs, filePath, fileLen, (short) 3, r.nextLong());
 +
 +      // verify locations of striped blocks
 +      LocatedBlocks locatedBlocks = client.getBlockLocations(fileName, 0, fileLen);
 +      DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize);
 +
 +      // add one datanode
 +      String newRack = "/rack" + (++numOfRacks);
 +      cluster.startDataNodes(conf, 1, true, null,
 +          new String[]{newRack}, null, new long[]{capacity});
 +      totalCapacity += capacity;
 +      cluster.triggerHeartbeats();
 +
 +      // run balancer and validate results
-       Balancer.Parameters p = Balancer.Parameters.DEFAULT;
++      BalancerParameters p = BalancerParameters.DEFAULT;
 +      Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
 +      runBalancer(conf, totalUsedSpace, totalCapacity, p, 0);
 +
 +      // verify locations of striped blocks
 +      locatedBlocks = client.getBlockLocations(fileName, 0, fileLen);
 +      DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize);
 +    } finally {
 +      cluster.shutdown();
 +    }
 +  }
 +
    /**
     * @param args
     */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
index 2a593d5,0000000..c827582
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
@@@ -1,284 -1,0 +1,284 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hdfs.server.namenode;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertTrue;
 +import static org.junit.Assert.assertFalse;
 +
 +import java.io.IOException;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.fs.permission.FsPermission;
 +import org.apache.hadoop.fs.permission.PermissionStatus;
 +import org.apache.hadoop.hdfs.DFSConfigKeys;
 +import org.apache.hadoop.hdfs.DFSTestUtil;
 +import org.apache.hadoop.hdfs.DistributedFileSystem;
 +import org.apache.hadoop.hdfs.MiniDFSCluster;
 +import org.apache.hadoop.hdfs.protocol.Block;
 +import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 +import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 +
 +import org.junit.Test;
 +
 +/**
 + * This class tests INodeFile with striped feature.
 + */
 +public class TestStripedINodeFile {
 +  public static final Log LOG = LogFactory.getLog(TestINodeFile.class);
 +
 +  private static final PermissionStatus perm = new PermissionStatus(
 +      "userName", null, FsPermission.getDefault());
 +
 +  private final BlockStoragePolicySuite defaultSuite =
 +      BlockStoragePolicySuite.createDefaultSuite();
 +  private final BlockStoragePolicy defaultPolicy =
 +      defaultSuite.getDefaultPolicy();
 +
 +  private static final ErasureCodingPolicy testECPolicy
 +      = ErasureCodingPolicyManager.getSystemDefaultPolicy();
 +
 +  private static INodeFile createStripedINodeFile() {
 +    return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
-         null, (short)0, 1024L, HdfsServerConstants.COLD_STORAGE_POLICY_ID, true);
++        null, (short)0, 1024L, HdfsConstants.COLD_STORAGE_POLICY_ID, true);
 +  }
 +
 +  @Test
 +  public void testBlockStripedFeature()
 +      throws IOException, InterruptedException{
 +    INodeFile inf = createStripedINodeFile();
 +    assertTrue(inf.isStriped());
 +  }
 +
 +  @Test
 +  public void testBlockStripedTotalBlockCount() {
 +    Block blk = new Block(1);
 +    BlockInfoStriped blockInfoStriped
 +        = new BlockInfoStriped(blk, testECPolicy);
 +    assertEquals(9, blockInfoStriped.getTotalBlockNum());
 +  }
 +
 +  @Test
 +  public void testBlockStripedLength()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
 +    BlockInfoStriped blockInfoStriped
 +        = new BlockInfoStriped(blk, testECPolicy);
 +    inf.addBlock(blockInfoStriped);
 +    assertEquals(1, inf.getBlocks().length);
 +  }
 +
 +  @Test
 +  public void testBlockStripedConsumedSpace()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
 +    BlockInfoStriped blockInfoStriped
 +        = new BlockInfoStriped(blk, testECPolicy);
 +    blockInfoStriped.setNumBytes(1);
 +    inf.addBlock(blockInfoStriped);
 +    //   0. Calculate the total bytes per stripes <Num Bytes per Stripes>
 +    //   1. Calculate the number of stripes in this block group. <Num Stripes>
 +    //   2. Calculate the last remaining length which does not make a stripe. <Last Stripe Length>
 +    //   3. Total consumed space is the total of
 +    //     a. The total of the full cells of data blocks and parity blocks.
 +    //     b. The remaining of data block which does not make a stripe.
 +    //     c. The last parity block cells. These size should be same
 +    //        to the first cell in this stripe.
 +    // So the total consumed space is the sum of
 +    //  a. <Cell Size> * (<Num Stripes> - 1) * <Total Block Num> = 0
 +    //  b. <Num Bytes> % <Num Bytes per Stripes> = 1
 +    //  c. <Last Stripe Length> * <Parity Block Num> = 1 * 3
 +    assertEquals(4, inf.storagespaceConsumedStriped().getStorageSpace());
 +    assertEquals(4, inf.storagespaceConsumed(defaultPolicy).getStorageSpace());
 +  }
 +
 +  @Test
 +  public void testMultipleBlockStripedConsumedSpace()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk1 = new Block(1);
 +    BlockInfoStriped blockInfoStriped1
 +        = new BlockInfoStriped(blk1, testECPolicy);
 +    blockInfoStriped1.setNumBytes(1);
 +    Block blk2 = new Block(2);
 +    BlockInfoStriped blockInfoStriped2
 +        = new BlockInfoStriped(blk2, testECPolicy);
 +    blockInfoStriped2.setNumBytes(1);
 +    inf.addBlock(blockInfoStriped1);
 +    inf.addBlock(blockInfoStriped2);
 +    // This is the double size of one block in above case.
 +    assertEquals(4 * 2, inf.storagespaceConsumedStriped().getStorageSpace());
 +    assertEquals(4 * 2, inf.storagespaceConsumed(defaultPolicy).getStorageSpace());
 +  }
 +
 +  @Test
 +  public void testBlockStripedFileSize()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
 +    BlockInfoStriped blockInfoStriped
 +        = new BlockInfoStriped(blk, testECPolicy);
 +    blockInfoStriped.setNumBytes(100);
 +    inf.addBlock(blockInfoStriped);
 +    // Compute file size should return actual data
 +    // size which is retained by this file.
 +    assertEquals(100, inf.computeFileSize());
 +    assertEquals(100, inf.computeFileSize(false, false));
 +  }
 +
 +  @Test
 +  public void testBlockStripedUCFileSize()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
 +    BlockInfoStriped bInfoUCStriped = new BlockInfoStriped(blk, testECPolicy);
 +    bInfoUCStriped.convertToBlockUnderConstruction(
 +        HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null);
 +    bInfoUCStriped.setNumBytes(100);
 +    inf.addBlock(bInfoUCStriped);
 +    assertEquals(100, inf.computeFileSize());
 +    assertEquals(0, inf.computeFileSize(false, false));
 +  }
 +
 +  @Test
 +  public void testBlockStripedComputeQuotaUsage()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
 +    BlockInfoStriped blockInfoStriped
 +        = new BlockInfoStriped(blk, testECPolicy);
 +    blockInfoStriped.setNumBytes(100);
 +    inf.addBlock(blockInfoStriped);
 +
 +    QuotaCounts counts =
 +        inf.computeQuotaUsageWithStriped(defaultPolicy,
 +            new QuotaCounts.Builder().build());
 +    assertEquals(1, counts.getNameSpace());
 +    // The total consumed space is the sum of
 +    //  a. <Cell Size> * (<Num Stripes> - 1) * <Total Block Num> = 0
 +    //  b. <Num Bytes> % <Num Bytes per Stripes> = 100
 +    //  c. <Last Stripe Length> * <Parity Block Num> = 100 * 3
 +    assertEquals(400, counts.getStorageSpace());
 +  }
 +
 +  @Test
 +  public void testBlockStripedUCComputeQuotaUsage()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
 +    BlockInfoStriped bInfoUCStriped = new BlockInfoStriped(blk, testECPolicy);
 +    bInfoUCStriped.convertToBlockUnderConstruction(
 +        HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null);
 +    bInfoUCStriped.setNumBytes(100);
 +    inf.addBlock(bInfoUCStriped);
 +
 +    QuotaCounts counts
 +        = inf.computeQuotaUsageWithStriped(defaultPolicy,
 +              new QuotaCounts.Builder().build());
 +    assertEquals(1024, inf.getPreferredBlockSize());
 +    assertEquals(1, counts.getNameSpace());
 +    // Consumed space in the case of BlockInfoStripedUC can be calculated
 +    // by using preferred block size. This is 1024 and total block num
 +    // is 9(= 3 + 6). Consumed storage space should be 1024 * 9 = 9216.
 +    assertEquals(9216, counts.getStorageSpace());
 +  }
 +
 +  /**
 +   * Test the behavior of striped and contiguous block deletions.
 +   */
 +  @Test(timeout = 60000)
 +  public void testDeleteOp() throws Exception {
 +    MiniDFSCluster cluster = null;
 +    try {
 +      final int len = 1024;
 +      final Path parentDir = new Path("/parentDir");
 +      final Path ecDir = new Path(parentDir, "ecDir");
 +      final Path ecFile = new Path(ecDir, "ecFile");
 +      final Path contiguousFile = new Path(parentDir, "someFile");
 +      final DistributedFileSystem dfs;
 +      final Configuration conf = new Configuration();
 +      final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS
 +          + HdfsConstants.NUM_PARITY_BLOCKS;
 +      conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY, 2);
 +
 +      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE)
 +          .build();
 +      cluster.waitActive();
 +
 +      FSNamesystem fsn = cluster.getNamesystem();
 +      dfs = cluster.getFileSystem();
 +      dfs.mkdirs(ecDir);
 +
 +      // set erasure coding policy
 +      dfs.setErasureCodingPolicy(ecDir, null);
 +      DFSTestUtil.createFile(dfs, ecFile, len, (short) 1, 0xFEED);
 +      DFSTestUtil.createFile(dfs, contiguousFile, len, (short) 1, 0xFEED);
 +      final FSDirectory fsd = fsn.getFSDirectory();
 +
 +      // Case-1: Verify the behavior of striped blocks
 +      // Get blocks of striped file
 +      INode inodeStriped = fsd.getINode("/parentDir/ecDir/ecFile");
 +      assertTrue("Failed to get INodeFile for /parentDir/ecDir/ecFile",
 +          inodeStriped instanceof INodeFile);
 +      INodeFile inodeStripedFile = (INodeFile) inodeStriped;
 +      BlockInfo[] stripedBlks = inodeStripedFile.getBlocks();
 +      for (BlockInfo blockInfo : stripedBlks) {
 +        assertFalse("Mistakenly marked the block as deleted!",
 +            blockInfo.isDeleted());
 +      }
 +
 +      // delete directory with erasure coding policy
 +      dfs.delete(ecDir, true);
 +      for (BlockInfo blockInfo : stripedBlks) {
 +        assertTrue("Didn't mark the block as deleted!", blockInfo.isDeleted());
 +      }
 +
 +      // Case-2: Verify the behavior of contiguous blocks
 +      // Get blocks of contiguous file
 +      INode inode = fsd.getINode("/parentDir/someFile");
 +      assertTrue("Failed to get INodeFile for /parentDir/someFile",
 +          inode instanceof INodeFile);
 +      INodeFile inodeFile = (INodeFile) inode;
 +      BlockInfo[] contiguousBlks = inodeFile.getBlocks();
 +      for (BlockInfo blockInfo : contiguousBlks) {
 +        assertFalse("Mistakenly marked the block as deleted!",
 +            blockInfo.isDeleted());
 +      }
 +
 +      // delete parent directory
 +      dfs.delete(parentDir, true);
 +      for (BlockInfo blockInfo : contiguousBlks) {
 +        assertTrue("Didn't mark the block as deleted!", blockInfo.isDeleted());
 +      }
 +    } finally {
 +      if (cluster != null) {
 +        cluster.shutdown();
 +      }
 +    }
 +  }
 +}


[28/50] [abbrv] hadoop git commit: Revert "Disable caching for JarURLConnection to avoid sharing JarFile with other users when loading resource from URL in Configuration class. Contributed by Zhihai Xu"

Posted by zh...@apache.org.
Revert "Disable caching for JarURLConnection to avoid sharing JarFile with other users when loading resource from URL in Configuration class. Contributed by Zhihai Xu"

This reverts commit e690a32e57bc14b0cca0e2d39da513841d81b4fc.


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

Branch: refs/heads/HDFS-7285
Commit: 3732d5238a9aa81514e27c7f0feabc454b57134b
Parents: 372ad27
Author: Zhihai Xu <zx...@apache.org>
Authored: Fri Sep 18 10:30:40 2015 -0700
Committer: Zhihai Xu <zx...@apache.org>
Committed: Fri Sep 18 10:30:40 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt          |  4 ----
 .../main/java/org/apache/hadoop/conf/Configuration.java  | 11 +----------
 2 files changed, 1 insertion(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3732d523/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 2bf5c9d..54d7b6b 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -782,10 +782,6 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12413. AccessControlList should avoid calling getGroupNames in
     isUserInList with empty groups. (Zhihai Xu via cnauroth)
 
-    HADOOP-12404. Disable caching for JarURLConnection to avoid sharing
-    JarFile with other users when loading resource from URL in Configuration
-    class. (zxu)
-
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3732d523/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
index 8801c6c..0b45429 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
@@ -34,9 +34,7 @@ import java.io.Reader;
 import java.io.Writer;
 import java.lang.ref.WeakReference;
 import java.net.InetSocketAddress;
-import java.net.JarURLConnection;
 import java.net.URL;
-import java.net.URLConnection;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -2533,14 +2531,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     if (url == null) {
       return null;
     }
-
-    URLConnection connection = url.openConnection();
-    if (connection instanceof JarURLConnection) {
-      // Disable caching for JarURLConnection to avoid sharing JarFile
-      // with other users.
-      connection.setUseCaches(false);
-    }
-    return parse(builder, connection.getInputStream(), url.toString());
+    return parse(builder, url.openStream(), url.toString());
   }
 
   private Document parse(DocumentBuilder builder, InputStream is,


[34/50] [abbrv] hadoop git commit: MAPREDUCE-6460. TestRMContainerAllocator.testAttemptNotFoundCausesRMCommunicatorException fails. Contributed by Zhihai Xu.

Posted by zh...@apache.org.
MAPREDUCE-6460. TestRMContainerAllocator.testAttemptNotFoundCausesRMCommunicatorException fails. 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/e3ace31e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e3ace31e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e3ace31e

Branch: refs/heads/HDFS-7285
Commit: e3ace31e2b7e3a8dc6cf3dc044d5ef2a9aa12e29
Parents: 66b46d0
Author: Zhihai Xu <zx...@apache.org>
Authored: Sat Sep 19 00:10:06 2015 -0700
Committer: Zhihai Xu <zx...@apache.org>
Committed: Sat Sep 19 00:10:06 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                              | 3 +++
 .../hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java      | 3 ++-
 2 files changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3ace31e/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index c67e494..42ea011 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -568,6 +568,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-5002. AM could potentially allocate a reduce container to a map
     attempt (Chang Li via jlowe)
 
+    MAPREDUCE-6460. TestRMContainerAllocator.
+    testAttemptNotFoundCausesRMCommunicatorException fails. (Zhihai Xu)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3ace31e/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
index c98ccd3..1a3829e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
@@ -125,6 +125,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
@@ -2809,7 +2810,7 @@ public class TestRMContainerAllocator {
 
     // Now kill the application
     rm.killApp(app.getApplicationId());
-
+    rm.waitForState(app.getApplicationId(), RMAppState.KILLED);
     allocator.schedule();
   }
 


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

Posted by zh...@apache.org.
Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

Change-Id: I8511c4d64b0959e79129febc179845a3892fedcc


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

Branch: refs/heads/HDFS-7285
Commit: 1080c3730068177ddd10dc313890ac1f5dc58f1a
Parents: a9e6681 dfd807a
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Mon Sep 21 13:51:36 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue Sep 22 13:27:09 2015 -0700

----------------------------------------------------------------------
 .../main/resources/assemblies/hadoop-tools.xml  |    7 +
 .../server/AuthenticationToken.java             |    3 +-
 hadoop-common-project/hadoop-common/CHANGES.txt |  205 +-
 .../src/main/conf/log4j.properties              |   17 +-
 .../org/apache/hadoop/conf/Configuration.java   |   11 +-
 .../fs/CommonConfigurationKeysPublic.java       |    6 +
 .../java/org/apache/hadoop/fs/FileUtil.java     |   14 +-
 .../main/java/org/apache/hadoop/fs/Trash.java   |    3 +-
 .../hadoop/fs/shell/CommandWithDestination.java |   33 +-
 .../apache/hadoop/fs/shell/CopyCommands.java    |   19 +-
 .../java/org/apache/hadoop/fs/shell/Delete.java |   68 +-
 .../apache/hadoop/ha/ActiveStandbyElector.java  |    4 +
 .../apache/hadoop/io/retry/RetryPolicies.java   |   13 +-
 .../main/java/org/apache/hadoop/ipc/Client.java |    9 +-
 .../hadoop/metrics2/impl/MetricsConfig.java     |    3 +
 .../metrics2/impl/MetricsSourceAdapter.java     |   12 +-
 .../hadoop/metrics2/impl/MetricsSystemImpl.java |   20 +-
 .../apache/hadoop/metrics2/package-info.java    |    4 +-
 .../apache/hadoop/metrics2/sink/StatsDSink.java |  218 ++
 .../apache/hadoop/security/SaslPlainServer.java |    2 +-
 .../security/authorize/AccessControlList.java   |    2 +-
 .../java/org/apache/hadoop/util/LineReader.java |   17 +-
 .../org/apache/hadoop/util/VersionInfo.java     |    2 +-
 .../src/main/resources/core-default.xml         |   11 +
 .../src/site/markdown/FileSystemShell.md        |    2 +-
 .../hadoop-common/src/site/markdown/Metrics.md  |    3 +
 .../java/org/apache/hadoop/cli/TestCLI.java     |    2 +-
 .../org/apache/hadoop/cli/util/CLICommand.java  |    5 +-
 .../org/apache/hadoop/cli/util/CLITestCmd.java  |    6 +-
 .../org/apache/hadoop/fs/TestFsShellCopy.java   |   46 +
 .../apache/hadoop/io/retry/TestRetryProxy.java  |   13 +-
 .../java/org/apache/hadoop/ipc/TestIPC.java     |   38 +-
 .../java/org/apache/hadoop/ipc/TestSaslRPC.java |    9 +-
 .../metrics2/impl/TestMetricsSystemImpl.java    |   13 +
 .../hadoop/metrics2/impl/TestStatsDMetrics.java |  122 +
 .../authorize/TestAccessControlList.java        |    9 +
 .../delegation/web/TestWebDelegationToken.java  |    7 +-
 .../src/test/resources/testConf.xml             |   22 +-
 .../dev-support/findbugsExcludeFile.xml         |    1 +
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |  105 +
 .../hdfs/client/HdfsClientConfigKeys.java       |   14 +
 .../apache/hadoop/hdfs/net/BasicInetPeer.java   |  133 +
 .../apache/hadoop/hdfs/net/EncryptedPeer.java   |  142 ++
 .../org/apache/hadoop/hdfs/net/NioInetPeer.java |  136 +
 .../java/org/apache/hadoop/hdfs/net/Peer.java   |    8 +-
 .../hdfs/protocol/ClientDatanodeProtocol.java   |    7 +
 .../hadoop/hdfs/protocol/DatanodeInfo.java      |   47 +-
 .../hadoop/hdfs/protocol/HdfsConstants.java     |    8 +
 .../hdfs/protocol/HdfsLocatedFileStatus.java    |   84 +
 .../protocol/datatransfer/IOStreamPair.java     |   37 +
 .../hdfs/protocol/datatransfer/PipelineAck.java |  243 ++
 .../datatransfer/TrustedChannelResolver.java    |   81 +
 .../sasl/DataEncryptionKeyFactory.java          |   38 +
 .../datatransfer/sasl/DataTransferSaslUtil.java |  519 ++++
 .../sasl/SaslDataTransferClient.java            |  498 ++++
 .../datatransfer/sasl/SaslParticipant.java      |  210 ++
 .../SaslResponseWithNegotiatedCipherOption.java |   33 +
 .../ClientDatanodeProtocolTranslatorPB.java     |   17 +
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  | 2186 ++++++++++++++++-
 .../apache/hadoop/hdfs/util/LongBitFormat.java  |   71 +
 .../hadoop/hdfs/web/ByteRangeInputStream.java   |   11 +
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  |    3 +-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |    4 +-
 .../src/main/proto/ClientDatanodeProtocol.proto |   16 +
 .../src/main/proto/hdfs.proto                   |    1 +
 .../services/org.apache.hadoop.fs.FileSystem    |   17 +
 .../hadoop/hdfs/nfs/mount/RpcProgramMountd.java |    4 +-
 .../hadoop/hdfs/nfs/nfs3/DFSClientCache.java    |    4 +-
 .../apache/hadoop/hdfs/nfs/nfs3/TestWrites.java |    7 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  409 ++-
 .../hdfs/server/namenode/FSEditLogTestUtil.java |    3 +-
 .../main/java/org/apache/hadoop/fs/Hdfs.java    |    7 +-
 .../java/org/apache/hadoop/fs/SWebHdfs.java     |   15 +-
 .../main/java/org/apache/hadoop/fs/WebHdfs.java |   14 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |   10 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   50 +-
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |   10 +-
 .../org/apache/hadoop/hdfs/DataStreamer.java    |    8 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |    4 +-
 .../org/apache/hadoop/hdfs/NameNodeProxies.java |   15 +-
 .../apache/hadoop/hdfs/net/BasicInetPeer.java   |  133 -
 .../hadoop/hdfs/net/DomainPeerServer.java       |    5 +
 .../apache/hadoop/hdfs/net/EncryptedPeer.java   |  142 --
 .../org/apache/hadoop/hdfs/net/NioInetPeer.java |  136 -
 .../org/apache/hadoop/hdfs/net/PeerServer.java  |    9 +-
 .../apache/hadoop/hdfs/net/TcpPeerServer.java   |   70 +-
 .../hdfs/protocol/HdfsLocatedFileStatus.java    |   84 -
 .../protocol/datatransfer/IOStreamPair.java     |   37 -
 .../hdfs/protocol/datatransfer/PipelineAck.java |  274 ---
 .../hdfs/protocol/datatransfer/Receiver.java    |   33 +-
 .../datatransfer/TrustedChannelResolver.java    |   81 -
 .../sasl/DataEncryptionKeyFactory.java          |   38 -
 .../datatransfer/sasl/DataTransferSaslUtil.java |  519 ----
 .../sasl/SaslDataTransferClient.java            |  498 ----
 .../sasl/SaslDataTransferServer.java            |    2 +-
 .../datatransfer/sasl/SaslParticipant.java      |  210 --
 .../SaslResponseWithNegotiatedCipherOption.java |   33 -
 ...tDatanodeProtocolServerSideTranslatorPB.java |   22 +-
 .../protocolPB/ClientNamenodeProtocolPB.java    |    4 +-
 ...tNamenodeProtocolServerSideTranslatorPB.java |  135 +-
 .../ClientNamenodeProtocolTranslatorPB.java     |  128 +-
 .../DatanodeProtocolClientSideTranslatorPB.java |   10 +-
 .../DatanodeProtocolServerSideTranslatorPB.java |   12 +-
 ...rDatanodeProtocolServerSideTranslatorPB.java |    2 +-
 .../protocolPB/JournalProtocolTranslatorPB.java |    2 +-
 .../NamenodeProtocolServerSideTranslatorPB.java |    3 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 2321 +-----------------
 .../QJournalProtocolTranslatorPB.java           |    3 +-
 .../hadoop/hdfs/qjournal/server/Journal.java    |   40 +-
 .../hdfs/qjournal/server/JournalMetrics.java    |    7 +-
 .../hadoop/hdfs/server/balancer/Balancer.java   |  170 +-
 .../server/balancer/BalancerParameters.java     |  168 ++
 .../server/blockmanagement/BlockManager.java    |   97 +-
 .../BlockPlacementPolicyWithNodeGroup.java      |   14 +-
 .../BlockStoragePolicySuite.java                |   13 +-
 .../server/blockmanagement/DatanodeManager.java |    4 +-
 .../blockmanagement/DecommissionManager.java    |    2 +-
 .../hdfs/server/common/HdfsServerConstants.java |    6 -
 .../hdfs/server/common/MetricsLoggerTask.java   |  174 ++
 .../hdfs/server/datanode/BPOfferService.java    |    3 +-
 .../hdfs/server/datanode/BPServiceActor.java    |    7 +-
 .../hdfs/server/datanode/BlockReceiver.java     |    7 +-
 .../hadoop/hdfs/server/datanode/DNConf.java     |   22 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |  130 +-
 .../hdfs/server/datanode/DataNodeMXBean.java    |    9 +-
 .../hdfs/server/datanode/DataXceiver.java       |   11 +-
 .../hdfs/server/datanode/DataXceiverServer.java |    7 +-
 .../hdfs/server/datanode/FinalizedReplica.java  |   15 +-
 .../hdfs/server/datanode/ReplicaInfo.java       |   82 -
 .../server/datanode/ReplicaUnderRecovery.java   |   10 -
 .../datanode/ReplicaWaitingToBeRecovered.java   |   15 +-
 .../erasurecode/ErasureCodingWorker.java        |    3 +-
 .../server/datanode/fsdataset/FsVolumeSpi.java  |    8 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |   16 +-
 .../datanode/fsdataset/impl/FsVolumeImpl.java   |   72 +-
 .../hdfs/server/namenode/BackupImage.java       |   10 +-
 .../hdfs/server/namenode/CacheManager.java      |    4 +-
 .../hadoop/hdfs/server/namenode/DfsServlet.java |    3 +-
 .../server/namenode/EditLogFileInputStream.java |   60 +-
 .../server/namenode/EncryptionZoneManager.java  |    4 +-
 .../server/namenode/FSDirStatAndListingOp.java  |    3 +-
 .../hdfs/server/namenode/FSDirXAttrOp.java      |    9 +-
 .../hdfs/server/namenode/FSDirectory.java       |  138 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |   10 +
 .../hdfs/server/namenode/FSEditLogLoader.java   |   57 +-
 .../hdfs/server/namenode/FSEditLogOp.java       |   20 +-
 .../hadoop/hdfs/server/namenode/FSImage.java    |  125 -
 .../server/namenode/FSImageFormatPBINode.java   |    9 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   88 +-
 .../server/namenode/FSPermissionChecker.java    |   41 +-
 .../server/namenode/FileJournalManager.java     |   47 +-
 .../hadoop/hdfs/server/namenode/INode.java      |   14 +-
 .../hdfs/server/namenode/INodeDirectory.java    |   14 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |   19 +-
 .../hadoop/hdfs/server/namenode/INodeMap.java   |    2 +-
 .../hdfs/server/namenode/INodeReference.java    |   10 +-
 .../hdfs/server/namenode/INodeSymlink.java      |    2 +-
 .../hdfs/server/namenode/ImageServlet.java      |    5 +-
 .../hdfs/server/namenode/LeaseManager.java      |    6 +-
 .../hadoop/hdfs/server/namenode/NameNode.java   |  192 +-
 .../hdfs/server/namenode/NameNodeMXBean.java    |    6 +
 .../hdfs/server/namenode/NamenodeFsck.java      |    6 +-
 .../hadoop/hdfs/server/namenode/Namesystem.java |    3 +
 .../hadoop/hdfs/server/namenode/SafeMode.java   |    3 -
 .../hdfs/server/namenode/SecondaryNameNode.java |    2 +-
 .../server/namenode/ha/BootstrapStandby.java    |    5 +-
 .../hdfs/server/namenode/ha/EditLogTailer.java  |    2 +-
 .../namenode/ha/IPFailoverProxyProvider.java    |    4 +-
 .../snapshot/DirectorySnapshottableFeature.java |   16 +-
 .../snapshot/DirectoryWithSnapshotFeature.java  |    5 +-
 .../snapshot/FSImageFormatPBSnapshot.java       |    6 +-
 .../hdfs/server/namenode/snapshot/Snapshot.java |    3 +-
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |   42 +-
 .../hdfs/tools/DFSZKFailoverController.java     |    3 +-
 .../org/apache/hadoop/hdfs/tools/GetGroups.java |    4 +-
 .../hadoop/hdfs/tools/NNHAServiceTarget.java    |    3 +-
 .../apache/hadoop/hdfs/util/LongBitFormat.java  |   71 -
 .../org/apache/hadoop/hdfs/web/JsonUtil.java    |    3 +
 .../services/org.apache.hadoop.fs.FileSystem    |    2 -
 .../src/main/resources/hdfs-default.xml         |   46 +
 .../src/main/webapps/hdfs/dfshealth.html        |    7 +-
 .../src/main/webapps/hdfs/dfshealth.js          |    9 +-
 .../src/main/webapps/hdfs/explorer.html         |   76 +-
 .../src/main/webapps/hdfs/explorer.js           |   89 +-
 .../src/main/webapps/static/dfs-dust.js         |   12 +-
 .../src/main/webapps/static/hadoop.css          |   22 +
 .../src/site/markdown/ArchivalStorage.md        |    4 +-
 .../src/site/markdown/HDFSCommands.md           |    6 +-
 .../hadoop-hdfs/src/site/markdown/HdfsDesign.md |    6 +-
 .../src/site/markdown/HdfsRollingUpgrade.md     |  293 +++
 .../src/site/markdown/HdfsSnapshots.md          |  301 +++
 .../site/resources/images/LazyPersistWrites.png |  Bin 107161 -> 90224 bytes
 .../src/site/xdoc/HdfsRollingUpgrade.xml        |  329 ---
 .../hadoop-hdfs/src/site/xdoc/HdfsSnapshots.xml |  303 ---
 .../org/apache/hadoop/cli/CLITestCmdDFS.java    |    8 +-
 .../hadoop/cli/CLITestCmdErasureCoding.java     |    5 +-
 .../java/org/apache/hadoop/cli/TestAclCLI.java  |    2 +-
 .../apache/hadoop/cli/TestCacheAdminCLI.java    |    7 +-
 .../apache/hadoop/cli/TestCryptoAdminCLI.java   |    6 +-
 .../org/apache/hadoop/cli/TestDeleteCLI.java    |   92 +
 .../apache/hadoop/cli/TestErasureCodingCLI.java |    2 +-
 .../java/org/apache/hadoop/cli/TestHDFSCLI.java |    4 +-
 .../org/apache/hadoop/cli/TestXAttrCLI.java     |    2 +-
 .../hadoop/fs/permission/TestStickyBit.java     |    3 +
 .../apache/hadoop/hdfs/BlockReaderTestUtil.java |    3 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |   24 +-
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |    2 +-
 .../hadoop/hdfs/TestAppendSnapshotTruncate.java |    3 +-
 .../hadoop/hdfs/TestBalancerBandwidth.java      |   55 +-
 .../hadoop/hdfs/TestBlockStoragePolicy.java     |   24 +-
 .../hadoop/hdfs/TestDFSClientFailover.java      |    5 +-
 .../apache/hadoop/hdfs/TestDFSPermission.java   |   42 +-
 .../org/apache/hadoop/hdfs/TestDFSShell.java    |   56 +
 .../hadoop/hdfs/TestDFSShellGenericOptions.java |   13 +-
 .../hadoop/hdfs/TestDefaultNameNodePort.java    |   30 +-
 .../hadoop/hdfs/TestEncryptedTransfer.java      |    4 +-
 .../org/apache/hadoop/hdfs/TestFileAppend.java  |   72 -
 .../org/apache/hadoop/hdfs/TestFileStatus.java  |    5 +-
 .../org/apache/hadoop/hdfs/TestGetBlocks.java   |    7 +-
 .../apache/hadoop/hdfs/TestPersistBlocks.java   |    5 +-
 .../java/org/apache/hadoop/hdfs/TestQuota.java  |   32 +
 .../hdfs/TestWriteStripedFileWithFailure.java   |    3 +
 .../sasl/SaslDataTransferTestCase.java          |    2 +-
 .../datatransfer/sasl/TestSaslDataTransfer.java |    2 +-
 .../hadoop/hdfs/protocolPB/TestPBHelper.java    |   32 +-
 .../hdfs/qjournal/TestSecureNNWithQJM.java      |    2 +-
 .../hdfs/qjournal/server/TestJournalNode.java   |    9 +
 .../hdfs/server/balancer/TestBalancer.java      |  199 +-
 .../balancer/TestBalancerWithHANameNodes.java   |    8 +-
 .../TestBalancerWithMultipleNameNodes.java      |  177 +-
 .../balancer/TestBalancerWithNodeGroup.java     |    4 +-
 .../blockmanagement/TestBlockTokenWithDFS.java  |    4 +-
 .../blockmanagement/TestReplicationPolicy.java  |    3 -
 .../TestReplicationPolicyWithNodeGroup.java     |   19 +
 .../hdfs/server/datanode/DataNodeTestUtils.java |   86 +-
 .../server/datanode/SimulatedFSDataset.java     |    2 +-
 .../server/datanode/TestDataNodeMXBean.java     |    6 +
 .../datanode/TestDataNodeMetricsLogger.java     |  224 ++
 .../datanode/TestDataNodeRollingUpgrade.java    |    8 +-
 .../TestDataNodeTransferSocketSize.java         |   71 +
 .../datanode/TestDataNodeVolumeFailure.java     |    4 +-
 .../server/datanode/TestDirectoryScanner.java   |    2 +-
 .../datanode/extdataset/ExternalVolumeImpl.java |    2 +-
 .../fsdataset/impl/FsDatasetTestUtil.java       |    6 -
 .../fsdataset/impl/LazyPersistTestCase.java     |   16 +-
 .../fsdataset/impl/TestDatanodeRestart.java     |   72 -
 .../datanode/fsdataset/impl/TestLazyWriter.java |    1 +
 .../fsdataset/impl/TestRbwSpaceReservation.java |  452 ----
 .../fsdataset/impl/TestSpaceReservation.java    |  576 +++++
 .../server/namenode/NNThroughputBenchmark.java  |    3 +-
 .../hdfs/server/namenode/NameNodeAdapter.java   |    2 +-
 .../TestCheckPointForSecurityTokens.java        |    4 +-
 .../namenode/TestDiskspaceQuotaUpdate.java      |    9 +-
 .../hdfs/server/namenode/TestEditLog.java       |  103 +-
 .../server/namenode/TestFSEditLogLoader.java    |   13 +-
 .../namenode/TestFSImageWithSnapshot.java       |    3 +-
 .../hdfs/server/namenode/TestFSNamesystem.java  |   25 +-
 .../hdfs/server/namenode/TestFileTruncate.java  |    5 +-
 .../hdfs/server/namenode/TestINodeFile.java     |    4 +-
 .../namenode/TestListCorruptFileBlocks.java     |    3 +-
 .../hdfs/server/namenode/TestMetaSave.java      |   11 +
 .../server/namenode/TestNameNodeMXBean.java     |   20 +
 .../namenode/TestNameNodeMetricsLogger.java     |    5 +-
 .../server/namenode/TestStorageRestore.java     |    3 +-
 .../server/namenode/TestStripedINodeFile.java   |    2 +-
 .../namenode/ha/TestFailureToReadEdits.java     |    3 +-
 .../TestGetContentSummaryWithSnapshot.java      |  126 +
 .../hdfs/web/TestByteRangeInputStream.java      |   79 +
 .../org/apache/hadoop/tools/TestJMXGet.java     |    3 +
 .../test/resources/hadoop-metrics2.properties   |   85 +
 .../src/test/resources/log4j.properties         |   13 +
 .../src/test/resources/testDeleteConf.xml       |   83 +
 .../src/test/resources/testHDFSConf.xml         |  198 +-
 hadoop-mapreduce-project/CHANGES.txt            |  101 +-
 hadoop-mapreduce-project/bin/mapred             |    8 +
 .../v2/app/job/impl/TaskAttemptImpl.java        |   92 +-
 .../v2/app/rm/RMContainerAllocator.java         |   12 +-
 .../v2/app/rm/RMContainerRequestor.java         |    2 +-
 .../apache/hadoop/mapreduce/v2/app/MRApp.java   |   11 +-
 .../v2/app/job/impl/TestTaskAttempt.java        |  154 ++
 .../v2/app/rm/TestRMContainerAllocator.java     |  113 +-
 .../org/apache/hadoop/mapreduce/Cluster.java    |    2 +-
 .../lib/input/UncompressedSplitLineReader.java  |   31 +-
 .../lib/output/FileOutputCommitter.java         |   52 +-
 .../hadoop/mapred/TestLineRecordReader.java     |  138 ++
 .../lib/input/TestLineRecordReader.java         |  161 ++
 .../org/apache/hadoop/mapred/YARNRunner.java    |    4 +
 .../org/apache/hadoop/cli/CLITestCmdMR.java     |    3 +-
 .../org/apache/hadoop/fs/TestFileSystem.java    |   14 +-
 .../java/org/apache/hadoop/hdfs/NNBench.java    |   14 +-
 .../apache/hadoop/mapred/TestYARNRunner.java    |   10 +-
 .../mapreduce/security/TestMRCredentials.java   |    4 +-
 .../apache/hadoop/mapred/ShuffleHandler.java    |  177 +-
 .../hadoop/mapred/TestShuffleHandler.java       |  129 +
 hadoop-project/pom.xml                          |    7 +-
 hadoop-tools/hadoop-archive-logs/pom.xml        |  171 ++
 .../apache/hadoop/tools/HadoopArchiveLogs.java  |  403 +++
 .../hadoop/tools/HadoopArchiveLogsRunner.java   |  180 ++
 .../hadoop/tools/TestHadoopArchiveLogs.java     |  293 +++
 .../tools/TestHadoopArchiveLogsRunner.java      |  143 ++
 .../tools/mapred/UniformSizeInputFormat.java    |    3 +-
 .../hadoop/tools/util/ProducerConsumer.java     |   19 +-
 .../apache/hadoop/tools/util/WorkReport.java    |    2 +-
 .../apache/hadoop/tools/util/WorkRequest.java   |    2 +-
 .../hadoop/mapred/gridmix/SerialJobFactory.java |    2 +-
 .../hadoop/yarn/sls/RumenToSLSConverter.java    |    2 +-
 hadoop-tools/hadoop-tools-dist/pom.xml          |    5 +
 hadoop-tools/pom.xml                            |    1 +
 hadoop-yarn-project/CHANGES.txt                 |  504 ++--
 .../MoveApplicationAcrossQueuesResponse.java    |    5 +-
 .../yarn/api/records/ApplicationReport.java     |   31 +-
 .../hadoop/yarn/api/records/NodeLabel.java      |   16 +
 .../hadoop/yarn/api/records/ResourceOption.java |    3 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |   37 +-
 .../hadoop/yarn/server/api/ContainerType.java   |    1 -
 .../ResourceManagerAdministrationProtocol.java  |   16 +-
 .../RefreshNodesResourcesRequest.java           |   39 +
 .../RefreshNodesResourcesResponse.java          |   39 +
 ...esourcemanager_administration_protocol.proto |    3 +-
 ..._server_resourcemanager_service_protos.proto |    6 +
 .../src/main/proto/yarn_protos.proto            |    2 +
 .../yarn/conf/TestYarnConfigurationFields.java  |    2 +
 .../hadoop-yarn/hadoop-yarn-client/pom.xml      |   12 +
 .../yarn/client/api/impl/AMRMClientImpl.java    |    2 +-
 .../impl/ContainerManagementProtocolProxy.java  |   14 +-
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |   10 +-
 .../apache/hadoop/yarn/client/cli/LogsCLI.java  |  104 +-
 .../apache/hadoop/yarn/client/cli/QueueCLI.java |   11 +-
 .../hadoop/yarn/client/cli/RMAdminCLI.java      |   61 +-
 .../hadoop/yarn/client/cli/TestLogsCLI.java     |   53 +-
 .../hadoop/yarn/client/cli/TestRMAdminCLI.java  |   31 +-
 .../hadoop/yarn/client/cli/TestYarnCLI.java     |    8 +-
 .../application_1440536969523_0001.har/_SUCCESS |    0
 .../application_1440536969523_0001.har/_index   |    3 +
 .../_masterindex                                |    2 +
 .../application_1440536969523_0001.har/part-0   |  Bin 0 -> 795 bytes
 .../hadoop-yarn/hadoop-yarn-common/pom.xml      |    4 +
 .../impl/pb/ApplicationReportPBImpl.java        |   38 +
 .../org/apache/hadoop/yarn/client/RMProxy.java  |    3 +-
 .../apache/hadoop/yarn/client/ServerProxy.java  |    3 +-
 .../yarn/logaggregation/LogCLIHelpers.java      |   16 +-
 .../nodelabels/CommonNodeLabelsManager.java     |    2 +-
 .../yarn/security/ContainerTokenSelector.java   |    2 +-
 .../hadoop/yarn/security/NMTokenSelector.java   |    2 +-
 ...nagerAdministrationProtocolPBClientImpl.java |   19 +
 ...agerAdministrationProtocolPBServiceImpl.java |   22 +
 .../pb/RefreshNodesResourcesRequestPBImpl.java  |   72 +
 .../pb/RefreshNodesResourcesResponsePBImpl.java |   72 +
 .../yarn/webapp/log/AggregatedLogsBlock.java    |    7 +
 .../src/main/resources/yarn-default.xml         |   63 +-
 .../hadoop/yarn/api/TestApplicatonReport.java   |    2 +-
 .../hadoop/yarn/api/TestPBImplRecords.java      |   16 +
 .../hadoop/yarn/event/TestAsyncDispatcher.java  |    2 +
 .../logaggregation/TestAggregatedLogsBlock.java |   63 +-
 .../hadoop/yarn/webapp/JerseyTestBase.java      |   14 +-
 .../application_1440536969523_0001.har/_SUCCESS |    0
 .../application_1440536969523_0001.har/_index   |    3 +
 .../_masterindex                                |    2 +
 .../application_1440536969523_0001.har/part-0   |  Bin 0 -> 795 bytes
 .../hadoop-yarn/hadoop-yarn-registry/pom.xml    |    2 +-
 ...pplicationHistoryManagerOnTimelineStore.java |   29 +-
 .../yarn/server/api/records/NodeStatus.java     |    2 +-
 .../api/records/impl/pb/NodeStatusPBImpl.java   |    8 +-
 .../metrics/ApplicationMetricsConstants.java    |    6 +
 .../server/utils/YarnServerSecurityUtils.java   |  142 ++
 .../hadoop/yarn/server/webapp/AppBlock.java     |    6 +
 .../hadoop/yarn/server/webapp/WebPageUtils.java |    3 +-
 .../hadoop/yarn/server/webapp/WebServices.java  |    9 +
 .../hadoop/yarn/server/webapp/dao/AppInfo.java  |   12 +
 .../nodemanager/LinuxContainerExecutor.java     |    2 +-
 .../yarn/server/nodemanager/NodeManager.java    |    2 +-
 .../nodemanager/NodeStatusUpdaterImpl.java      |   25 +-
 .../amrmproxy/AMRMProxyApplicationContext.java  |   70 +
 .../AMRMProxyApplicationContextImpl.java        |  132 +
 .../nodemanager/amrmproxy/AMRMProxyService.java |  592 +++++
 .../amrmproxy/AMRMProxyTokenSecretManager.java  |  265 ++
 .../amrmproxy/AbstractRequestInterceptor.java   |  102 +
 .../amrmproxy/DefaultRequestInterceptor.java    |  138 ++
 .../amrmproxy/RequestInterceptor.java           |   71 +
 .../containermanager/ContainerManagerImpl.java  |   73 +-
 .../localizer/LocalResourcesTrackerImpl.java    |   64 +-
 .../localizer/ResourceLocalizationService.java  |    2 +-
 .../logaggregation/AppLogAggregator.java        |    2 +
 .../logaggregation/AppLogAggregatorImpl.java    |    6 +
 .../logaggregation/LogAggregationService.java   |   14 +-
 .../nodelabels/AbstractNodeLabelsProvider.java  |    7 +-
 .../ConfigurationNodeLabelsProvider.java        |   11 -
 .../nodemanager/webapp/NMWebServices.java       |   12 +-
 .../nodemanager/webapp/dao/ContainerInfo.java   |   46 +-
 .../nodemanager/DummyContainerManager.java      |    7 +-
 .../yarn/server/nodemanager/TestEventFlow.java  |    3 +-
 .../nodemanager/TestNodeManagerResync.java      |    2 +-
 .../nodemanager/TestNodeStatusUpdater.java      |    6 +-
 .../amrmproxy/BaseAMRMProxyTest.java            |  677 +++++
 .../amrmproxy/MockRequestInterceptor.java       |   65 +
 .../amrmproxy/MockResourceManagerFacade.java    |  469 ++++
 .../PassThroughRequestInterceptor.java          |   58 +
 .../amrmproxy/TestAMRMProxyService.java         |  484 ++++
 .../BaseContainerManagerTest.java               |    3 +-
 .../containermanager/TestContainerManager.java  |    5 +-
 .../TestContainerManagerRecovery.java           |    3 +-
 .../containermanager/TestNMProxy.java           |    3 +-
 .../TestLocalResourcesTrackerImpl.java          |   86 +-
 .../localizer/TestResourceRetention.java        |    2 +-
 .../TestLogAggregationService.java              |   17 +-
 .../TestConfigurationNodeLabelsProvider.java    |   69 +-
 .../webapp/TestNMWebServicesContainers.java     |   13 +-
 .../server/resourcemanager/AdminService.java    |   81 +-
 .../ApplicationMasterService.java               |   69 +-
 .../server/resourcemanager/ClientRMService.java |    2 +-
 .../resourcemanager/EmbeddedElectorService.java |    4 +
 .../resourcemanager/NodesListManager.java       |  144 +-
 .../resourcemanager/RMActiveServiceContext.java |   16 +-
 .../server/resourcemanager/RMAppManager.java    |    9 +
 .../yarn/server/resourcemanager/RMContext.java  |    5 +
 .../server/resourcemanager/RMContextImpl.java   |   12 +-
 .../resourcemanager/RMFatalEventType.java       |    5 +-
 .../server/resourcemanager/ResourceManager.java |    4 +-
 .../resourcemanager/ResourceTrackerService.java |   20 +-
 .../blacklist/BlacklistManager.java             |   47 +
 .../blacklist/BlacklistUpdates.java             |   47 +
 .../blacklist/DisabledBlacklistManager.java     |   45 +
 .../blacklist/SimpleBlacklistManager.java       |   84 +
 .../metrics/ApplicationCreatedEvent.java        |   16 +-
 .../metrics/SystemMetricsPublisher.java         |   14 +-
 .../placement/PlacementManager.java             |   95 +
 .../placement/PlacementRule.java                |   55 +
 .../UserGroupMappingPlacementRule.java          |  164 ++
 .../resourcemanager/recovery/RMStateStore.java  |   15 +-
 .../resource/DynamicResourceConfiguration.java  |  149 ++
 .../server/resourcemanager/rmapp/RMApp.java     |    6 +
 .../server/resourcemanager/rmapp/RMAppImpl.java |   59 +-
 .../rmapp/attempt/RMAppAttempt.java             |    7 +
 .../rmapp/attempt/RMAppAttemptImpl.java         |   58 +-
 .../server/resourcemanager/rmnode/RMNode.java   |    3 -
 .../resourcemanager/rmnode/RMNodeEventType.java |    2 +-
 .../resourcemanager/rmnode/RMNodeImpl.java      |  456 ++--
 .../scheduler/AbstractYarnScheduler.java        |    2 +
 .../scheduler/AppSchedulingInfo.java            |   78 +-
 .../scheduler/SchedulerApplicationAttempt.java  |   37 +-
 .../scheduler/YarnScheduler.java                |   13 +
 .../scheduler/capacity/CapacityScheduler.java   |  153 +-
 .../CapacitySchedulerConfiguration.java         |   32 +-
 .../scheduler/capacity/LeafQueue.java           |    6 +-
 .../common/fica/FiCaSchedulerUtils.java         |   48 -
 .../scheduler/fair/FSAppAttempt.java            |   19 +-
 .../scheduler/fair/FairScheduler.java           |   37 +-
 .../fair/FairSchedulerConfiguration.java        |   22 +-
 .../scheduler/fifo/FifoScheduler.java           |   11 +-
 .../scheduler/policy/OrderingPolicy.java        |    2 +-
 .../resourcemanager/webapp/AboutBlock.java      |    2 +
 .../webapp/CapacitySchedulerPage.java           |   17 +-
 .../webapp/JAXBContextResolver.java             |    2 +-
 .../webapp/MetricsOverviewTable.java            |    3 +
 .../resourcemanager/webapp/NodeLabelsPage.java  |    6 +-
 .../resourcemanager/webapp/RMAppBlock.java      |    3 +-
 .../server/resourcemanager/webapp/RMWebApp.java |    5 +
 .../resourcemanager/webapp/RMWebAppFilter.java  |    5 +-
 .../resourcemanager/webapp/RMWebServices.java   |  123 +-
 .../webapp/dao/AppAttemptInfo.java              |   10 +-
 .../resourcemanager/webapp/dao/AppInfo.java     |   29 +-
 .../resourcemanager/webapp/dao/AppPriority.java |   44 +
 .../dao/CapacitySchedulerLeafQueueInfo.java     |   14 +-
 .../resourcemanager/webapp/dao/ClusterInfo.java |    6 +
 .../webapp/dao/SchedulerInfo.java               |    6 +
 .../yarn/server/resourcemanager/MockRM.java     |   20 +-
 .../server/resourcemanager/RMHATestBase.java    |    2 +-
 .../server/resourcemanager/TestAppManager.java  |   54 +-
 .../resourcemanager/TestClientRMService.java    |  295 +--
 .../resourcemanager/TestRMAdminService.java     |   46 +
 .../yarn/server/resourcemanager/TestRMHA.java   |   71 +
 .../resourcemanager/TestRMNodeTransitions.java  |  202 +-
 .../TestTokenClientRMService.java               |  300 +++
 .../TestWorkPreservingRMRestart.java            |   21 +-
 .../applicationsmanager/MockAsm.java            |   10 +
 .../applicationsmanager/TestAMRestart.java      |  149 +-
 .../blacklist/TestBlacklistManager.java         |  118 +
 .../TestRMAppLogAggregationStatus.java          |    2 +-
 .../metrics/TestSystemMetricsPublisher.java     |   20 +-
 .../TestUserGroupMappingPlacementRule.java      |   89 +
 .../server/resourcemanager/rmapp/MockRMApp.java |   13 +
 .../rmapp/TestNodesListManager.java             |  102 +
 .../rmapp/TestRMAppTransitions.java             |    9 +-
 .../scheduler/TestAbstractYarnScheduler.java    |   14 +-
 .../capacity/TestCapacityScheduler.java         |  147 +-
 .../scheduler/capacity/TestQueueMappings.java   |  203 +-
 .../scheduler/fair/FairSchedulerTestBase.java   |    8 +-
 .../scheduler/fair/TestFairScheduler.java       |  120 +-
 .../scheduler/fifo/TestFifoScheduler.java       |    2 +-
 .../security/TestRMDelegationTokens.java        |    3 +
 .../resourcemanager/webapp/TestNodesPage.java   |    5 +-
 .../webapp/TestRMWebServices.java               |    8 +-
 .../webapp/TestRMWebServicesApps.java           |   34 +-
 .../TestRMWebServicesAppsModification.java      |  120 +
 .../webapp/TestRMWebServicesCapacitySched.java  |    2 +-
 .../TestRMWebServicesDelegationTokens.java      |    6 +-
 .../hadoop/yarn/server/MiniYARNCluster.java     |   42 +-
 .../src/site/markdown/CapacityScheduler.md      |    6 +-
 .../src/site/markdown/ResourceManagerRest.md    |  152 +-
 .../src/site/markdown/TimelineServer.md         |   38 +-
 500 files changed, 21586 insertions(+), 9191 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
index 0000000,23e8f57..2121dcf
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
@@@ -1,0 -1,83 +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.hdfs.protocol;
+ 
+ import java.net.URI;
+ 
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.classification.InterfaceStability;
+ import org.apache.hadoop.fs.FileEncryptionInfo;
+ import org.apache.hadoop.fs.LocatedFileStatus;
+ import org.apache.hadoop.fs.Path;
+ import org.apache.hadoop.fs.permission.FsPermission;
+ import org.apache.hadoop.hdfs.DFSUtilClient;
+ 
+ /** 
+  * Interface that represents the over the wire information
+  * including block locations for a file.
+  */
+ @InterfaceAudience.Private
+ @InterfaceStability.Evolving
+ public class HdfsLocatedFileStatus extends HdfsFileStatus {
+   private final LocatedBlocks locations;
+ 
+   /**
+    * Constructor
+    * 
+    * @param length size
+    * @param isdir if this is directory
+    * @param block_replication the file's replication factor
+    * @param blocksize the file's block size
+    * @param modification_time most recent modification time
+    * @param access_time most recent access time
+    * @param permission permission
+    * @param owner owner
+    * @param group group
+    * @param symlink symbolic link
+    * @param path local path name in java UTF8 format 
+    * @param fileId the file id
+    * @param locations block locations
+    * @param feInfo file encryption info
+    */
+   public HdfsLocatedFileStatus(long length, boolean isdir,
+       int block_replication, long blocksize, long modification_time,
+       long access_time, FsPermission permission, String owner, String group,
+       byte[] symlink, byte[] path, long fileId, LocatedBlocks locations,
 -      int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy) {
++      int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy,
++      ErasureCodingPolicy ecPolicy) {
+     super(length, isdir, block_replication, blocksize, modification_time,
+         access_time, permission, owner, group, symlink, path, fileId,
 -        childrenNum, feInfo, storagePolicy);
++        childrenNum, feInfo, storagePolicy, ecPolicy);
+     this.locations = locations;
+   }
+ 
+   public LocatedBlocks getBlockLocations() {
+     return locations;
+   }
+ 
+   public final LocatedFileStatus makeQualifiedLocated(URI defaultUri,
+       Path path) {
+     return new LocatedFileStatus(getLen(), isDir(), getReplication(),
+         getBlockSize(), getModificationTime(),
+         getAccessTime(),
+         getPermission(), getOwner(), getGroup(),
+         isSymlink() ? new Path(getSymlink()) : null,
+         (getFullPath(path)).makeQualified(
+             defaultUri, null), // fully-qualify path
+         DFSUtilClient.locatedBlocks2Locations(getBlockLocations()));
+   }
+ }


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

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 53c6cdb,28ea866..8874c4d
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@@ -23,8 -23,8 +23,9 @@@ import java.util.concurrent.TimeUnit
  import org.apache.hadoop.classification.InterfaceAudience;
  import org.apache.hadoop.fs.CommonConfigurationKeys;
  import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant;
  import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaLruTracker;
  import org.apache.hadoop.http.HttpConfig;
  
@@@ -171,8 -171,8 +172,10 @@@ public class DFSConfigKeys extends Comm
    public static final int     DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT = 3;
    public static final String  DFS_NAMENODE_REPLICATION_MIN_KEY = "dfs.namenode.replication.min";
    public static final int     DFS_NAMENODE_REPLICATION_MIN_DEFAULT = 1;
 +  public static final String  DFS_NAMENODE_STRIPE_MIN_KEY = "dfs.namenode.stripe.min";
 +  public static final int     DFS_NAMENODE_STRIPE_MIN_DEFAULT = 1;
+   public static final String  DFS_NAMENODE_SAFEMODE_REPLICATION_MIN_KEY =
+       "dfs.namenode.safemode.replication.min";
    public static final String  DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY = "dfs.namenode.replication.pending.timeout-sec";
    public static final int     DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_DEFAULT = -1;
    public static final String  DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY = "dfs.namenode.replication.max-streams";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index ac927ef,f4cf4c2..5bf52c5
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@@ -84,9 -85,7 +85,8 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
  import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
  import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
- import org.apache.hadoop.hdfs.server.namenode.NameNode;
  import org.apache.hadoop.io.Text;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
  import org.apache.hadoop.net.NetUtils;
  import org.apache.hadoop.security.AccessControlException;
  import org.apache.hadoop.security.Credentials;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index 3217484,d93277c..1e4b899
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@@ -429,10 -422,10 +429,11 @@@ public class ClientNamenodeProtocolServ
            req.getClientName(), flags);
        AppendResponseProto.Builder builder = AppendResponseProto.newBuilder();
        if (result.getLastBlock() != null) {
-         builder.setBlock(PBHelper.convertLocatedBlock(result.getLastBlock()));
 -        builder.setBlock(PBHelperClient.convert(result.getLastBlock()));
++        builder.setBlock(PBHelperClient.convertLocatedBlock(
++            result.getLastBlock()));
        }
        if (result.getFileStatus() != null) {
-         builder.setStat(PBHelper.convert(result.getFileStatus()));
+         builder.setStat(PBHelperClient.convert(result.getFileStatus()));
        }
        return builder.build();
      } catch (IOException e) {
@@@ -505,7 -498,7 +506,7 @@@
            (favor == null || favor.size() == 0) ? null : favor
                .toArray(new String[favor.size()]));
        return AddBlockResponseProto.newBuilder()
-           .setBlock(PBHelper.convertLocatedBlock(result)).build();
 -          .setBlock(PBHelperClient.convert(result)).build();
++          .setBlock(PBHelperClient.convertLocatedBlock(result)).build();
      } catch (IOException e) {
        throw new ServiceException(e);
      }
@@@ -525,11 -518,11 +526,11 @@@
                new DatanodeInfoProto[existingList.size()])),
            existingStorageIDsList.toArray(
                new String[existingStorageIDsList.size()]),
-           PBHelper.convert(excludesList.toArray(
-               new DatanodeInfoProto[excludesList.size()])), 
+           PBHelperClient.convert(excludesList.toArray(
+               new DatanodeInfoProto[excludesList.size()])),
            req.getNumAdditionalNodes(), req.getClientName());
        return GetAdditionalDatanodeResponseProto.newBuilder().setBlock(
-           PBHelper.convertLocatedBlock(result))
 -          PBHelperClient.convert(result))
++      PBHelperClient.convertLocatedBlock(result))
            .build();
      } catch (IOException e) {
        throw new ServiceException(e);
@@@ -555,7 -548,7 +556,7 @@@
        ReportBadBlocksRequestProto req) throws ServiceException {
      try {
        List<LocatedBlockProto> bl = req.getBlocksList();
-       server.reportBadBlocks(PBHelper.convertLocatedBlocks(
 -      server.reportBadBlocks(PBHelperClient.convertLocatedBlock(
++      server.reportBadBlocks(PBHelperClient.convertLocatedBlocks(
            bl.toArray(new LocatedBlockProto[bl.size()])));
      } catch (IOException e) {
        throw new ServiceException(e);
@@@ -960,8 -953,8 +961,8 @@@
        RpcController controller, UpdateBlockForPipelineRequestProto req)
        throws ServiceException {
      try {
-       LocatedBlockProto result = PBHelper.convertLocatedBlock(
 -      LocatedBlockProto result = PBHelperClient.convert(server
 -          .updateBlockForPipeline(PBHelperClient.convert(req.getBlock()),
++      LocatedBlockProto result = PBHelperClient.convertLocatedBlock(
 +          server.updateBlockForPipeline(PBHelperClient.convert(req.getBlock()),
                req.getClientName()));
        return UpdateBlockForPipelineResponseProto.newBuilder().setBlock(result)
            .build();
@@@ -1401,20 -1394,6 +1402,20 @@@
    }
  
    @Override
 +  public SetErasureCodingPolicyResponseProto setErasureCodingPolicy(
 +      RpcController controller, SetErasureCodingPolicyRequestProto req)
 +      throws ServiceException {
 +    try {
-       ErasureCodingPolicy ecPolicy = req.hasEcPolicy() ? PBHelper.convertErasureCodingPolicy(req
-           .getEcPolicy()) : null;
++      ErasureCodingPolicy ecPolicy = req.hasEcPolicy() ?
++          PBHelperClient.convertErasureCodingPolicy(req.getEcPolicy()) : null;
 +      server.setErasureCodingPolicy(req.getSrc(), ecPolicy);
 +      return SetErasureCodingPolicyResponseProto.newBuilder().build();
 +    } catch (IOException e) {
 +      throw new ServiceException(e);
 +    }
 +  }
 +
 +  @Override
    public SetXAttrResponseProto setXAttr(RpcController controller,
        SetXAttrRequestProto req) throws ServiceException {
      try {
@@@ -1535,35 -1514,4 +1536,35 @@@
        throw new ServiceException(e);
      }
    }
 +
 +  @Override
 +  public GetErasureCodingPoliciesResponseProto getErasureCodingPolicies(RpcController controller,
 +      GetErasureCodingPoliciesRequestProto request) throws ServiceException {
 +    try {
 +      ErasureCodingPolicy[] ecPolicies = server.getErasureCodingPolicies();
 +      GetErasureCodingPoliciesResponseProto.Builder resBuilder = GetErasureCodingPoliciesResponseProto
 +          .newBuilder();
 +      for (ErasureCodingPolicy ecPolicy : ecPolicies) {
-         resBuilder.addEcPolicies(PBHelper.convertErasureCodingPolicy(ecPolicy));
++        resBuilder.addEcPolicies(PBHelperClient.convertErasureCodingPolicy(ecPolicy));
 +      }
 +      return resBuilder.build();
 +    } catch (IOException e) {
 +      throw new ServiceException(e);
 +    }
 +  }
 +
 +  @Override
 +  public GetErasureCodingPolicyResponseProto getErasureCodingPolicy(RpcController controller,
 +      GetErasureCodingPolicyRequestProto request) throws ServiceException {
 +    try {
 +      ErasureCodingPolicy ecPolicy = server.getErasureCodingPolicy(request.getSrc());
 +      GetErasureCodingPolicyResponseProto.Builder builder = GetErasureCodingPolicyResponseProto.newBuilder();
 +      if (ecPolicy != null) {
-         builder.setEcPolicy(PBHelper.convertErasureCodingPolicy(ecPolicy));
++        builder.setEcPolicy(PBHelperClient.convertErasureCodingPolicy(ecPolicy));
 +      }
 +      return builder.build();
 +    } catch (IOException e) {
 +      throw new ServiceException(e);
 +    }
 +  }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 8419244,f4ce46d..7b02691
--- 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
@@@ -338,9 -327,9 +338,9 @@@ public class ClientNamenodeProtocolTran
          .build();
      try {
        AppendResponseProto res = rpcProxy.append(null, req);
-       LocatedBlock lastBlock = res.hasBlock() ? PBHelper
+       LocatedBlock lastBlock = res.hasBlock() ? PBHelperClient
 -          .convert(res.getBlock()) : null;
 +          .convertLocatedBlockProto(res.getBlock()) : null;
-       HdfsFileStatus stat = (res.hasStat()) ? PBHelper.convert(res.getStat())
+       HdfsFileStatus stat = (res.hasStat()) ? PBHelperClient.convert(res.getStat())
            : null;
        return new LastBlockWithStatus(lastBlock, stat);
      } catch (ServiceException e) {
@@@ -427,8 -416,7 +427,8 @@@
        req.addAllFavoredNodes(Arrays.asList(favoredNodes));
      }
      try {
-       return PBHelper.convertLocatedBlockProto(
 -      return PBHelperClient.convert(rpcProxy.addBlock(null, req.build()).getBlock());
++      return PBHelperClient.convertLocatedBlockProto(
 +          rpcProxy.addBlock(null, req.build()).getBlock());
      } catch (ServiceException e) {
        throw ProtobufHelper.getRemoteException(e);
      }
@@@ -453,8 -441,8 +453,8 @@@
          .setClientName(clientName)
          .build();
      try {
-       return PBHelper.convertLocatedBlockProto(
 -      return PBHelperClient.convert(rpcProxy.getAdditionalDatanode(null, req)
 -          .getBlock());
++      return PBHelperClient.convertLocatedBlockProto(
 +          rpcProxy.getAdditionalDatanode(null, req).getBlock());
      } catch (ServiceException e) {
        throw ProtobufHelper.getRemoteException(e);
      }
@@@ -481,7 -469,7 +481,7 @@@
    @Override
    public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
      ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder()
-         .addAllBlocks(Arrays.asList(PBHelper.convertLocatedBlocks(blocks)))
 -        .addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlock(blocks)))
++        .addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlocks(blocks)))
          .build();
      try {
        rpcProxy.reportBadBlocks(null, req);
@@@ -913,7 -901,7 +913,7 @@@
          .setClientName(clientName)
          .build();
      try {
-       return PBHelper.convertLocatedBlockProto(
 -      return PBHelperClient.convert(
++      return PBHelperClient.convertLocatedBlockProto(
            rpcProxy.updateBlockForPipeline(null, req).getBlock());
      } catch (ServiceException e) {
        throw ProtobufHelper.getRemoteException(e);
@@@ -1419,23 -1407,6 +1419,23 @@@
    }
  
    @Override
 +  public void setErasureCodingPolicy(String src, ErasureCodingPolicy ecPolicy)
 +      throws IOException {
 +    final SetErasureCodingPolicyRequestProto.Builder builder =
 +        SetErasureCodingPolicyRequestProto.newBuilder();
 +    builder.setSrc(src);
 +    if (ecPolicy != null) {
-       builder.setEcPolicy(PBHelper.convertErasureCodingPolicy(ecPolicy));
++      builder.setEcPolicy(PBHelperClient.convertErasureCodingPolicy(ecPolicy));
 +    }
 +    SetErasureCodingPolicyRequestProto req = builder.build();
 +    try {
 +      rpcProxy.setErasureCodingPolicy(null, req);
 +    } catch (ServiceException e) {
 +      throw ProtobufHelper.getRemoteException(e);
 +    }
 +  }
 +
 +  @Override
    public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
        throws IOException {
      SetXAttrRequestProto req = SetXAttrRequestProto.newBuilder()
@@@ -1557,37 -1528,4 +1557,37 @@@
        throw ProtobufHelper.getRemoteException(e);
      }
    }
 +
 +  @Override
 +  public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException {
 +    try {
 +      GetErasureCodingPoliciesResponseProto response = rpcProxy
 +          .getErasureCodingPolicies(null, VOID_GET_EC_POLICIES_REQUEST);
 +      ErasureCodingPolicy[] ecPolicies =
 +          new ErasureCodingPolicy[response.getEcPoliciesCount()];
 +      int i = 0;
 +      for (ErasureCodingPolicyProto ecPolicyProto : response.getEcPoliciesList()) {
-         ecPolicies[i++] = PBHelper.convertErasureCodingPolicy(ecPolicyProto);
++        ecPolicies[i++] = PBHelperClient.convertErasureCodingPolicy(ecPolicyProto);
 +      }
 +      return ecPolicies;
 +    } catch (ServiceException e) {
 +      throw ProtobufHelper.getRemoteException(e);
 +    }
 +  }
 +
 +  @Override
 +  public ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException {
 +    GetErasureCodingPolicyRequestProto req = GetErasureCodingPolicyRequestProto.newBuilder()
 +        .setSrc(src).build();
 +    try {
 +      GetErasureCodingPolicyResponseProto response = rpcProxy.getErasureCodingPolicy(
 +          null, req);
 +      if (response.hasEcPolicy()) {
-         return PBHelper.convertErasureCodingPolicy(response.getEcPolicy());
++        return PBHelperClient.convertErasureCodingPolicy(response.getEcPolicy());
 +      }
 +      return null;
 +    } catch (ServiceException e) {
 +      throw ProtobufHelper.getRemoteException(e);
 +    }
 +  }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
index f20e58a,18f89f8..194e563
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
@@@ -281,7 -281,7 +281,7 @@@ public class DatanodeProtocolClientSide
      ReportBadBlocksRequestProto.Builder builder = ReportBadBlocksRequestProto
          .newBuilder();
      for (int i = 0; i < blocks.length; i++) {
-       builder.addBlocks(i, PBHelper.convertLocatedBlock(blocks[i]));
 -      builder.addBlocks(i, PBHelperClient.convert(blocks[i]));
++      builder.addBlocks(i, PBHelperClient.convertLocatedBlock(blocks[i]));
      }
      ReportBadBlocksRequestProto req = builder.build();
      try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
index 1ff80b3,94d1f0c..a1ea9a6
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
@@@ -259,7 -259,7 +259,7 @@@ public class DatanodeProtocolServerSide
      List<LocatedBlockProto> lbps = request.getBlocksList();
      LocatedBlock [] blocks = new LocatedBlock [lbps.size()];
      for(int i=0; i<lbps.size(); i++) {
-       blocks[i] = PBHelper.convertLocatedBlockProto(lbps.get(i));
 -      blocks[i] = PBHelperClient.convert(lbps.get(i));
++      blocks[i] = PBHelperClient.convertLocatedBlockProto(lbps.get(i));
      }
      try {
        impl.reportBadBlocks(blocks);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index ce39e15,3de4513..ece9984
--- 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
@@@ -17,108 -17,23 +17,25 @@@
   */
  package org.apache.hadoop.hdfs.protocolPB;
  
- import static com.google.common.base.Preconditions.checkNotNull;
- import static org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos
-     .EncryptionZoneProto;
- import static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CipherSuiteProto;
- import static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CryptoProtocolVersionProto;
- 
- import java.io.IOException;
  import java.util.ArrayList;
  import java.util.Arrays;
 +import java.util.Collection;
- import java.util.EnumSet;
- import java.util.HashMap;
  import java.util.List;
- import java.util.Map;
- import java.util.Map.Entry;
- import java.util.Set;
- 
- import org.apache.hadoop.fs.CacheFlag;
- import org.apache.hadoop.fs.ContentSummary;
- import org.apache.hadoop.fs.CreateFlag;
- import org.apache.hadoop.fs.FsServerDefaults;
- import org.apache.hadoop.fs.Path;
+ 
+ import com.google.protobuf.ByteString;
+ 
  import org.apache.hadoop.fs.StorageType;
- import org.apache.hadoop.fs.XAttr;
- import org.apache.hadoop.fs.XAttrSetFlag;
- import org.apache.hadoop.fs.permission.AclEntry;
- import org.apache.hadoop.fs.permission.AclEntryScope;
- import org.apache.hadoop.fs.permission.AclEntryType;
- import org.apache.hadoop.fs.permission.AclStatus;
- import org.apache.hadoop.fs.permission.FsAction;
- import org.apache.hadoop.fs.permission.FsPermission;
  import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
  import org.apache.hadoop.ha.proto.HAServiceProtocolProtos;
  import org.apache.hadoop.hdfs.DFSUtilClient;
- import org.apache.hadoop.hdfs.inotify.EventBatch;
- import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
- import org.apache.hadoop.hdfs.inotify.Event;
- import org.apache.hadoop.hdfs.inotify.EventBatchList;
  import org.apache.hadoop.hdfs.protocol.Block;
- import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
- import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
- import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
- import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
- import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
- import org.apache.hadoop.hdfs.protocol.CachePoolStats;
- import org.apache.hadoop.crypto.CipherOption;
- import org.apache.hadoop.crypto.CipherSuite;
- import org.apache.hadoop.hdfs.protocol.ClientProtocol;
- import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
- import org.apache.hadoop.crypto.CryptoProtocolVersion;
  import org.apache.hadoop.hdfs.protocol.DatanodeID;
  import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
- import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
- import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo;
- import org.apache.hadoop.hdfs.protocol.DirectoryListing;
- import org.apache.hadoop.hdfs.protocol.EncryptionZone;
- import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
- import org.apache.hadoop.fs.FileEncryptionInfo;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
- import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
- import org.apache.hadoop.hdfs.protocol.HdfsConstants;
- import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
- import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
- import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
- import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
- import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
- import org.apache.hadoop.hdfs.protocol.LocatedBlock;
- import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
- import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
- import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
- import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
- import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
- import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
- import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
- import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
- import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto;
- import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryScopeProto;
- import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryTypeProto;
- import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.FsActionProto;
- import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclStatusProto;
- import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusResponseProto;
- import org.apache.hadoop.hdfs.protocol.proto.*;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveEntryProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveStatsProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheFlagProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolStatsProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateFlagProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeReportTypeProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeStorageReportProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeActionProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeInfoProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
- import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto;
+ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 -import org.apache.hadoop.hdfs.protocol.LocatedBlock;
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto;
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
 +import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockECRecoveryCommandProto;
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockIdCommandProto;
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
@@@ -130,14 -45,8 +47,17 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto;
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto;
 +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECRecoveryInfoProto;
++import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaOptionEntryProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlocksWithLocationsProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointCommandProto;
@@@ -381,73 -168,23 +182,34 @@@ public class PBHelper 
          si, convert(reg.getRole()));
    }
  
-   // DatanodeId
-   public static DatanodeID convert(DatanodeIDProto dn) {
-     return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getDatanodeUuid(),
-         dn.getXferPort(), dn.getInfoPort(), dn.hasInfoSecurePort() ? dn
-         .getInfoSecurePort() : 0, dn.getIpcPort());
-   }
- 
-   // Arrays of DatanodeId
-   public static DatanodeIDProto[] convert(DatanodeID[] did) {
-     if (did == null)
-       return null;
-     final int len = did.length;
-     DatanodeIDProto[] result = new DatanodeIDProto[len];
-     for (int i = 0; i < len; ++i) {
-       result[i] = PBHelperClient.convert(did[i]);
-     }
-     return result;
-   }
-   
-   public static DatanodeID[] convert(DatanodeIDProto[] did) {
-     if (did == null) return null;
-     final int len = did.length;
-     DatanodeID[] result = new DatanodeID[len];
-     for (int i = 0; i < len; ++i) {
-       result[i] = convert(did[i]);
-     }
-     return result;
-   }
-   
-   // Block
-   public static BlockProto convert(Block b) {
-     return BlockProto.newBuilder().setBlockId(b.getBlockId())
-         .setGenStamp(b.getGenerationStamp()).setNumBytes(b.getNumBytes())
-         .build();
-   }
- 
-   public static Block convert(BlockProto b) {
-     return new Block(b.getBlockId(), b.getNumBytes(), b.getGenStamp());
-   }
- 
    public static BlockWithLocationsProto convert(BlockWithLocations blk) {
 -    return BlockWithLocationsProto.newBuilder()
 -        .setBlock(PBHelperClient.convert(blk.getBlock()))
 +    BlockWithLocationsProto.Builder builder = BlockWithLocationsProto
-         .newBuilder().setBlock(convert(blk.getBlock()))
++        .newBuilder().setBlock(PBHelperClient.convert(blk.getBlock()))
          .addAllDatanodeUuids(Arrays.asList(blk.getDatanodeUuids()))
          .addAllStorageUuids(Arrays.asList(blk.getStorageIDs()))
 -        .addAllStorageTypes(PBHelperClient.convertStorageTypes(blk.getStorageTypes()))
 -        .build();
 +        .addAllStorageTypes(PBHelperClient.convertStorageTypes(blk.getStorageTypes()));
 +    if (blk instanceof StripedBlockWithLocations) {
 +      StripedBlockWithLocations sblk = (StripedBlockWithLocations) blk;
-       builder.setIndices(getByteString(sblk.getIndices()));
++      builder.setIndices(PBHelperClient.getByteString(sblk.getIndices()));
 +      builder.setDataBlockNum(sblk.getDataBlockNum());
 +    }
 +    return builder.build();
    }
  
    public static BlockWithLocations convert(BlockWithLocationsProto b) {
      final List<String> datanodeUuids = b.getDatanodeUuidsList();
      final List<String> storageUuids = b.getStorageUuidsList();
      final List<StorageTypeProto> storageTypes = b.getStorageTypesList();
-     BlockWithLocations blk = new BlockWithLocations(convert(b.getBlock()),
 -    return new BlockWithLocations(PBHelperClient.convert(b.getBlock()),
++    BlockWithLocations blk = new BlockWithLocations(PBHelperClient.
++        convert(b.getBlock()),
          datanodeUuids.toArray(new String[datanodeUuids.size()]),
          storageUuids.toArray(new String[storageUuids.size()]),
-         convertStorageTypes(storageTypes, storageUuids.size()));
+         PBHelperClient.convertStorageTypes(storageTypes, storageUuids.size()));
 +    if (b.hasIndices()) {
 +      blk = new StripedBlockWithLocations(blk, b.getIndices().toByteArray(),
 +          (short) b.getDataBlockNum());
 +    }
 +    return blk;
    }
  
    public static BlocksWithLocationsProto convert(BlocksWithLocations blks) {
@@@ -596,7 -333,7 +358,7 @@@
      if (b == null) {
        return null;
      }
-     LocatedBlockProto lb = PBHelper.convertLocatedBlock(b);
 -    LocatedBlockProto lb = PBHelperClient.convert((LocatedBlock) b);
++    LocatedBlockProto lb = PBHelperClient.convertLocatedBlock(b);
      RecoveringBlockProto.Builder builder = RecoveringBlockProto.newBuilder();
      builder.setBlock(lb).setNewGenStamp(b.getNewGenerationStamp());
      if(b.getNewBlock() != null)
@@@ -606,206 -343,12 +368,12 @@@
  
    public static RecoveringBlock convert(RecoveringBlockProto b) {
      ExtendedBlock block = PBHelperClient.convert(b.getBlock().getB());
-     DatanodeInfo[] locs = convert(b.getBlock().getLocsList());
+     DatanodeInfo[] locs = PBHelperClient.convert(b.getBlock().getLocsList());
      return (b.hasTruncateBlock()) ?
-         new RecoveringBlock(block, locs, PBHelper.convert(b.getTruncateBlock())) :
+         new RecoveringBlock(block, locs, PBHelperClient.convert(b.getTruncateBlock())) :
          new RecoveringBlock(block, locs, b.getNewGenStamp());
    }
-   
-   static public DatanodeInfo convert(DatanodeInfoProto di) {
-     if (di == null) return null;
-     return new DatanodeInfo(
-         PBHelper.convert(di.getId()),
-         di.hasLocation() ? di.getLocation() : null , 
-         di.getCapacity(),  di.getDfsUsed(),  di.getRemaining(),
-         di.getBlockPoolUsed(), di.getCacheCapacity(), di.getCacheUsed(),
-         di.getLastUpdate(), di.getLastUpdateMonotonic(),
-         di.getXceiverCount(), PBHelper.convert(di.getAdminState()));
-   }
-   
-   static public DatanodeInfo[] convert(DatanodeInfoProto di[]) {
-     if (di == null) return null;
-     DatanodeInfo[] result = new DatanodeInfo[di.length];
-     for (int i = 0; i < di.length; i++) {
-       result[i] = convert(di[i]);
-     }    
-     return result;
-   }
- 
-   public static DatanodeInfo[] convert(List<DatanodeInfoProto> list) {
-     DatanodeInfo[] info = new DatanodeInfo[list.size()];
-     for (int i = 0; i < info.length; i++) {
-       info[i] = convert(list.get(i));
-     }
-     return info;
-   }
- 
-   public static DatanodeStorageReportProto convertDatanodeStorageReport(
-       DatanodeStorageReport report) {
-     return DatanodeStorageReportProto.newBuilder()
-         .setDatanodeInfo(PBHelperClient.convert(report.getDatanodeInfo()))
-         .addAllStorageReports(convertStorageReports(report.getStorageReports()))
-         .build();
-   }
- 
-   public static List<DatanodeStorageReportProto> convertDatanodeStorageReports(
-       DatanodeStorageReport[] reports) {
-     final List<DatanodeStorageReportProto> protos
-         = new ArrayList<DatanodeStorageReportProto>(reports.length);
-     for(int i = 0; i < reports.length; i++) {
-       protos.add(convertDatanodeStorageReport(reports[i]));
-     }
-     return protos;
-   }
- 
-   public static DatanodeStorageReport convertDatanodeStorageReport(
-       DatanodeStorageReportProto proto) {
-     return new DatanodeStorageReport(
-         convert(proto.getDatanodeInfo()),
-         convertStorageReports(proto.getStorageReportsList()));
-   }
- 
-   public static DatanodeStorageReport[] convertDatanodeStorageReports(
-       List<DatanodeStorageReportProto> protos) {
-     final DatanodeStorageReport[] reports
-         = new DatanodeStorageReport[protos.size()];
-     for(int i = 0; i < reports.length; i++) {
-       reports[i] = convertDatanodeStorageReport(protos.get(i));
-     }
-     return reports;
-   }
- 
-   public static AdminStates convert(AdminState adminState) {
-     switch(adminState) {
-     case DECOMMISSION_INPROGRESS:
-       return AdminStates.DECOMMISSION_INPROGRESS;
-     case DECOMMISSIONED:
-       return AdminStates.DECOMMISSIONED;
-     case NORMAL:
-     default:
-       return AdminStates.NORMAL;
-     }
-   }
-   
-   public static LocatedBlockProto convertLocatedBlock(LocatedBlock b) {
-     if (b == null) return null;
-     Builder builder = LocatedBlockProto.newBuilder();
-     DatanodeInfo[] locs = b.getLocations();
-     List<DatanodeInfo> cachedLocs =
-         Lists.newLinkedList(Arrays.asList(b.getCachedLocations()));
-     for (int i = 0; i < locs.length; i++) {
-       DatanodeInfo loc = locs[i];
-       builder.addLocs(i, PBHelperClient.convert(loc));
-       boolean locIsCached = cachedLocs.contains(loc);
-       builder.addIsCached(locIsCached);
-       if (locIsCached) {
-         cachedLocs.remove(loc);
-       }
-     }
-     Preconditions.checkArgument(cachedLocs.size() == 0,
-         "Found additional cached replica locations that are not in the set of"
-         + " storage-backed locations!");
- 
-     StorageType[] storageTypes = b.getStorageTypes();
-     if (storageTypes != null) {
-       for (StorageType storageType : storageTypes) {
-         builder.addStorageTypes(PBHelperClient.convertStorageType(storageType));
-       }
-     }
-     final String[] storageIDs = b.getStorageIDs();
-     if (storageIDs != null) {
-       builder.addAllStorageIDs(Arrays.asList(storageIDs));
-     }
-     if (b instanceof LocatedStripedBlock) {
-       LocatedStripedBlock sb = (LocatedStripedBlock) b;
-       int[] indices = sb.getBlockIndices();
-       Token<BlockTokenIdentifier>[] blockTokens = sb.getBlockTokens();
-       for (int i = 0; i < indices.length; i++) {
-         builder.addBlockIndex(indices[i]);
-         builder.addBlockTokens(PBHelperClient.convert(blockTokens[i]));
-       }
-     }
- 
-     return builder.setB(PBHelperClient.convert(b.getBlock()))
-         .setBlockToken(PBHelperClient.convert(b.getBlockToken()))
-         .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build();
-   }
-   
-   public static LocatedBlock convertLocatedBlockProto(LocatedBlockProto proto) {
-     if (proto == null) return null;
-     List<DatanodeInfoProto> locs = proto.getLocsList();
-     DatanodeInfo[] targets = new DatanodeInfo[locs.size()];
-     for (int i = 0; i < locs.size(); i++) {
-       targets[i] = PBHelper.convert(locs.get(i));
-     }
- 
-     final StorageType[] storageTypes = convertStorageTypes(
-         proto.getStorageTypesList(), locs.size());
- 
-     final int storageIDsCount = proto.getStorageIDsCount();
-     final String[] storageIDs;
-     if (storageIDsCount == 0) {
-       storageIDs = null;
-     } else {
-       Preconditions.checkState(storageIDsCount == locs.size());
-       storageIDs = proto.getStorageIDsList().toArray(new String[storageIDsCount]);
-     }
- 
-     int[] indices = null;
-     final int indexCount = proto.getBlockIndexCount();
-     if (indexCount > 0) {
-       indices = new int[indexCount];
-       for (int i = 0; i < indexCount; i++) {
-         indices[i] = proto.getBlockIndex(i);
-       }
-     }
- 
-     // Set values from the isCached list, re-using references from loc
-     List<DatanodeInfo> cachedLocs = new ArrayList<DatanodeInfo>(locs.size());
-     List<Boolean> isCachedList = proto.getIsCachedList();
-     for (int i=0; i<isCachedList.size(); i++) {
-       if (isCachedList.get(i)) {
-         cachedLocs.add(targets[i]);
-       }
-     }
- 
-     final LocatedBlock lb;
-     if (indices == null) {
-       lb = new LocatedBlock(PBHelperClient.convert(proto.getB()), targets,
-           storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(),
-           cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()]));
-     } else {
-       lb = new LocatedStripedBlock(PBHelperClient.convert(proto.getB()), targets,
-           storageIDs, storageTypes, indices, proto.getOffset(),
-           proto.getCorrupt(),
-           cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()]));
-       List<TokenProto> tokenProtos = proto.getBlockTokensList();
-       Token<BlockTokenIdentifier>[] blockTokens = new Token[indices.length];
-       for (int i = 0; i < indices.length; i++) {
-         blockTokens[i] = PBHelper.convert(tokenProtos.get(i));
-       }
-       ((LocatedStripedBlock) lb).setBlockTokens(blockTokens);
-     }
-     lb.setBlockToken(PBHelper.convert(proto.getBlockToken()));
- 
-     return lb;
-   }
- 
-   public static Token<BlockTokenIdentifier> convert(
-       TokenProto blockToken) {
-     return new Token<BlockTokenIdentifier>(blockToken.getIdentifier()
-         .toByteArray(), blockToken.getPassword().toByteArray(), new Text(
-         blockToken.getKind()), new Text(blockToken.getService()));
-   }
- 
--  
-   public static Token<DelegationTokenIdentifier> convertDelegationToken(
-       TokenProto blockToken) {
-     return new Token<DelegationTokenIdentifier>(blockToken.getIdentifier()
-         .toByteArray(), blockToken.getPassword().toByteArray(), new Text(
-         blockToken.getKind()), new Text(blockToken.getService()));
-   }
 +
    public static ReplicaState convert(ReplicaStateProto state) {
      switch (state) {
      case RBW:
@@@ -1198,549 -704,8 +736,7 @@@
          .setCapabilities(info.getCapabilities())
          .build();
    }
-   
-   // Located Block Arrays and Lists
-   public static LocatedBlockProto[] convertLocatedBlocks(LocatedBlock[] lb) {
-     if (lb == null) return null;
-     return convertLocatedBlocks2(Arrays.asList(lb))
-         .toArray(new LocatedBlockProto[lb.length]);
-   }
-   
-   public static LocatedBlock[] convertLocatedBlocks(LocatedBlockProto[] lb) {
-     if (lb == null) return null;
-     return convertLocatedBlocks(Arrays.asList(lb))
-         .toArray(new LocatedBlock[lb.length]);
-   }
-   
-   public static List<LocatedBlock> convertLocatedBlocks(
-       List<LocatedBlockProto> lb) {
-     if (lb == null) return null;
-     final int len = lb.size();
-     List<LocatedBlock> result = new ArrayList<>(len);
-     for (LocatedBlockProto aLb : lb) {
-       result.add(PBHelper.convertLocatedBlockProto(aLb));
-     }
-     return result;
-   }
-   
-   public static List<LocatedBlockProto> convertLocatedBlocks2(
-       List<LocatedBlock> lb) {
-     if (lb == null) return null;
-     final int len = lb.size();
-     List<LocatedBlockProto> result = new ArrayList<>(len);
-     for (LocatedBlock aLb : lb) {
-       result.add(PBHelper.convertLocatedBlock(aLb));
-     }
-     return result;
-   }
-   
-   
-   // LocatedBlocks
-   public static LocatedBlocks convert(LocatedBlocksProto lb) {
-     return new LocatedBlocks(
-         lb.getFileLength(), lb.getUnderConstruction(),
-         PBHelper.convertLocatedBlocks(lb.getBlocksList()),
-         lb.hasLastBlock() ?
-             PBHelper.convertLocatedBlockProto(lb.getLastBlock()) : null,
-         lb.getIsLastBlockComplete(),
-         lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) : null,
-         lb.hasEcPolicy() ? convertErasureCodingPolicy(lb.getEcPolicy()) : null);
-   }
-   
-   public static LocatedBlocksProto convert(LocatedBlocks lb) {
-     if (lb == null) {
-       return null;
-     }
-     LocatedBlocksProto.Builder builder = 
-         LocatedBlocksProto.newBuilder();
-     if (lb.getLastLocatedBlock() != null) {
-       builder.setLastBlock(
-           PBHelper.convertLocatedBlock(lb.getLastLocatedBlock()));
-     }
-     if (lb.getFileEncryptionInfo() != null) {
-       builder.setFileEncryptionInfo(convert(lb.getFileEncryptionInfo()));
-     }
-     if (lb.getErasureCodingPolicy() != null) {
-       builder.setEcPolicy(convertErasureCodingPolicy(lb.getErasureCodingPolicy()));
-     }
-     return builder.setFileLength(lb.getFileLength())
-         .setUnderConstruction(lb.isUnderConstruction())
-         .addAllBlocks(PBHelper.convertLocatedBlocks2(lb.getLocatedBlocks()))
-         .setIsLastBlockComplete(lb.isLastBlockComplete()).build();
-   }
-   
-   // DataEncryptionKey
-   public static DataEncryptionKey convert(DataEncryptionKeyProto bet) {
-     String encryptionAlgorithm = bet.getEncryptionAlgorithm();
-     return new DataEncryptionKey(bet.getKeyId(),
-         bet.getBlockPoolId(),
-         bet.getNonce().toByteArray(),
-         bet.getEncryptionKey().toByteArray(),
-         bet.getExpiryDate(),
-         encryptionAlgorithm.isEmpty() ? null : encryptionAlgorithm);
-   }
-   
-   public static DataEncryptionKeyProto convert(DataEncryptionKey bet) {
-     DataEncryptionKeyProto.Builder b = DataEncryptionKeyProto.newBuilder()
-         .setKeyId(bet.keyId)
-         .setBlockPoolId(bet.blockPoolId)
-         .setNonce(ByteString.copyFrom(bet.nonce))
-         .setEncryptionKey(ByteString.copyFrom(bet.encryptionKey))
-         .setExpiryDate(bet.expiryDate);
-     if (bet.encryptionAlgorithm != null) {
-       b.setEncryptionAlgorithm(bet.encryptionAlgorithm);
-     }
-     return b.build();
-   }
-   
-   public static FsServerDefaults convert(FsServerDefaultsProto fs) {
-     if (fs == null) return null;
-     return new FsServerDefaults(
-         fs.getBlockSize(), fs.getBytesPerChecksum(), 
-         fs.getWritePacketSize(), (short) fs.getReplication(),
-         fs.getFileBufferSize(),
-         fs.getEncryptDataTransfer(),
-         fs.getTrashInterval(),
-         PBHelperClient.convert(fs.getChecksumType()));
-   }
-   
-   public static FsServerDefaultsProto convert(FsServerDefaults fs) {
-     if (fs == null) return null;
-     return FsServerDefaultsProto.newBuilder().
-       setBlockSize(fs.getBlockSize()).
-       setBytesPerChecksum(fs.getBytesPerChecksum()).
-       setWritePacketSize(fs.getWritePacketSize())
-       .setReplication(fs.getReplication())
-       .setFileBufferSize(fs.getFileBufferSize())
-       .setEncryptDataTransfer(fs.getEncryptDataTransfer())
-       .setTrashInterval(fs.getTrashInterval())
-       .setChecksumType(PBHelperClient.convert(fs.getChecksumType()))
-       .build();
-   }
-   
-   public static FsPermissionProto convert(FsPermission p) {
-     return FsPermissionProto.newBuilder().setPerm(p.toExtendedShort()).build();
-   }
-   
-   public static FsPermission convert(FsPermissionProto p) {
-     return new FsPermissionExtension((short)p.getPerm());
-   }
-   
-   
-   // The creatFlag field in PB is a bitmask whose values are the same a the 
-   // emum values of CreateFlag
-   public static int convertCreateFlag(EnumSetWritable<CreateFlag> flag) {
-     int value = 0;
-     if (flag.contains(CreateFlag.APPEND)) {
-       value |= CreateFlagProto.APPEND.getNumber();
-     }
-     if (flag.contains(CreateFlag.CREATE)) {
-       value |= CreateFlagProto.CREATE.getNumber();
-     }
-     if (flag.contains(CreateFlag.OVERWRITE)) {
-       value |= CreateFlagProto.OVERWRITE.getNumber();
-     }
-     if (flag.contains(CreateFlag.LAZY_PERSIST)) {
-       value |= CreateFlagProto.LAZY_PERSIST.getNumber();
-     }
-     if (flag.contains(CreateFlag.NEW_BLOCK)) {
-       value |= CreateFlagProto.NEW_BLOCK.getNumber();
-     }
-     return value;
-   }
-   
-   public static EnumSetWritable<CreateFlag> convertCreateFlag(int flag) {
-     EnumSet<CreateFlag> result = 
-        EnumSet.noneOf(CreateFlag.class);   
-     if ((flag & CreateFlagProto.APPEND_VALUE) == CreateFlagProto.APPEND_VALUE) {
-       result.add(CreateFlag.APPEND);
-     }
-     if ((flag & CreateFlagProto.CREATE_VALUE) == CreateFlagProto.CREATE_VALUE) {
-       result.add(CreateFlag.CREATE);
-     }
-     if ((flag & CreateFlagProto.OVERWRITE_VALUE) 
-         == CreateFlagProto.OVERWRITE_VALUE) {
-       result.add(CreateFlag.OVERWRITE);
-     }
-     if ((flag & CreateFlagProto.LAZY_PERSIST_VALUE)
-         == CreateFlagProto.LAZY_PERSIST_VALUE) {
-       result.add(CreateFlag.LAZY_PERSIST);
-     }
-     if ((flag & CreateFlagProto.NEW_BLOCK_VALUE)
-         == CreateFlagProto.NEW_BLOCK_VALUE) {
-       result.add(CreateFlag.NEW_BLOCK);
-     }
-     return new EnumSetWritable<CreateFlag>(result, CreateFlag.class);
-   }
- 
-   public static int convertCacheFlags(EnumSet<CacheFlag> flags) {
-     int value = 0;
-     if (flags.contains(CacheFlag.FORCE)) {
-       value |= CacheFlagProto.FORCE.getNumber();
-     }
-     return value;
-   }
- 
-   public static EnumSet<CacheFlag> convertCacheFlags(int flags) {
-     EnumSet<CacheFlag> result = EnumSet.noneOf(CacheFlag.class);
-     if ((flags & CacheFlagProto.FORCE_VALUE) == CacheFlagProto.FORCE_VALUE) {
-       result.add(CacheFlag.FORCE);
-     }
-     return result;
-   }
- 
-   public static HdfsFileStatus convert(HdfsFileStatusProto fs) {
-     if (fs == null)
-       return null;
-     return new HdfsLocatedFileStatus(
-         fs.getLength(), fs.getFileType().equals(FileType.IS_DIR), 
-         fs.getBlockReplication(), fs.getBlocksize(),
-         fs.getModificationTime(), fs.getAccessTime(),
-         PBHelper.convert(fs.getPermission()), fs.getOwner(), fs.getGroup(), 
-         fs.getFileType().equals(FileType.IS_SYMLINK) ? 
-             fs.getSymlink().toByteArray() : null,
-         fs.getPath().toByteArray(),
-         fs.hasFileId()? fs.getFileId(): HdfsConstants.GRANDFATHER_INODE_ID,
-         fs.hasLocations() ? PBHelper.convert(fs.getLocations()) : null,
-         fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
-         fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,
-         fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy()
-             : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
-         fs.hasEcPolicy() ? PBHelper.convertErasureCodingPolicy(fs.getEcPolicy()) : null);
-   }
- 
-   public static SnapshottableDirectoryStatus convert(
-       SnapshottableDirectoryStatusProto sdirStatusProto) {
-     if (sdirStatusProto == null) {
-       return null;
-     }
-     final HdfsFileStatusProto status = sdirStatusProto.getDirStatus();
-     return new SnapshottableDirectoryStatus(
-         status.getModificationTime(),
-         status.getAccessTime(),
-         PBHelper.convert(status.getPermission()),
-         status.getOwner(),
-         status.getGroup(),
-         status.getPath().toByteArray(),
-         status.getFileId(),
-         status.getChildrenNum(),
-         sdirStatusProto.getSnapshotNumber(),
-         sdirStatusProto.getSnapshotQuota(),
-         sdirStatusProto.getParentFullpath().toByteArray());
-   }
-   
-   public static HdfsFileStatusProto convert(HdfsFileStatus fs) {
-     if (fs == null)
-       return null;
-     FileType fType = FileType.IS_FILE;
-     if (fs.isDir()) {
-       fType = FileType.IS_DIR;
-     } else if (fs.isSymlink()) {
-       fType = FileType.IS_SYMLINK;
-     }
- 
-     HdfsFileStatusProto.Builder builder = 
-      HdfsFileStatusProto.newBuilder().
-       setLength(fs.getLen()).
-       setFileType(fType).
-       setBlockReplication(fs.getReplication()).
-       setBlocksize(fs.getBlockSize()).
-       setModificationTime(fs.getModificationTime()).
-       setAccessTime(fs.getAccessTime()).
-       setPermission(PBHelper.convert(fs.getPermission())).
-       setOwner(fs.getOwner()).
-       setGroup(fs.getGroup()).
-       setFileId(fs.getFileId()).
-       setChildrenNum(fs.getChildrenNum()).
-       setPath(ByteString.copyFrom(fs.getLocalNameInBytes())).
-       setStoragePolicy(fs.getStoragePolicy());
-     if (fs.isSymlink())  {
-       builder.setSymlink(ByteString.copyFrom(fs.getSymlinkInBytes()));
-     }
-     if (fs.getFileEncryptionInfo() != null) {
-       builder.setFileEncryptionInfo(convert(fs.getFileEncryptionInfo()));
-     }
-     if (fs instanceof HdfsLocatedFileStatus) {
-       final HdfsLocatedFileStatus lfs = (HdfsLocatedFileStatus) fs;
-       LocatedBlocks locations = lfs.getBlockLocations();
-       if (locations != null) {
-         builder.setLocations(PBHelper.convert(locations));
-       }
-     }
-     if(fs.getErasureCodingPolicy() != null) {
-       builder.setEcPolicy(PBHelper.convertErasureCodingPolicy(fs.getErasureCodingPolicy()));
-     }
-     return builder.build();
-   }
-   
-   public static SnapshottableDirectoryStatusProto convert(
-       SnapshottableDirectoryStatus status) {
-     if (status == null) {
-       return null;
-     }
-     int snapshotNumber = status.getSnapshotNumber();
-     int snapshotQuota = status.getSnapshotQuota();
-     byte[] parentFullPath = status.getParentFullPath();
-     ByteString parentFullPathBytes = ByteString.copyFrom(
-         parentFullPath == null ? DFSUtilClient.EMPTY_BYTES : parentFullPath);
-     HdfsFileStatusProto fs = convert(status.getDirStatus());
-     SnapshottableDirectoryStatusProto.Builder builder = 
-         SnapshottableDirectoryStatusProto
-         .newBuilder().setSnapshotNumber(snapshotNumber)
-         .setSnapshotQuota(snapshotQuota).setParentFullpath(parentFullPathBytes)
-         .setDirStatus(fs);
-     return builder.build();
-   }
-   
-   public static HdfsFileStatusProto[] convert(HdfsFileStatus[] fs) {
-     if (fs == null) return null;
-     final int len = fs.length;
-     HdfsFileStatusProto[] result = new HdfsFileStatusProto[len];
-     for (int i = 0; i < len; ++i) {
-       result[i] = PBHelper.convert(fs[i]);
-     }
-     return result;
-   }
-   
-   public static HdfsFileStatus[] convert(HdfsFileStatusProto[] fs) {
-     if (fs == null) return null;
-     final int len = fs.length;
-     HdfsFileStatus[] result = new HdfsFileStatus[len];
-     for (int i = 0; i < len; ++i) {
-       result[i] = PBHelper.convert(fs[i]);
-     }
-     return result;
-   }
-   
-   public static DirectoryListing convert(DirectoryListingProto dl) {
-     if (dl == null)
-       return null;
-     List<HdfsFileStatusProto> partList =  dl.getPartialListingList();
-     return new DirectoryListing( 
-         partList.isEmpty() ? new HdfsLocatedFileStatus[0] 
-           : PBHelper.convert(
-               partList.toArray(new HdfsFileStatusProto[partList.size()])),
-         dl.getRemainingEntries());
-   }
- 
-   public static DirectoryListingProto convert(DirectoryListing d) {
-     if (d == null)
-       return null;
-     return DirectoryListingProto.newBuilder().
-         addAllPartialListing(Arrays.asList(
-             PBHelper.convert(d.getPartialListing()))).
-         setRemainingEntries(d.getRemainingEntries()).
-         build();
-   }
- 
-   public static long[] convert(GetFsStatsResponseProto res) {
-     long[] result = new long[7];
-     result[ClientProtocol.GET_STATS_CAPACITY_IDX] = res.getCapacity();
-     result[ClientProtocol.GET_STATS_USED_IDX] = res.getUsed();
-     result[ClientProtocol.GET_STATS_REMAINING_IDX] = res.getRemaining();
-     result[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX] = res.getUnderReplicated();
-     result[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX] = res.getCorruptBlocks();
-     result[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX] = res.getMissingBlocks();
-     result[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX] =
-         res.getMissingReplOneBlocks();
-     return result;
-   }
-   
-   public static GetFsStatsResponseProto convert(long[] fsStats) {
-     GetFsStatsResponseProto.Builder result = GetFsStatsResponseProto
-         .newBuilder();
-     if (fsStats.length >= ClientProtocol.GET_STATS_CAPACITY_IDX + 1)
-       result.setCapacity(fsStats[ClientProtocol.GET_STATS_CAPACITY_IDX]);
-     if (fsStats.length >= ClientProtocol.GET_STATS_USED_IDX + 1)
-       result.setUsed(fsStats[ClientProtocol.GET_STATS_USED_IDX]);
-     if (fsStats.length >= ClientProtocol.GET_STATS_REMAINING_IDX + 1)
-       result.setRemaining(fsStats[ClientProtocol.GET_STATS_REMAINING_IDX]);
-     if (fsStats.length >= ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX + 1)
-       result.setUnderReplicated(
-               fsStats[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX]);
-     if (fsStats.length >= ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX + 1)
-       result.setCorruptBlocks(
-           fsStats[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX]);
-     if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX + 1)
-       result.setMissingBlocks(
-           fsStats[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX]);
-     if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX + 1)
-       result.setMissingReplOneBlocks(
-           fsStats[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX]);
-     return result.build();
-   }
-   
-   public static DatanodeReportTypeProto
-     convert(DatanodeReportType t) {
-     switch (t) {
-     case ALL: return DatanodeReportTypeProto.ALL;
-     case LIVE: return DatanodeReportTypeProto.LIVE;
-     case DEAD: return DatanodeReportTypeProto.DEAD;
-     case DECOMMISSIONING: return DatanodeReportTypeProto.DECOMMISSIONING;
-     default: 
-       throw new IllegalArgumentException("Unexpected data type report:" + t);
-     }
-   }
-   
-   public static DatanodeReportType 
-     convert(DatanodeReportTypeProto t) {
-     switch (t) {
-     case ALL: return DatanodeReportType.ALL;
-     case LIVE: return DatanodeReportType.LIVE;
-     case DEAD: return DatanodeReportType.DEAD;
-     case DECOMMISSIONING: return DatanodeReportType.DECOMMISSIONING;
-     default: 
-       throw new IllegalArgumentException("Unexpected data type report:" + t);
-     }
-   }
- 
-   public static SafeModeActionProto convert(
-       SafeModeAction a) {
-     switch (a) {
-     case SAFEMODE_LEAVE:
-       return SafeModeActionProto.SAFEMODE_LEAVE;
-     case SAFEMODE_ENTER:
-       return SafeModeActionProto.SAFEMODE_ENTER;
-     case SAFEMODE_GET:
-       return SafeModeActionProto.SAFEMODE_GET;
-     default:
-       throw new IllegalArgumentException("Unexpected SafeModeAction :" + a);
-     }
-   }
-   
-   public static SafeModeAction convert(
-       ClientNamenodeProtocolProtos.SafeModeActionProto a) {
-     switch (a) {
-     case SAFEMODE_LEAVE:
-       return SafeModeAction.SAFEMODE_LEAVE;
-     case SAFEMODE_ENTER:
-       return SafeModeAction.SAFEMODE_ENTER;
-     case SAFEMODE_GET:
-       return SafeModeAction.SAFEMODE_GET;
-     default:
-       throw new IllegalArgumentException("Unexpected SafeModeAction :" + a);
-     }
-   }
-   
-   public static RollingUpgradeActionProto convert(RollingUpgradeAction a) {
-     switch (a) {
-     case QUERY:
-       return RollingUpgradeActionProto.QUERY;
-     case PREPARE:
-       return RollingUpgradeActionProto.START;
-     case FINALIZE:
-       return RollingUpgradeActionProto.FINALIZE;
-     default:
-       throw new IllegalArgumentException("Unexpected value: " + a);
-     }
-   }
-   
-   public static RollingUpgradeAction convert(RollingUpgradeActionProto a) {
-     switch (a) {
-     case QUERY:
-       return RollingUpgradeAction.QUERY;
-     case START:
-       return RollingUpgradeAction.PREPARE;
-     case FINALIZE:
-       return RollingUpgradeAction.FINALIZE;
-     default:
-       throw new IllegalArgumentException("Unexpected value: " + a);
-     }
-   }
- 
-   public static RollingUpgradeStatusProto convertRollingUpgradeStatus(
-       RollingUpgradeStatus status) {
-     return RollingUpgradeStatusProto.newBuilder()
-         .setBlockPoolId(status.getBlockPoolId())
-         .setFinalized(status.isFinalized())
-         .build();
-   }
- 
-   public static RollingUpgradeStatus convert(RollingUpgradeStatusProto proto) {
-     return new RollingUpgradeStatus(proto.getBlockPoolId(),
-         proto.getFinalized());
-   }
- 
-   public static RollingUpgradeInfoProto convert(RollingUpgradeInfo info) {
-     return RollingUpgradeInfoProto.newBuilder()
-         .setStatus(convertRollingUpgradeStatus(info))
-         .setCreatedRollbackImages(info.createdRollbackImages())
-         .setStartTime(info.getStartTime())
-         .setFinalizeTime(info.getFinalizeTime())
-         .build();
-   }
- 
-   public static RollingUpgradeInfo convert(RollingUpgradeInfoProto proto) {
-     RollingUpgradeStatusProto status = proto.getStatus();
-     return new RollingUpgradeInfo(status.getBlockPoolId(),
-         proto.getCreatedRollbackImages(),
-         proto.getStartTime(), proto.getFinalizeTime());
-   }
- 
-   public static CorruptFileBlocks convert(CorruptFileBlocksProto c) {
-     if (c == null)
-       return null;
-     List<String> fileList = c.getFilesList();
-     return new CorruptFileBlocks(fileList.toArray(new String[fileList.size()]),
-         c.getCookie());
-   }
- 
-   public static CorruptFileBlocksProto convert(CorruptFileBlocks c) {
-     if (c == null)
-       return null;
-     return CorruptFileBlocksProto.newBuilder().
-         addAllFiles(Arrays.asList(c.getFiles())).
-         setCookie(c.getCookie()).
-         build();
-   }
-   
-   public static ContentSummary convert(ContentSummaryProto cs) {
-     if (cs == null) return null;
-     ContentSummary.Builder builder = new ContentSummary.Builder();
-     builder.length(cs.getLength()).
-         fileCount(cs.getFileCount()).
-         directoryCount(cs.getDirectoryCount()).
-         quota(cs.getQuota()).
-         spaceConsumed(cs.getSpaceConsumed()).
-         spaceQuota(cs.getSpaceQuota());
-     if (cs.hasTypeQuotaInfos()) {
-       for (HdfsProtos.StorageTypeQuotaInfoProto info :
-           cs.getTypeQuotaInfos().getTypeQuotaInfoList()) {
-         StorageType type = PBHelperClient.convertStorageType(info.getType());
-         builder.typeConsumed(type, info.getConsumed());
-         builder.typeQuota(type, info.getQuota());
-       }
-     }
-     return builder.build();
-   }
-   
-   public static ContentSummaryProto convert(ContentSummary cs) {
-     if (cs == null) return null;
-     ContentSummaryProto.Builder builder = ContentSummaryProto.newBuilder();
-         builder.setLength(cs.getLength()).
-         setFileCount(cs.getFileCount()).
-         setDirectoryCount(cs.getDirectoryCount()).
-         setQuota(cs.getQuota()).
-         setSpaceConsumed(cs.getSpaceConsumed()).
-         setSpaceQuota(cs.getSpaceQuota());
- 
-     if (cs.isTypeQuotaSet() || cs.isTypeConsumedAvailable()) {
-       HdfsProtos.StorageTypeQuotaInfosProto.Builder isb =
-           HdfsProtos.StorageTypeQuotaInfosProto.newBuilder();
-       for (StorageType t: StorageType.getTypesSupportingQuota()) {
-         HdfsProtos.StorageTypeQuotaInfoProto info =
-             HdfsProtos.StorageTypeQuotaInfoProto.newBuilder().
-                 setType(PBHelperClient.convertStorageType(t)).
-                 setConsumed(cs.getTypeConsumed(t)).
-                 setQuota(cs.getTypeQuota(t)).
-                 build();
-         isb.addTypeQuotaInfo(info);
-       }
-       builder.setTypeQuotaInfos(isb);
-     }
-     return builder.build();
-   }
  
 -
    public static NNHAStatusHeartbeat convert(NNHAStatusHeartbeatProto s) {
      if (s == null) return null;
      switch (s.getState()) {
@@@ -2933,179 -790,4 +821,140 @@@
          setLeaseId(context.getLeaseId()).
          build();
    }
 +
-   public static ECSchema convertECSchema(ECSchemaProto schema) {
-     List<ECSchemaOptionEntryProto> optionsList = schema.getOptionsList();
-     Map<String, String> options = new HashMap<>(optionsList.size());
-     for (ECSchemaOptionEntryProto option : optionsList) {
-       options.put(option.getKey(), option.getValue());
++  private static List<Integer> convertIntArray(short[] liveBlockIndices) {
++    List<Integer> liveBlockIndicesList = new ArrayList<>();
++    for (short s : liveBlockIndices) {
++      liveBlockIndicesList.add((int) s);
 +    }
-     return new ECSchema(schema.getCodecName(), schema.getDataUnits(),
-         schema.getParityUnits(), options);
-   }
- 
-   public static ECSchemaProto convertECSchema(ECSchema schema) {
-     ECSchemaProto.Builder builder = ECSchemaProto.newBuilder()
-         .setCodecName(schema.getCodecName())
-         .setDataUnits(schema.getNumDataUnits())
-         .setParityUnits(schema.getNumParityUnits());
-     Set<Entry<String, String>> entrySet = schema.getExtraOptions().entrySet();
-     for (Entry<String, String> entry : entrySet) {
-       builder.addOptions(ECSchemaOptionEntryProto.newBuilder()
-           .setKey(entry.getKey()).setValue(entry.getValue()).build());
++    return liveBlockIndicesList;
++  }
++
++  private static StorageTypesProto convertStorageTypesProto(
++      StorageType[] targetStorageTypes) {
++    StorageTypesProto.Builder builder = StorageTypesProto.newBuilder();
++    for (StorageType storageType : targetStorageTypes) {
++      builder.addStorageTypes(PBHelperClient.convertStorageType(storageType));
 +    }
 +    return builder.build();
 +  }
 +
-   public static ErasureCodingPolicy convertErasureCodingPolicy(
-       ErasureCodingPolicyProto policy) {
-     return new ErasureCodingPolicy(policy.getName(),
-         convertECSchema(policy.getSchema()),
-         policy.getCellSize());
++  private static HdfsProtos.StorageUuidsProto convertStorageIDs(String[] targetStorageIDs) {
++    HdfsProtos.StorageUuidsProto.Builder builder = HdfsProtos.StorageUuidsProto.newBuilder();
++    for (String storageUuid : targetStorageIDs) {
++      builder.addStorageUuids(storageUuid);
++    }
++    return builder.build();
 +  }
 +
-   public static ErasureCodingPolicyProto convertErasureCodingPolicy(
-       ErasureCodingPolicy policy) {
-     ErasureCodingPolicyProto.Builder builder = ErasureCodingPolicyProto
-         .newBuilder()
-         .setName(policy.getName())
-         .setSchema(convertECSchema(policy.getSchema()))
-         .setCellSize(policy.getCellSize());
++  private static DatanodeInfosProto convertToDnInfosProto(DatanodeInfo[] dnInfos) {
++    DatanodeInfosProto.Builder builder = DatanodeInfosProto.newBuilder();
++    for (DatanodeInfo datanodeInfo : dnInfos) {
++      builder.addDatanodes(PBHelperClient.convert(datanodeInfo));
++    }
 +    return builder.build();
 +  }
-   
++
++  private static String[] convert(HdfsProtos.StorageUuidsProto targetStorageUuidsProto) {
++    List<String> storageUuidsList = targetStorageUuidsProto
++        .getStorageUuidsList();
++    String[] storageUuids = new String[storageUuidsList.size()];
++    for (int i = 0; i < storageUuidsList.size(); i++) {
++      storageUuids[i] = storageUuidsList.get(i);
++    }
++    return storageUuids;
++  }
++
 +  public static BlockECRecoveryInfo convertBlockECRecoveryInfo(
 +      BlockECRecoveryInfoProto blockEcRecoveryInfoProto) {
 +    ExtendedBlockProto blockProto = blockEcRecoveryInfoProto.getBlock();
 +    ExtendedBlock block = PBHelperClient.convert(blockProto);
 +
 +    DatanodeInfosProto sourceDnInfosProto = blockEcRecoveryInfoProto
 +        .getSourceDnInfos();
-     DatanodeInfo[] sourceDnInfos = convert(sourceDnInfosProto);
++    DatanodeInfo[] sourceDnInfos = PBHelperClient.convert(sourceDnInfosProto);
 +
 +    DatanodeInfosProto targetDnInfosProto = blockEcRecoveryInfoProto
 +        .getTargetDnInfos();
-     DatanodeInfo[] targetDnInfos = convert(targetDnInfosProto);
++    DatanodeInfo[] targetDnInfos = PBHelperClient.convert(targetDnInfosProto);
 +
-     StorageUuidsProto targetStorageUuidsProto = blockEcRecoveryInfoProto
++    HdfsProtos.StorageUuidsProto targetStorageUuidsProto = blockEcRecoveryInfoProto
 +        .getTargetStorageUuids();
 +    String[] targetStorageUuids = convert(targetStorageUuidsProto);
 +
 +    StorageTypesProto targetStorageTypesProto = blockEcRecoveryInfoProto
 +        .getTargetStorageTypes();
-     StorageType[] convertStorageTypes = convertStorageTypes(
++    StorageType[] convertStorageTypes = PBHelperClient.convertStorageTypes(
 +        targetStorageTypesProto.getStorageTypesList(), targetStorageTypesProto
 +            .getStorageTypesList().size());
 +
 +    List<Integer> liveBlockIndicesList = blockEcRecoveryInfoProto
 +        .getLiveBlockIndicesList();
 +    short[] liveBlkIndices = new short[liveBlockIndicesList.size()];
 +    for (int i = 0; i < liveBlockIndicesList.size(); i++) {
 +      liveBlkIndices[i] = liveBlockIndicesList.get(i).shortValue();
 +    }
 +
 +    ErasureCodingPolicy ecPolicy =
-         convertErasureCodingPolicy(blockEcRecoveryInfoProto.getEcPolicy());
++        PBHelperClient.convertErasureCodingPolicy(
++            blockEcRecoveryInfoProto.getEcPolicy());
 +
 +    return new BlockECRecoveryInfo(block, sourceDnInfos, targetDnInfos,
 +        targetStorageUuids, convertStorageTypes, liveBlkIndices, ecPolicy);
 +  }
 +
 +  public static BlockECRecoveryInfoProto convertBlockECRecoveryInfo(
 +      BlockECRecoveryInfo blockEcRecoveryInfo) {
 +    BlockECRecoveryInfoProto.Builder builder = BlockECRecoveryInfoProto
 +        .newBuilder();
 +    builder.setBlock(PBHelperClient.convert(
 +        blockEcRecoveryInfo.getExtendedBlock()));
 +
 +    DatanodeInfo[] sourceDnInfos = blockEcRecoveryInfo.getSourceDnInfos();
 +    builder.setSourceDnInfos(convertToDnInfosProto(sourceDnInfos));
 +
 +    DatanodeInfo[] targetDnInfos = blockEcRecoveryInfo.getTargetDnInfos();
 +    builder.setTargetDnInfos(convertToDnInfosProto(targetDnInfos));
 +
 +    String[] targetStorageIDs = blockEcRecoveryInfo.getTargetStorageIDs();
 +    builder.setTargetStorageUuids(convertStorageIDs(targetStorageIDs));
 +
 +    StorageType[] targetStorageTypes = blockEcRecoveryInfo
 +        .getTargetStorageTypes();
 +    builder.setTargetStorageTypes(convertStorageTypesProto(targetStorageTypes));
 +
 +    short[] liveBlockIndices = blockEcRecoveryInfo.getLiveBlockIndices();
 +    builder.addAllLiveBlockIndices(convertIntArray(liveBlockIndices));
 +
-     builder.setEcPolicy(convertErasureCodingPolicy(blockEcRecoveryInfo
-         .getErasureCodingPolicy()));
- 
-     return builder.build();
-   }
- 
-   private static List<Integer> convertIntArray(short[] liveBlockIndices) {
-     List<Integer> liveBlockIndicesList = new ArrayList<Integer>();
-     for (short s : liveBlockIndices) {
-       liveBlockIndicesList.add((int) s);
-     }
-     return liveBlockIndicesList;
-   }
++    builder.setEcPolicy(PBHelperClient.convertErasureCodingPolicy(
++        blockEcRecoveryInfo.getErasureCodingPolicy()));
 +
-   private static StorageTypesProto convertStorageTypesProto(
-       StorageType[] targetStorageTypes) {
-     StorageTypesProto.Builder builder = StorageTypesProto.newBuilder();
-     for (StorageType storageType : targetStorageTypes) {
-       builder.addStorageTypes(PBHelperClient.convertStorageType(storageType));
-     }
 +    return builder.build();
 +  }
 +
-   private static StorageUuidsProto convertStorageIDs(String[] targetStorageIDs) {
-     StorageUuidsProto.Builder builder = StorageUuidsProto.newBuilder();
-     for (String storageUuid : targetStorageIDs) {
-       builder.addStorageUuids(storageUuid);
-     }
-     return builder.build();
-   }
- 
-   private static DatanodeInfosProto convertToDnInfosProto(DatanodeInfo[] dnInfos) {
-     DatanodeInfosProto.Builder builder = DatanodeInfosProto.newBuilder();
-     for (DatanodeInfo datanodeInfo : dnInfos) {
-       builder.addDatanodes(PBHelperClient.convert(datanodeInfo));
-     }
-     return builder.build();
-   }
- 
-   private static String[] convert(StorageUuidsProto targetStorageUuidsProto) {
-     List<String> storageUuidsList = targetStorageUuidsProto
-         .getStorageUuidsList();
-     String[] storageUuids = new String[storageUuidsList.size()];
-     for (int i = 0; i < storageUuidsList.size(); i++) {
-       storageUuids[i] = storageUuidsList.get(i);
-     }
-     return storageUuids;
-   }
-   
 +  public static BlockECRecoveryCommandProto convert(
 +      BlockECRecoveryCommand blkECRecoveryCmd) {
 +    BlockECRecoveryCommandProto.Builder builder = BlockECRecoveryCommandProto
 +        .newBuilder();
 +    Collection<BlockECRecoveryInfo> blockECRecoveryInfos = blkECRecoveryCmd
 +        .getECTasks();
 +    for (BlockECRecoveryInfo blkECRecoveryInfo : blockECRecoveryInfos) {
 +      builder
 +          .addBlockECRecoveryinfo(convertBlockECRecoveryInfo(blkECRecoveryInfo));
 +    }
 +    return builder.build();
 +  }
-   
++
 +  public static BlockECRecoveryCommand convert(
 +      BlockECRecoveryCommandProto blkECRecoveryCmdProto) {
-     Collection<BlockECRecoveryInfo> blkECRecoveryInfos = new ArrayList<BlockECRecoveryInfo>();
++    Collection<BlockECRecoveryInfo> blkECRecoveryInfos = new ArrayList<>();
 +    List<BlockECRecoveryInfoProto> blockECRecoveryinfoList = blkECRecoveryCmdProto
 +        .getBlockECRecoveryinfoList();
 +    for (BlockECRecoveryInfoProto blockECRecoveryInfoProto : blockECRecoveryinfoList) {
 +      blkECRecoveryInfos
 +          .add(convertBlockECRecoveryInfo(blockECRecoveryInfoProto));
 +    }
 +    return new BlockECRecoveryCommand(DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY,
 +        blkECRecoveryInfos);
 +  }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 1211169,b0a11fe..e7f9262
--- 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
@@@ -1160,8 -1107,8 +1176,8 @@@ public class BlockManager implements Bl
     * Adds block to list of blocks which will be invalidated on all its
     * datanodes.
     */
 -  private void addToInvalidates(Block b) {
 +  private void addToInvalidates(BlockInfo storedBlock) {
-     if (!namesystem.isPopulatingReplQueues()) {
+     if (!isPopulatingReplQueues()) {
        return;
      }
      StringBuilder datanodes = new StringBuilder();
@@@ -1287,8 -1215,8 +1303,8 @@@
      if (hasEnoughLiveReplicas || hasMoreCorruptReplicas
          || corruptedDuringWrite) {
        // the block is over-replicated so invalidate the replicas immediately
 -      invalidateBlock(b, node);
 +      invalidateBlock(b, node, numberOfReplicas);
-     } else if (namesystem.isPopulatingReplQueues()) {
+     } else if (isPopulatingReplQueues()) {
        // add the block to neededReplication
        updateNeededReplications(b.getStored(), -1, 0);
      }
@@@ -2654,9 -2488,9 +2670,9 @@@
        DatanodeStorageInfo storageInfo)
    throws IOException {
      assert (storedBlock != null && namesystem.hasWriteLock());
--    if (!namesystem.isInStartupSafeMode() 
-         || namesystem.isPopulatingReplQueues()) {
++    if (!namesystem.isInStartupSafeMode()
+         || isPopulatingReplQueues()) {
 -      addStoredBlock(storedBlock, storageInfo, null, false);
 +      addStoredBlock(storedBlock, reported, storageInfo, null, false);
        return;
      }
  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
index a80bfd6,6d199d7..fb86ff3
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
@@@ -542,12 -546,12 +542,12 @@@ public class DecommissionManager 
          if (blockManager.isNeededReplication(block, liveReplicas)) {
            if (!blockManager.neededReplications.contains(block) &&
                blockManager.pendingReplications.getNumReplicas(block) == 0 &&
-               namesystem.isPopulatingReplQueues()) {
+               blockManager.isPopulatingReplQueues()) {
              // Process these blocks only when active NN is out of safe mode.
              blockManager.neededReplications.add(block,
 -                curReplicas,
 +                liveReplicas,
                  num.decommissionedAndDecommissioning(),
 -                block.getReplication());
 +                blockManager.getExpectedReplicaNum(block));
            }
          }
  

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 82a0f62,2aad83d..9228bec
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@@ -1156,8 -1175,7 +1179,9 @@@ public class DataNode extends Reconfigu
      saslClient = new SaslDataTransferClient(dnConf.conf, 
          dnConf.saslPropsResolver, dnConf.trustedChannelResolver);
      saslServer = new SaslDataTransferServer(dnConf, blockPoolTokenSecretManager);
 +    // Initialize ErasureCoding worker
 +    ecWorker = new ErasureCodingWorker(conf, this);
+     startMetricsLogger(conf);
    }
  
    /**
@@@ -3264,9 -3256,72 +3291,76 @@@
      checkSuperuserPrivilege();
      spanReceiverHost.removeSpanReceiver(id);
    }
 +  
 +  public ErasureCodingWorker getErasureCodingWorker(){
 +    return ecWorker;
-     
++  }
+ 
+   /**
+    * Get timeout value of each OOB type from configuration
+    */
+   private void initOOBTimeout() {
+     final int oobStart = Status.OOB_RESTART_VALUE; // the first OOB type
+     final int oobEnd = Status.OOB_RESERVED3_VALUE; // the last OOB type
+     final int numOobTypes = oobEnd - oobStart + 1;
+     oobTimeouts = new long[numOobTypes];
+ 
+     final String[] ele = conf.get(DFS_DATANODE_OOB_TIMEOUT_KEY,
+         DFS_DATANODE_OOB_TIMEOUT_DEFAULT).split(",");
+     for (int i = 0; i < numOobTypes; i++) {
+       oobTimeouts[i] = (i < ele.length) ? Long.parseLong(ele[i]) : 0;
+     }
+   }
+ 
+   /**
+    * Get the timeout to be used for transmitting the OOB type
+    * @return the timeout in milliseconds
+    */
+   public long getOOBTimeout(Status status)
+       throws IOException {
+     if (status.getNumber() < Status.OOB_RESTART_VALUE ||
+         status.getNumber() > Status.OOB_RESERVED3_VALUE) {
+       // Not an OOB.
+       throw new IOException("Not an OOB status: " + status);
+     }
+ 
+     return oobTimeouts[status.getNumber() - Status.OOB_RESTART_VALUE];
+   }
+ 
+   /**
+    * Start a timer to periodically write DataNode metrics to the log file. This
+    * behavior can be disabled by configuration.
+    *
+    * @param metricConf
+    */
+   protected void startMetricsLogger(Configuration metricConf) {
+     long metricsLoggerPeriodSec = metricConf.getInt(
+         DFS_DATANODE_METRICS_LOGGER_PERIOD_SECONDS_KEY,
+         DFS_DATANODE_METRICS_LOGGER_PERIOD_SECONDS_DEFAULT);
+ 
+     if (metricsLoggerPeriodSec <= 0) {
+       return;
+     }
+ 
+     MetricsLoggerTask.makeMetricsLoggerAsync(METRICS_LOG);
+ 
+     // Schedule the periodic logging.
+     metricsLoggerTimer = new ScheduledThreadPoolExecutor(1);
+     metricsLoggerTimer.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
+     metricsLoggerTimer.scheduleWithFixedDelay(new MetricsLoggerTask(METRICS_LOG,
+         "DataNode", (short) 0), metricsLoggerPeriodSec, metricsLoggerPeriodSec,
+         TimeUnit.SECONDS);
+   }
+ 
+   protected void stopMetricsLogger() {
+     if (metricsLoggerTimer != null) {
+       metricsLoggerTimer.shutdown();
+       metricsLoggerTimer = null;
+     }
+   }
+ 
+   @VisibleForTesting
+   ScheduledThreadPoolExecutor getMetricsLoggerTimer() {
+     return metricsLoggerTimer;
    }
  }


[27/50] [abbrv] hadoop git commit: MAPREDUCE-6478. Add an option to skip cleanupJob stage or ignore cleanup failure during commitJob. (Junping Du via wangda)

Posted by zh...@apache.org.
MAPREDUCE-6478. Add an option to skip cleanupJob stage or ignore cleanup failure during commitJob. (Junping Du via wangda)


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

Branch: refs/heads/HDFS-7285
Commit: 372ad270a0d7ea5c581cd9a42b3c3cb189eca204
Parents: 9bc913a
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Sep 18 10:10:32 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Sep 18 10:10:32 2015 -0700

----------------------------------------------------------------------
 .../lib/output/FileOutputCommitter.java         | 52 ++++++++++++++++++--
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 2 files changed, 52 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/372ad270/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java
index 1e5d71a..5eb2384 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java
@@ -64,9 +64,23 @@ public class FileOutputCommitter extends OutputCommitter {
   public static final String FILEOUTPUTCOMMITTER_ALGORITHM_VERSION =
       "mapreduce.fileoutputcommitter.algorithm.version";
   public static final int FILEOUTPUTCOMMITTER_ALGORITHM_VERSION_DEFAULT = 2;
+  // Skip cleanup _temporary folders under job's output directory
+  public static final String FILEOUTPUTCOMMITTER_CLEANUP_SKIPPED =
+      "mapreduce.fileoutputcommitter.cleanup.skipped";
+  public static final boolean
+      FILEOUTPUTCOMMITTER_CLEANUP_SKIPPED_DEFAULT = false;
+
+  // Ignore exceptions in cleanup _temporary folder under job's output directory
+  public static final String FILEOUTPUTCOMMITTER_CLEANUP_FAILURES_IGNORED =
+      "mapreduce.fileoutputcommitter.cleanup-failures.ignored";
+  public static final boolean
+      FILEOUTPUTCOMMITTER_CLEANUP_FAILURES_IGNORED_DEFAULT = false;
+
   private Path outputPath = null;
   private Path workPath = null;
   private final int algorithmVersion;
+  private final boolean skipCleanup;
+  private final boolean ignoreCleanupFailures;
 
   /**
    * Create a file output committer
@@ -101,6 +115,21 @@ public class FileOutputCommitter extends OutputCommitter {
     if (algorithmVersion != 1 && algorithmVersion != 2) {
       throw new IOException("Only 1 or 2 algorithm version is supported");
     }
+
+    // if skip cleanup
+    skipCleanup = conf.getBoolean(
+        FILEOUTPUTCOMMITTER_CLEANUP_SKIPPED,
+        FILEOUTPUTCOMMITTER_CLEANUP_SKIPPED_DEFAULT);
+
+    // if ignore failures in cleanup
+    ignoreCleanupFailures = conf.getBoolean(
+        FILEOUTPUTCOMMITTER_CLEANUP_FAILURES_IGNORED,
+        FILEOUTPUTCOMMITTER_CLEANUP_FAILURES_IGNORED_DEFAULT);
+
+    LOG.info("FileOutputCommitter skip cleanup _temporary folders under " +
+        "output directory:" + skipCleanup + ", ignore cleanup failures: " +
+        ignoreCleanupFailures);
+
     if (outputPath != null) {
       FileSystem fs = outputPath.getFileSystem(context.getConfiguration());
       this.outputPath = fs.makeQualified(outputPath);
@@ -327,8 +356,25 @@ public class FileOutputCommitter extends OutputCommitter {
         }
       }
 
-      // delete the _temporary folder and create a _done file in the o/p folder
-      cleanupJob(context);
+      if (skipCleanup) {
+        LOG.info("Skip cleanup the _temporary folders under job's output " +
+            "directory in commitJob.");
+      } else {
+        // delete the _temporary folder and create a _done file in the o/p
+        // folder
+        try {
+          cleanupJob(context);
+        } catch (IOException e) {
+          if (ignoreCleanupFailures) {
+            // swallow exceptions in cleanup as user configure to make sure
+            // commitJob could be success even when cleanup get failure.
+            LOG.error("Error in cleanup job, manually cleanup is needed.", e);
+          } else {
+            // throw back exception to fail commitJob.
+            throw e;
+          }
+        }
+      }
       // True if the job requires output.dir marked on successful job.
       // Note that by default it is set to true.
       if (context.getConfiguration().getBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, true)) {
@@ -600,4 +646,4 @@ public class FileOutputCommitter extends OutputCommitter {
       LOG.warn("Output Path is null in recoverTask()");
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/372ad270/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 32bf7dc..822624f 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -451,6 +451,9 @@ Release 2.8.0 - UNRELEASED
     YARN-2597. MiniYARNCluster should propagate reason for AHS not starting.
     (stevel)
 
+    MAPREDUCE-6478. Add an option to skip cleanupJob stage or ignore cleanup
+    failure during commitJob. (Junping Du via wangda)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not


[23/50] [abbrv] hadoop git commit: Update CHANGES.txt to reflect LineRecordReader fixes committed to 2.7.2

Posted by zh...@apache.org.
Update CHANGES.txt to reflect LineRecordReader fixes committed to 2.7.2


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

Branch: refs/heads/HDFS-7285
Commit: 288c8854a698c4b8378f0cc9399a0cccef4f9049
Parents: 2ff6faf
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Sep 18 15:14:18 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Sep 18 15:14:18 2015 +0000

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt | 16 ++++++++--------
 1 file changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/288c8854/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index cd84a34..0c2c8a4 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -521,10 +521,6 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6403. Fix typo in the usage of NNBench.
     (Jagadesh Kiran N via aajisaka)
 
-    MAPREDUCE-5948. org.apache.hadoop.mapred.LineRecordReader does not handle
-    multibyte record delimiters well (Vinayakumar B, Rushabh Shah, and Akira
-    AJISAKA via jlowe)
-
     MAPREDUCE-6400. Multiple shuffle transfer fails because input is closed
     too early (Brahma Reddy Battula, Akira AJISAKA, and Gera Shegalov via
     jlowe)
@@ -566,10 +562,6 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6452. NPE when intermediate encrypt enabled for LocalRunner.
     (Zhihai Xu)
 
-    MAPREDUCE-6481. LineRecordReader may give incomplete record and wrong
-    position/key information for uncompressed input sometimes. (Zhihai Xu via
-    jlowe)
-
     MAPREDUCE-5002. AM could potentially allocate a reduce container to a map
     attempt (Chang Li via jlowe)
 
@@ -606,6 +598,14 @@ Release 2.7.2 - UNRELEASED
     MAPREDUCE-6472. MapReduce AM should have java.io.tmpdir=./tmp to be
     consistent with tasks (Naganarasimha G R via jlowe)
 
+    MAPREDUCE-5948. org.apache.hadoop.mapred.LineRecordReader does not handle
+    multibyte record delimiters well (Vinayakumar B, Rushabh Shah, and Akira
+    AJISAKA via jlowe)
+
+    MAPREDUCE-6481. LineRecordReader may give incomplete record and wrong
+    position/key information for uncompressed input sometimes. (Zhihai Xu via
+    jlowe)
+
 Release 2.7.1 - 2015-07-06 
 
   INCOMPATIBLE CHANGES


[39/50] [abbrv] hadoop git commit: YARN-4188. Make MoveApplicationAcrossQueues abstract, newInstance static

Posted by zh...@apache.org.
YARN-4188. Make MoveApplicationAcrossQueues abstract, newInstance static

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/8e01b0d9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8e01b0d9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8e01b0d9

Branch: refs/heads/HDFS-7285
Commit: 8e01b0d97ac3d74b049a801dfa1cc6e77d8f680a
Parents: b00392d
Author: Chris Douglas <cd...@apache.org>
Authored: Mon Sep 21 17:51:48 2015 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Mon Sep 21 17:51:48 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                 | 3 +++
 .../protocolrecords/MoveApplicationAcrossQueuesResponse.java    | 5 +++--
 2 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8e01b0d9/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 3ef5b75..0e20d9a 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -868,6 +868,9 @@ Release 2.8.0 - UNRELEASED
     YARN-4113. RM should respect retry-interval when uses RetryPolicies.RETRY_FOREVER. 
     (Sunil G via wangda)
 
+    YARN-4188. Make MoveApplicationAcrossQueues abstract, newInstance static.
+    (Giovanni Matteo Fumarola via cdouglas)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8e01b0d9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/MoveApplicationAcrossQueuesResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/MoveApplicationAcrossQueuesResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/MoveApplicationAcrossQueuesResponse.java
index 109e7c4..35846dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/MoveApplicationAcrossQueuesResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/MoveApplicationAcrossQueuesResponse.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.hadoop.yarn.api.protocolrecords;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -36,10 +37,10 @@ import org.apache.hadoop.yarn.util.Records;
  */
 @Public
 @Unstable
-public class MoveApplicationAcrossQueuesResponse {
+public abstract class MoveApplicationAcrossQueuesResponse {
   @Private
   @Unstable
-  public MoveApplicationAcrossQueuesResponse newInstance() {
+  public static MoveApplicationAcrossQueuesResponse newInstance() {
     MoveApplicationAcrossQueuesResponse response =
         Records.newRecord(MoveApplicationAcrossQueuesResponse.class);
     return response;


[35/50] [abbrv] hadoop git commit: HADOOP-12360. Create StatsD metrics2 sink. (Dave Marion via stevel)

Posted by zh...@apache.org.
HADOOP-12360. Create StatsD metrics2 sink. (Dave Marion via stevel)


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

Branch: refs/heads/HDFS-7285
Commit: c39ddc306d7efd93332fc1062ff4c9b59e94f3e1
Parents: e3ace31
Author: Steve Loughran <st...@apache.org>
Authored: Sat Sep 19 12:41:29 2015 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Sat Sep 19 12:41:42 2015 +0100

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   2 +
 .../apache/hadoop/metrics2/package-info.java    |   4 +-
 .../apache/hadoop/metrics2/sink/StatsDSink.java | 218 +++++++++++++++++++
 .../hadoop/metrics2/impl/TestStatsDMetrics.java | 122 +++++++++++
 4 files changed, 345 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c39ddc30/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 2bf5c9d..0c8550d 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -546,6 +546,8 @@ Release 2.8.0 - UNRELEASED
     HADOOP-5732. Add SFTP FileSystem. (Ramtin Boustani and Inigo Goiri via
     cdouglas)
 
+    HADOOP-12360. Create StatsD metrics2 sink. (Dave Marion via stevel)
+
   IMPROVEMENTS
 
     HADOOP-12271. Hadoop Jar Error Should Be More Explanatory

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c39ddc30/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/package-info.java
index be21499..877c912 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/package-info.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/package-info.java
@@ -74,7 +74,9 @@
     </dd>
     <dt> <code>org.apache.hadoop.metrics2.sink</code></dt>
     <dd>Builtin metrics sink implementations including the
-      {@link org.apache.hadoop.metrics2.sink.FileSink}.
+      {@link org.apache.hadoop.metrics2.sink.FileSink},
+      {@link org.apache.hadoop.metrics2.sink.GraphiteSink}, and
+      {@link org.apache.hadoop.metrics2.sink.StatsDSink}.
     </dd>
     <dt> <code>org.apache.hadoop.metrics2.util</code></dt>
     <dd>General utilities for implementing metrics sinks etc., including the

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c39ddc30/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/StatsDSink.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/StatsDSink.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/StatsDSink.java
new file mode 100644
index 0000000..b8059af
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/StatsDSink.java
@@ -0,0 +1,218 @@
+/**
+ * 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.metrics2.sink;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.DatagramPacket;
+import java.net.DatagramSocket;
+import java.net.InetSocketAddress;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.commons.configuration.SubsetConfiguration;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricType;
+import org.apache.hadoop.metrics2.MetricsException;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.MetricsTag;
+import org.apache.hadoop.metrics2.impl.MsInfo;
+import org.apache.hadoop.net.NetUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A metrics sink that writes metrics to a StatsD daemon.
+ * This sink will produce metrics of the form
+ * '[hostname].servicename.context.name.metricname:value|type'
+ * where hostname is optional. This is useful when sending to
+ * a daemon that is running on the localhost and will add the
+ * hostname to the metric (such as the
+ * <a href="https://collectd.org/">CollectD</a> StatsD plugin).
+ * <br/>
+ * To configure this plugin, you will need to add the following
+ * entries to your hadoop-metrics2.properties file:
+ * <br/>
+ * <pre>
+ * *.sink.statsd.class=org.apache.hadoop.metrics2.sink.StatsDSink
+ * [prefix].sink.statsd.server.host=
+ * [prefix].sink.statsd.server.port=
+ * [prefix].sink.statsd.skip.hostname=true|false (optional)
+ * [prefix].sink.statsd.service.name=NameNode (name you want for service)
+ * </pre>
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class StatsDSink implements MetricsSink, Closeable {
+  private static final Logger LOG = LoggerFactory.getLogger(StatsDSink.class);
+  private static final String PERIOD = ".";
+  private static final String SERVER_HOST_KEY = "server.host";
+  private static final String SERVER_PORT_KEY = "server.port";
+  private static final String HOST_NAME_KEY = "host.name";
+  private static final String SERVICE_NAME_KEY = "service.name";
+  private static final String SKIP_HOSTNAME_KEY = "skip.hostname";
+  private boolean skipHostname = false;
+  private String hostName = null;
+  private String serviceName = null;
+  private StatsD statsd = null;
+
+  @Override
+  public void init(SubsetConfiguration conf) {
+    // Get StatsD host configurations.
+    final String serverHost = conf.getString(SERVER_HOST_KEY);
+    final int serverPort = Integer.parseInt(conf.getString(SERVER_PORT_KEY));
+
+    skipHostname = conf.getBoolean(SKIP_HOSTNAME_KEY, false);
+    if (!skipHostname) {
+      hostName = conf.getString(HOST_NAME_KEY, null);
+      if (null == hostName) {
+        hostName = NetUtils.getHostname();
+      }
+    }
+
+    serviceName = conf.getString(SERVICE_NAME_KEY, null);
+
+    statsd = new StatsD(serverHost, serverPort);
+  }
+
+  @Override
+  public void putMetrics(MetricsRecord record) {
+
+    String hn = hostName;
+    String ctx = record.context();
+    String sn = serviceName;
+
+    for (MetricsTag tag : record.tags()) {
+      if (tag.info().name().equals(MsInfo.Hostname.name())
+          && tag.value() != null) {
+        hn = tag.value();
+      } else if (tag.info().name().equals(MsInfo.Context.name())
+          && tag.value() != null) {
+        ctx = tag.value();
+      } else if (tag.info().name().equals(MsInfo.ProcessName.name())
+          && tag.value() != null) {
+        sn = tag.value();
+      }
+    }
+
+    StringBuilder buf = new StringBuilder();
+    if (!skipHostname && hn != null) {
+      int idx = hn.indexOf(".");
+      if (idx == -1) {
+        buf.append(hn).append(PERIOD);
+      } else {
+        buf.append(hn.substring(0, idx)).append(PERIOD);
+      }
+    }
+    buf.append(sn).append(PERIOD);
+    buf.append(ctx).append(PERIOD);
+    buf.append(record.name().replaceAll("\\.", "-")).append(PERIOD);
+
+    // Collect datapoints.
+    for (AbstractMetric metric : record.metrics()) {
+      String type = null;
+      if (metric.type().equals(MetricType.COUNTER)) {
+        type = "c";
+      } else if (metric.type().equals(MetricType.GAUGE)) {
+        type = "g";
+      }
+      StringBuilder line = new StringBuilder();
+      line.append(buf.toString())
+          .append(metric.name().replace(' ', '_'))
+          .append(":")
+          .append(metric.value())
+          .append("|")
+          .append(type);
+      writeMetric(line.toString());
+    }
+
+  }
+
+  public void writeMetric(String line) {
+    try {
+      statsd.write(line);
+    } catch (IOException e) {
+      LOG.warn("Error sending metrics to StatsD", e);
+      throw new MetricsException("Error writing metric to StatsD", e);
+    }
+  }
+
+  @Override
+  public void flush() {
+  }
+
+  @Override
+  public void close() throws IOException {
+    statsd.close();
+  }
+
+  /**
+   * Class that sends UDP packets to StatsD daemon.
+   *
+   */
+  public static class StatsD {
+
+    private DatagramSocket socket = null;
+    private DatagramPacket packet = null;
+    private String serverHost;
+    private int serverPort;
+
+    public StatsD(String serverHost, int serverPort) {
+      this.serverHost = serverHost;
+      this.serverPort = serverPort;
+    }
+
+    public void createSocket() throws IOException {
+      try {
+        InetSocketAddress address =
+            new InetSocketAddress(this.serverHost, this.serverPort);
+        socket = new DatagramSocket();
+        packet =
+            new DatagramPacket("".getBytes(StandardCharsets.UTF_8), 0, 0,
+                address.getAddress(), this.serverPort);
+      } catch (IOException ioe) {
+        throw NetUtils.wrapException(this.serverHost, this.serverPort,
+            "localhost", 0, ioe);
+      }
+    }
+
+    public void write(String msg) throws IOException {
+      if (null == socket) {
+        createSocket();
+      }
+      LOG.debug("Sending metric: {}", msg);
+      packet.setData(msg.getBytes(StandardCharsets.UTF_8));
+      socket.send(packet);
+    }
+
+    public void close() throws IOException {
+      try {
+        if (socket != null) {
+          socket.close();
+        }
+      } finally {
+        socket = null;
+      }
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c39ddc30/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestStatsDMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestStatsDMetrics.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestStatsDMetrics.java
new file mode 100644
index 0000000..355c1cb
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestStatsDMetrics.java
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.metrics2.impl;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.net.DatagramPacket;
+import java.net.DatagramSocket;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricType;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsTag;
+import org.apache.hadoop.metrics2.sink.StatsDSink;
+import org.apache.hadoop.metrics2.sink.StatsDSink.StatsD;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+
+public class TestStatsDMetrics {
+
+  private AbstractMetric makeMetric(String name, Number value,
+      MetricType type) {
+    AbstractMetric metric = mock(AbstractMetric.class);
+    when(metric.name()).thenReturn(name);
+    when(metric.value()).thenReturn(value);
+    when(metric.type()).thenReturn(type);
+    return metric;
+  }
+
+  @Test(timeout=3000)
+  public void testPutMetrics() throws IOException, InterruptedException {
+    final StatsDSink sink = new StatsDSink();
+    List<MetricsTag> tags = new ArrayList<MetricsTag>();
+    tags.add(new MetricsTag(MsInfo.Hostname, "host"));
+    tags.add(new MetricsTag(MsInfo.Context, "jvm"));
+    tags.add(new MetricsTag(MsInfo.ProcessName, "process"));
+    Set<AbstractMetric> metrics = new HashSet<AbstractMetric>();
+    metrics.add(makeMetric("foo1", 1.25, MetricType.COUNTER));
+    metrics.add(makeMetric("foo2", 2.25, MetricType.GAUGE));
+    final MetricsRecord record =
+        new MetricsRecordImpl(MsInfo.Context, (long) 10000, tags, metrics);
+
+    try (DatagramSocket sock = new DatagramSocket()) {
+      sock.setReceiveBufferSize(8192);
+      final StatsDSink.StatsD mockStatsD =
+          new StatsD(sock.getLocalAddress().getHostName(),
+              sock.getLocalPort());
+      Whitebox.setInternalState(sink, "statsd", mockStatsD);
+      final DatagramPacket p = new DatagramPacket(new byte[8192], 8192);
+      sink.putMetrics(record);
+      sock.receive(p);
+
+      String result =new String(p.getData(), 0, p.getLength(),
+          Charset.forName("UTF-8"));
+      assertTrue(
+          "Received data did not match data sent",
+          result.equals("host.process.jvm.Context.foo1:1.25|c") ||
+          result.equals("host.process.jvm.Context.foo2:2.25|g"));
+
+    } finally {
+      sink.close();
+    }
+  }
+
+  @Test(timeout=3000)
+  public void testPutMetrics2() throws IOException {
+    StatsDSink sink = new StatsDSink();
+    List<MetricsTag> tags = new ArrayList<MetricsTag>();
+    tags.add(new MetricsTag(MsInfo.Hostname, null));
+    tags.add(new MetricsTag(MsInfo.Context, "jvm"));
+    tags.add(new MetricsTag(MsInfo.ProcessName, "process"));
+    Set<AbstractMetric> metrics = new HashSet<AbstractMetric>();
+    metrics.add(makeMetric("foo1", 1, MetricType.COUNTER));
+    metrics.add(makeMetric("foo2", 2, MetricType.GAUGE));
+    MetricsRecord record =
+        new MetricsRecordImpl(MsInfo.Context, (long) 10000, tags, metrics);
+
+    try (DatagramSocket sock = new DatagramSocket()) {
+      sock.setReceiveBufferSize(8192);
+      final StatsDSink.StatsD mockStatsD =
+          new StatsD(sock.getLocalAddress().getHostName(),
+              sock.getLocalPort());
+      Whitebox.setInternalState(sink, "statsd", mockStatsD);
+      final DatagramPacket p = new DatagramPacket(new byte[8192], 8192);
+      sink.putMetrics(record);
+      sock.receive(p);
+      String result =
+          new String(p.getData(), 0, p.getLength(), Charset.forName("UTF-8"));
+
+      assertTrue("Received data did not match data sent",
+          result.equals("process.jvm.Context.foo1:1|c") ||
+          result.equals("process.jvm.Context.foo2:2|g"));
+    } finally {
+      sink.close();
+    }
+  }
+
+}


[42/50] [abbrv] hadoop git commit: HDFS-9111. Move hdfs-client protobuf convert methods from PBHelper to PBHelperClient. Contributed by Mingliang Liu.

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
deleted file mode 100644
index 23e8f57..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.protocol;
-
-import java.net.URI;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.FileEncryptionInfo;
-import org.apache.hadoop.fs.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DFSUtilClient;
-
-/** 
- * Interface that represents the over the wire information
- * including block locations for a file.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class HdfsLocatedFileStatus extends HdfsFileStatus {
-  private final LocatedBlocks locations;
-
-  /**
-   * Constructor
-   * 
-   * @param length size
-   * @param isdir if this is directory
-   * @param block_replication the file's replication factor
-   * @param blocksize the file's block size
-   * @param modification_time most recent modification time
-   * @param access_time most recent access time
-   * @param permission permission
-   * @param owner owner
-   * @param group group
-   * @param symlink symbolic link
-   * @param path local path name in java UTF8 format 
-   * @param fileId the file id
-   * @param locations block locations
-   * @param feInfo file encryption info
-   */
-  public HdfsLocatedFileStatus(long length, boolean isdir,
-      int block_replication, long blocksize, long modification_time,
-      long access_time, FsPermission permission, String owner, String group,
-      byte[] symlink, byte[] path, long fileId, LocatedBlocks locations,
-      int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy) {
-    super(length, isdir, block_replication, blocksize, modification_time,
-        access_time, permission, owner, group, symlink, path, fileId,
-        childrenNum, feInfo, storagePolicy);
-    this.locations = locations;
-  }
-
-  public LocatedBlocks getBlockLocations() {
-    return locations;
-  }
-
-  public final LocatedFileStatus makeQualifiedLocated(URI defaultUri,
-      Path path) {
-    return new LocatedFileStatus(getLen(), isDir(), getReplication(),
-        getBlockSize(), getModificationTime(),
-        getAccessTime(),
-        getPermission(), getOwner(), getGroup(),
-        isSymlink() ? new Path(getSymlink()) : null,
-        (getFullPath(path)).makeQualified(
-            defaultUri, null), // fully-qualify path
-        DFSUtilClient.locatedBlocks2Locations(getBlockLocations()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
index 85da414..59cf884 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockP
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReleaseShortCircuitAccessRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmRequestProto;
-import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
@@ -115,7 +114,7 @@ public abstract class Receiver implements DataTransferProtocol {
         proto.getClass().getSimpleName());
     try {
       readBlock(PBHelperClient.convert(proto.getHeader().getBaseHeader().getBlock()),
-        PBHelper.convert(proto.getHeader().getBaseHeader().getToken()),
+        PBHelperClient.convert(proto.getHeader().getBaseHeader().getToken()),
         proto.getHeader().getClientName(),
         proto.getOffset(),
         proto.getLen(),
@@ -131,17 +130,17 @@ public abstract class Receiver implements DataTransferProtocol {
   /** Receive OP_WRITE_BLOCK */
   private void opWriteBlock(DataInputStream in) throws IOException {
     final OpWriteBlockProto proto = OpWriteBlockProto.parseFrom(vintPrefixed(in));
-    final DatanodeInfo[] targets = PBHelper.convert(proto.getTargetsList());
+    final DatanodeInfo[] targets = PBHelperClient.convert(proto.getTargetsList());
     TraceScope traceScope = continueTraceSpan(proto.getHeader(),
         proto.getClass().getSimpleName());
     try {
       writeBlock(PBHelperClient.convert(proto.getHeader().getBaseHeader().getBlock()),
           PBHelperClient.convertStorageType(proto.getStorageType()),
-          PBHelper.convert(proto.getHeader().getBaseHeader().getToken()),
+          PBHelperClient.convert(proto.getHeader().getBaseHeader().getToken()),
           proto.getHeader().getClientName(),
           targets,
-          PBHelper.convertStorageTypes(proto.getTargetStorageTypesList(), targets.length),
-          PBHelper.convert(proto.getSource()),
+          PBHelperClient.convertStorageTypes(proto.getTargetStorageTypesList(), targets.length),
+          PBHelperClient.convert(proto.getSource()),
           fromProto(proto.getStage()),
           proto.getPipelineSize(),
           proto.getMinBytesRcvd(), proto.getMaxBytesRcvd(),
@@ -152,7 +151,7 @@ public abstract class Receiver implements DataTransferProtocol {
             CachingStrategy.newDefaultStrategy()),
           (proto.hasAllowLazyPersist() ? proto.getAllowLazyPersist() : false),
           (proto.hasPinning() ? proto.getPinning(): false),
-          (PBHelper.convertBooleanList(proto.getTargetPinningsList())));
+          (PBHelperClient.convertBooleanList(proto.getTargetPinningsList())));
     } finally {
      if (traceScope != null) traceScope.close();
     }
@@ -162,15 +161,15 @@ public abstract class Receiver implements DataTransferProtocol {
   private void opTransferBlock(DataInputStream in) throws IOException {
     final OpTransferBlockProto proto =
       OpTransferBlockProto.parseFrom(vintPrefixed(in));
-    final DatanodeInfo[] targets = PBHelper.convert(proto.getTargetsList());
+    final DatanodeInfo[] targets = PBHelperClient.convert(proto.getTargetsList());
     TraceScope traceScope = continueTraceSpan(proto.getHeader(),
         proto.getClass().getSimpleName());
     try {
       transferBlock(PBHelperClient.convert(proto.getHeader().getBaseHeader().getBlock()),
-          PBHelper.convert(proto.getHeader().getBaseHeader().getToken()),
+          PBHelperClient.convert(proto.getHeader().getBaseHeader().getToken()),
           proto.getHeader().getClientName(),
           targets,
-          PBHelper.convertStorageTypes(proto.getTargetStorageTypesList(), targets.length));
+          PBHelperClient.convertStorageTypes(proto.getTargetStorageTypesList(), targets.length));
     } finally {
       if (traceScope != null) traceScope.close();
     }
@@ -181,12 +180,12 @@ public abstract class Receiver implements DataTransferProtocol {
     final OpRequestShortCircuitAccessProto proto =
       OpRequestShortCircuitAccessProto.parseFrom(vintPrefixed(in));
     SlotId slotId = (proto.hasSlotId()) ? 
-        PBHelper.convert(proto.getSlotId()) : null;
+        PBHelperClient.convert(proto.getSlotId()) : null;
     TraceScope traceScope = continueTraceSpan(proto.getHeader(),
         proto.getClass().getSimpleName());
     try {
       requestShortCircuitFds(PBHelperClient.convert(proto.getHeader().getBlock()),
-          PBHelper.convert(proto.getHeader().getToken()),
+          PBHelperClient.convert(proto.getHeader().getToken()),
           slotId, proto.getMaxVersion(),
           proto.getSupportsReceiptVerification());
     } finally {
@@ -202,7 +201,7 @@ public abstract class Receiver implements DataTransferProtocol {
     TraceScope traceScope = continueTraceSpan(proto.getTraceInfo(),
         proto.getClass().getSimpleName());
     try {
-      releaseShortCircuitFds(PBHelper.convert(proto.getSlotId()));
+      releaseShortCircuitFds(PBHelperClient.convert(proto.getSlotId()));
     } finally {
       if (traceScope != null) traceScope.close();
     }
@@ -229,9 +228,9 @@ public abstract class Receiver implements DataTransferProtocol {
     try {
       replaceBlock(PBHelperClient.convert(proto.getHeader().getBlock()),
           PBHelperClient.convertStorageType(proto.getStorageType()),
-          PBHelper.convert(proto.getHeader().getToken()),
+          PBHelperClient.convert(proto.getHeader().getToken()),
           proto.getDelHint(),
-          PBHelper.convert(proto.getSource()));
+          PBHelperClient.convert(proto.getSource()));
     } finally {
       if (traceScope != null) traceScope.close();
     }
@@ -244,7 +243,7 @@ public abstract class Receiver implements DataTransferProtocol {
         proto.getClass().getSimpleName());
     try {
       copyBlock(PBHelperClient.convert(proto.getHeader().getBlock()),
-          PBHelper.convert(proto.getHeader().getToken()));
+          PBHelperClient.convert(proto.getHeader().getToken()));
     } finally {
       if (traceScope != null) traceScope.close();
     }
@@ -257,7 +256,7 @@ public abstract class Receiver implements DataTransferProtocol {
         proto.getClass().getSimpleName());
     try {
     blockChecksum(PBHelperClient.convert(proto.getHeader().getBlock()),
-        PBHelper.convert(proto.getHeader().getToken()));
+        PBHelperClient.convert(proto.getHeader().getToken()));
     } finally {
       if (traceScope != null) traceScope.close();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
index 3adb4a8..76ad820 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
@@ -123,7 +123,9 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
       throws ServiceException {
     BlockLocalPathInfo resp;
     try {
-      resp = impl.getBlockLocalPathInfo(PBHelperClient.convert(request.getBlock()), PBHelper.convert(request.getToken()));
+      resp = impl.getBlockLocalPathInfo(
+                 PBHelperClient.convert(request.getBlock()),
+                 PBHelperClient.convert(request.getToken()));
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -150,7 +152,7 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
     GetDatanodeInfoResponseProto res;
     try {
       res = GetDatanodeInfoResponseProto.newBuilder()
-          .setLocalInfo(PBHelper.convert(impl.getDatanodeInfo())).build();
+          .setLocalInfo(PBHelperClient.convert(impl.getDatanodeInfo())).build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
index 0d3796c..e7ce44b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs.protocolPB;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
@@ -31,7 +31,7 @@ import org.apache.hadoop.security.token.TokenInfo;
 @InterfaceAudience.Private
 @InterfaceStability.Stable
 @KerberosInfo(
-    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY)
+    serverPrincipal = HdfsClientConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY)
 @TokenInfo(DelegationTokenSelector.class)
 @ProtocolInfo(protocolName = HdfsConstants.CLIENT_NAMENODE_PROTOCOL_NAME,
     protocolVersion = 1)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index 9f26ca3..d93277c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -367,7 +367,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       Builder builder = GetBlockLocationsResponseProto
           .newBuilder();
       if (b != null) {
-        builder.setLocations(PBHelper.convert(b)).build();
+        builder.setLocations(PBHelperClient.convert(b)).build();
       }
       return builder.build();
     } catch (IOException e) {
@@ -382,7 +382,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
     try {
       FsServerDefaults result = server.getServerDefaults();
       return GetServerDefaultsResponseProto.newBuilder()
-          .setServerDefaults(PBHelper.convert(result))
+          .setServerDefaults(PBHelperClient.convert(result))
           .build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -395,14 +395,14 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       CreateRequestProto req) throws ServiceException {
     try {
       HdfsFileStatus result = server.create(req.getSrc(),
-          PBHelper.convert(req.getMasked()), req.getClientName(),
-          PBHelper.convertCreateFlag(req.getCreateFlag()), req.getCreateParent(),
+          PBHelperClient.convert(req.getMasked()), req.getClientName(),
+          PBHelperClient.convertCreateFlag(req.getCreateFlag()), req.getCreateParent(),
           (short) req.getReplication(), req.getBlockSize(),
-          PBHelper.convertCryptoProtocolVersions(
+          PBHelperClient.convertCryptoProtocolVersions(
               req.getCryptoProtocolVersionList()));
 
       if (result != null) {
-        return CreateResponseProto.newBuilder().setFs(PBHelper.convert(result))
+        return CreateResponseProto.newBuilder().setFs(PBHelperClient.convert(result))
             .build();
       }
       return VOID_CREATE_RESPONSE;
@@ -416,16 +416,16 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       AppendRequestProto req) throws ServiceException {
     try {
       EnumSetWritable<CreateFlag> flags = req.hasFlag() ?
-          PBHelper.convertCreateFlag(req.getFlag()) :
+          PBHelperClient.convertCreateFlag(req.getFlag()) :
           new EnumSetWritable<>(EnumSet.of(CreateFlag.APPEND));
       LastBlockWithStatus result = server.append(req.getSrc(),
           req.getClientName(), flags);
       AppendResponseProto.Builder builder = AppendResponseProto.newBuilder();
       if (result.getLastBlock() != null) {
-        builder.setBlock(PBHelper.convert(result.getLastBlock()));
+        builder.setBlock(PBHelperClient.convert(result.getLastBlock()));
       }
       if (result.getFileStatus() != null) {
-        builder.setStat(PBHelper.convert(result.getFileStatus()));
+        builder.setStat(PBHelperClient.convert(result.getFileStatus()));
       }
       return builder.build();
     } catch (IOException e) {
@@ -450,7 +450,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public SetPermissionResponseProto setPermission(RpcController controller,
       SetPermissionRequestProto req) throws ServiceException {
     try {
-      server.setPermission(req.getSrc(), PBHelper.convert(req.getPermission()));
+      server.setPermission(req.getSrc(), PBHelperClient.convert(req.getPermission()));
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -493,12 +493,12 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
           req.getSrc(),
           req.getClientName(),
           req.hasPrevious() ? PBHelperClient.convert(req.getPrevious()) : null,
-          (excl == null || excl.size() == 0) ? null : PBHelper.convert(excl
+          (excl == null || excl.size() == 0) ? null : PBHelperClient.convert(excl
               .toArray(new DatanodeInfoProto[excl.size()])), req.getFileId(),
           (favor == null || favor.size() == 0) ? null : favor
               .toArray(new String[favor.size()]));
       return AddBlockResponseProto.newBuilder()
-          .setBlock(PBHelper.convert(result)).build();
+          .setBlock(PBHelperClient.convert(result)).build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -514,15 +514,15 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       List<DatanodeInfoProto> excludesList = req.getExcludesList();
       LocatedBlock result = server.getAdditionalDatanode(req.getSrc(),
           req.getFileId(), PBHelperClient.convert(req.getBlk()),
-          PBHelper.convert(existingList.toArray(
+          PBHelperClient.convert(existingList.toArray(
               new DatanodeInfoProto[existingList.size()])),
           existingStorageIDsList.toArray(
               new String[existingStorageIDsList.size()]),
-          PBHelper.convert(excludesList.toArray(
-              new DatanodeInfoProto[excludesList.size()])), 
+          PBHelperClient.convert(excludesList.toArray(
+              new DatanodeInfoProto[excludesList.size()])),
           req.getNumAdditionalNodes(), req.getClientName());
       return GetAdditionalDatanodeResponseProto.newBuilder().setBlock(
-          PBHelper.convert(result))
+          PBHelperClient.convert(result))
           .build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -548,8 +548,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       ReportBadBlocksRequestProto req) throws ServiceException {
     try {
       List<LocatedBlockProto> bl = req.getBlocksList();
-      server.reportBadBlocks(PBHelper.convertLocatedBlock(
-              bl.toArray(new LocatedBlockProto[bl.size()])));
+      server.reportBadBlocks(PBHelperClient.convertLocatedBlock(
+          bl.toArray(new LocatedBlockProto[bl.size()])));
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -620,7 +620,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       MkdirsRequestProto req) throws ServiceException {
     try {
       boolean result = server.mkdirs(req.getSrc(),
-          PBHelper.convert(req.getMasked()), req.getCreateParent());
+          PBHelperClient.convert(req.getMasked()), req.getCreateParent());
       return MkdirsResponseProto.newBuilder().setResult(result).build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -636,7 +636,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
           req.getNeedLocation());
       if (result !=null) {
         return GetListingResponseProto.newBuilder().setDirList(
-          PBHelper.convert(result)).build();
+          PBHelperClient.convert(result)).build();
       } else {
         return VOID_GETLISTING_RESPONSE;
       }
@@ -684,7 +684,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public GetFsStatsResponseProto getFsStats(RpcController controller,
       GetFsStatusRequestProto req) throws ServiceException {
     try {
-      return PBHelper.convert(server.getStats());
+      return PBHelperClient.convert(server.getStats());
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -696,7 +696,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throws ServiceException {
     try {
       List<? extends DatanodeInfoProto> result = PBHelperClient.convert(server
-          .getDatanodeReport(PBHelper.convert(req.getType())));
+          .getDatanodeReport(PBHelperClient.convert(req.getType())));
       return GetDatanodeReportResponseProto.newBuilder()
           .addAllDi(result).build();
     } catch (IOException e) {
@@ -709,8 +709,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       RpcController controller, GetDatanodeStorageReportRequestProto req)
       throws ServiceException {
     try {
-      List<DatanodeStorageReportProto> reports = PBHelper.convertDatanodeStorageReports(
-          server.getDatanodeStorageReport(PBHelper.convert(req.getType())));
+      List<DatanodeStorageReportProto> reports = PBHelperClient.convertDatanodeStorageReports(
+          server.getDatanodeStorageReport(PBHelperClient.convert(req.getType())));
       return GetDatanodeStorageReportResponseProto.newBuilder()
           .addAllDatanodeStorageReports(reports)
           .build();
@@ -736,7 +736,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public SetSafeModeResponseProto setSafeMode(RpcController controller,
       SetSafeModeRequestProto req) throws ServiceException {
     try {
-      boolean result = server.setSafeMode(PBHelper.convert(req.getAction()),
+      boolean result = server.setSafeMode(PBHelperClient.convert(req.getAction()),
           req.getChecked());
       return SetSafeModeResponseProto.newBuilder().setResult(result).build();
     } catch (IOException e) {
@@ -799,10 +799,10 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       RollingUpgradeRequestProto req) throws ServiceException {
     try {
       final RollingUpgradeInfo info = server.rollingUpgrade(
-          PBHelper.convert(req.getAction()));
+          PBHelperClient.convert(req.getAction()));
       final RollingUpgradeResponseProto.Builder b = RollingUpgradeResponseProto.newBuilder();
       if (info != null) {
-        b.setRollingUpgradeInfo(PBHelper.convert(info));
+        b.setRollingUpgradeInfo(PBHelperClient.convert(info));
       }
       return b.build();
     } catch (IOException e) {
@@ -818,7 +818,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       CorruptFileBlocks result = server.listCorruptFileBlocks(
           req.getPath(), req.hasCookie() ? req.getCookie(): null);
       return ListCorruptFileBlocksResponseProto.newBuilder()
-          .setCorrupt(PBHelper.convert(result))
+          .setCorrupt(PBHelperClient.convert(result))
           .build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -845,7 +845,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
  
       if (result != null) {
         return GetFileInfoResponseProto.newBuilder().setFs(
-            PBHelper.convert(result)).build();
+            PBHelperClient.convert(result)).build();
       }
       return VOID_GETFILEINFO_RESPONSE;      
     } catch (IOException e) {
@@ -860,7 +860,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       HdfsFileStatus result = server.getFileLinkInfo(req.getSrc());
       if (result != null) {
         return GetFileLinkInfoResponseProto.newBuilder().setFs(
-            PBHelper.convert(result)).build();
+            PBHelperClient.convert(result)).build();
       } else {
         return VOID_GETFILELINKINFO_RESPONSE;      
       }
@@ -877,7 +877,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
     try {
       ContentSummary result = server.getContentSummary(req.getPath());
       return GetContentSummaryResponseProto.newBuilder()
-          .setSummary(PBHelper.convert(result)).build();
+          .setSummary(PBHelperClient.convert(result)).build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -925,7 +925,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       CreateSymlinkRequestProto req) throws ServiceException {
     try {
       server.createSymlink(req.getTarget(), req.getLink(),
-          PBHelper.convert(req.getDirPerm()), req.getCreateParent());
+          PBHelperClient.convert(req.getDirPerm()), req.getCreateParent());
       return VOID_CREATESYMLINK_RESPONSE;
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -953,7 +953,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       RpcController controller, UpdateBlockForPipelineRequestProto req)
       throws ServiceException {
     try {
-      LocatedBlockProto result = PBHelper.convert(server
+      LocatedBlockProto result = PBHelperClient.convert(server
           .updateBlockForPipeline(PBHelperClient.convert(req.getBlock()),
               req.getClientName()));
       return UpdateBlockForPipelineResponseProto.newBuilder().setBlock(result)
@@ -972,7 +972,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       server.updatePipeline(req.getClientName(),
           PBHelperClient.convert(req.getOldBlock()),
           PBHelperClient.convert(req.getNewBlock()),
-          PBHelper.convert(newNodes.toArray(new DatanodeIDProto[newNodes.size()])),
+          PBHelperClient.convert(newNodes.toArray(new DatanodeIDProto[newNodes.size()])),
           newStorageIDs.toArray(new String[newStorageIDs.size()]));
       return VOID_UPDATEPIPELINE_RESPONSE;
     } catch (IOException e) {
@@ -1003,7 +1003,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       RpcController controller, RenewDelegationTokenRequestProto req)
       throws ServiceException {
     try {
-      long result = server.renewDelegationToken(PBHelper
+      long result = server.renewDelegationToken(PBHelperClient
           .convertDelegationToken(req.getToken()));
       return RenewDelegationTokenResponseProto.newBuilder()
           .setNewExpiryTime(result).build();
@@ -1017,7 +1017,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       RpcController controller, CancelDelegationTokenRequestProto req)
       throws ServiceException {
     try {
-      server.cancelDelegationToken(PBHelper.convertDelegationToken(req
+      server.cancelDelegationToken(PBHelperClient.convertDelegationToken(req
           .getToken()));
       return VOID_CANCELDELEGATIONTOKEN_RESPONSE;
     } catch (IOException e) {
@@ -1046,7 +1046,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
           GetDataEncryptionKeyResponseProto.newBuilder();
       DataEncryptionKey encryptionKey = server.getDataEncryptionKey();
       if (encryptionKey != null) {
-        builder.setDataEncryptionKey(PBHelper.convert(encryptionKey));
+        builder.setDataEncryptionKey(PBHelperClient.convert(encryptionKey));
       }
       return builder.build();
     } catch (IOException e) {
@@ -1125,7 +1125,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
           .getSnapshottableDirListing();
       if (result != null) {
         return GetSnapshottableDirListingResponseProto.newBuilder().
-            setSnapshottableDirList(PBHelper.convert(result)).build();
+            setSnapshottableDirList(PBHelperClient.convert(result)).build();
       } else {
         return NULL_GET_SNAPSHOTTABLE_DIR_LISTING_RESPONSE;
       }
@@ -1143,7 +1143,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
           request.getSnapshotRoot(), request.getFromSnapshot(),
           request.getToSnapshot());
       return GetSnapshotDiffReportResponseProto.newBuilder()
-          .setDiffReport(PBHelper.convert(report)).build();
+          .setDiffReport(PBHelperClient.convert(report)).build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -1167,8 +1167,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throws ServiceException {
     try {
       long id = server.addCacheDirective(
-          PBHelper.convert(request.getInfo()),
-          PBHelper.convertCacheFlags(request.getCacheFlags()));
+          PBHelperClient.convert(request.getInfo()),
+          PBHelperClient.convertCacheFlags(request.getCacheFlags()));
       return AddCacheDirectiveResponseProto.newBuilder().
               setId(id).build();
     } catch (IOException e) {
@@ -1182,8 +1182,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throws ServiceException {
     try {
       server.modifyCacheDirective(
-          PBHelper.convert(request.getInfo()),
-          PBHelper.convertCacheFlags(request.getCacheFlags()));
+          PBHelperClient.convert(request.getInfo()),
+          PBHelperClient.convertCacheFlags(request.getCacheFlags()));
       return ModifyCacheDirectiveResponseProto.newBuilder().build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -1210,14 +1210,14 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
           throws ServiceException {
     try {
       CacheDirectiveInfo filter =
-          PBHelper.convert(request.getFilter());
+          PBHelperClient.convert(request.getFilter());
       BatchedEntries<CacheDirectiveEntry> entries =
         server.listCacheDirectives(request.getPrevId(), filter);
       ListCacheDirectivesResponseProto.Builder builder =
           ListCacheDirectivesResponseProto.newBuilder();
       builder.setHasMore(entries.hasMore());
       for (int i=0, n=entries.size(); i<n; i++) {
-        builder.addElements(PBHelper.convert(entries.get(i)));
+        builder.addElements(PBHelperClient.convert(entries.get(i)));
       }
       return builder.build();
     } catch (IOException e) {
@@ -1229,7 +1229,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public AddCachePoolResponseProto addCachePool(RpcController controller,
       AddCachePoolRequestProto request) throws ServiceException {
     try {
-      server.addCachePool(PBHelper.convert(request.getInfo()));
+      server.addCachePool(PBHelperClient.convert(request.getInfo()));
       return AddCachePoolResponseProto.newBuilder().build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -1240,7 +1240,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public ModifyCachePoolResponseProto modifyCachePool(RpcController controller,
       ModifyCachePoolRequestProto request) throws ServiceException {
     try {
-      server.modifyCachePool(PBHelper.convert(request.getInfo()));
+      server.modifyCachePool(PBHelperClient.convert(request.getInfo()));
       return ModifyCachePoolResponseProto.newBuilder().build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -1268,7 +1268,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
         ListCachePoolsResponseProto.newBuilder();
       responseBuilder.setHasMore(entries.hasMore());
       for (int i=0, n=entries.size(); i<n; i++) {
-        responseBuilder.addEntries(PBHelper.convert(entries.get(i)));
+        responseBuilder.addEntries(PBHelperClient.convert(entries.get(i)));
       }
       return responseBuilder.build();
     } catch (IOException e) {
@@ -1281,7 +1281,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       RpcController controller, ModifyAclEntriesRequestProto req)
       throws ServiceException {
     try {
-      server.modifyAclEntries(req.getSrc(), PBHelper.convertAclEntry(req.getAclSpecList()));
+      server.modifyAclEntries(req.getSrc(), PBHelperClient.convertAclEntry(req.getAclSpecList()));
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -1294,7 +1294,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throws ServiceException {
     try {
       server.removeAclEntries(req.getSrc(),
-          PBHelper.convertAclEntry(req.getAclSpecList()));
+          PBHelperClient.convertAclEntry(req.getAclSpecList()));
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -1328,7 +1328,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public SetAclResponseProto setAcl(RpcController controller,
       SetAclRequestProto req) throws ServiceException {
     try {
-      server.setAcl(req.getSrc(), PBHelper.convertAclEntry(req.getAclSpecList()));
+      server.setAcl(req.getSrc(), PBHelperClient.convertAclEntry(req.getAclSpecList()));
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -1339,7 +1339,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public GetAclStatusResponseProto getAclStatus(RpcController controller,
       GetAclStatusRequestProto req) throws ServiceException {
     try {
-      return PBHelper.convert(server.getAclStatus(req.getSrc()));
+      return PBHelperClient.convert(server.getAclStatus(req.getSrc()));
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -1366,7 +1366,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
           GetEZForPathResponseProto.newBuilder();
       final EncryptionZone ret = server.getEZForPath(req.getSrc());
       if (ret != null) {
-        builder.setZone(PBHelper.convert(ret));
+        builder.setZone(PBHelperClient.convert(ret));
       }
       return builder.build();
     } catch (IOException e) {
@@ -1385,7 +1385,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
           ListEncryptionZonesResponseProto.newBuilder();
       builder.setHasMore(entries.hasMore());
       for (int i=0; i<entries.size(); i++) {
-        builder.addZones(PBHelper.convert(entries.get(i)));
+        builder.addZones(PBHelperClient.convert(entries.get(i)));
       }
       return builder.build();
     } catch (IOException e) {
@@ -1397,8 +1397,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public SetXAttrResponseProto setXAttr(RpcController controller,
       SetXAttrRequestProto req) throws ServiceException {
     try {
-      server.setXAttr(req.getSrc(), PBHelper.convertXAttr(req.getXAttr()), 
-          PBHelper.convert(req.getFlag()));
+      server.setXAttr(req.getSrc(), PBHelperClient.convertXAttr(req.getXAttr()),
+          PBHelperClient.convert(req.getFlag()));
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -1409,8 +1409,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public GetXAttrsResponseProto getXAttrs(RpcController controller,
       GetXAttrsRequestProto req) throws ServiceException {
     try {
-      return PBHelper.convertXAttrsResponse(server.getXAttrs(req.getSrc(), 
-          PBHelper.convertXAttrs(req.getXAttrsList())));
+      return PBHelperClient.convertXAttrsResponse(server.getXAttrs(req.getSrc(),
+          PBHelperClient.convertXAttrs(req.getXAttrsList())));
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -1420,7 +1420,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public ListXAttrsResponseProto listXAttrs(RpcController controller,
     ListXAttrsRequestProto req) throws ServiceException {
     try {
-      return PBHelper.convertListXAttrsResponse(server.listXAttrs(req.getSrc()));
+      return PBHelperClient.convertListXAttrsResponse(server.listXAttrs(req.getSrc()));
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -1430,7 +1430,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public RemoveXAttrResponseProto removeXAttr(RpcController controller,
       RemoveXAttrRequestProto req) throws ServiceException {
     try {
-      server.removeXAttr(req.getSrc(), PBHelper.convertXAttr(req.getXAttr()));
+      server.removeXAttr(req.getSrc(), PBHelperClient.convertXAttr(req.getXAttr()));
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -1441,7 +1441,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public CheckAccessResponseProto checkAccess(RpcController controller,
      CheckAccessRequestProto req) throws ServiceException {
     try {
-      server.checkAccess(req.getPath(), PBHelper.convert(req.getMode()));
+      server.checkAccess(req.getPath(), PBHelperClient.convert(req.getMode()));
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -1465,7 +1465,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       RpcController controller, GetStoragePolicyRequestProto request)
       throws ServiceException {
     try {
-      BlockStoragePolicyProto policy = PBHelper.convert(server
+      BlockStoragePolicyProto policy = PBHelperClient.convert(server
           .getStoragePolicy(request.getPath()));
       return GetStoragePolicyResponseProto.newBuilder()
           .setStoragePolicy(policy).build();
@@ -1486,7 +1486,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
         return builder.build();
       }
       for (BlockStoragePolicy policy : policies) {
-        builder.addPolicies(PBHelper.convert(policy));
+        builder.addPolicies(PBHelperClient.convert(policy));
       }
       return builder.build();
     } catch (IOException e) {
@@ -1508,7 +1508,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public GetEditsFromTxidResponseProto getEditsFromTxid(RpcController controller,
       GetEditsFromTxidRequestProto req) throws ServiceException {
     try {
-      return PBHelper.convertEditsResponse(server.getEditsFromTxid(
+      return PBHelperClient.convertEditsResponse(server.getEditsFromTxid(
           req.getTxid()));
     } catch (IOException e) {
       throw new ServiceException(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/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 d30982a..f4ce46d 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
@@ -255,8 +255,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
     try {
       GetBlockLocationsResponseProto resp = rpcProxy.getBlockLocations(null,
           req);
-      return resp.hasLocations() ? 
-        PBHelper.convert(resp.getLocations()) : null;
+      return resp.hasLocations() ?
+        PBHelperClient.convert(resp.getLocations()) : null;
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -266,7 +266,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   public FsServerDefaults getServerDefaults() throws IOException {
     GetServerDefaultsRequestProto req = VOID_GET_SERVER_DEFAULT_REQUEST;
     try {
-      return PBHelper
+      return PBHelperClient
           .convert(rpcProxy.getServerDefaults(null, req).getServerDefaults());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
@@ -285,17 +285,17 @@ public class ClientNamenodeProtocolTranslatorPB implements
       IOException {
     CreateRequestProto.Builder builder = CreateRequestProto.newBuilder()
         .setSrc(src)
-        .setMasked(PBHelper.convert(masked))
+        .setMasked(PBHelperClient.convert(masked))
         .setClientName(clientName)
-        .setCreateFlag(PBHelper.convertCreateFlag(flag))
+        .setCreateFlag(PBHelperClient.convertCreateFlag(flag))
         .setCreateParent(createParent)
         .setReplication(replication)
         .setBlockSize(blockSize);
-    builder.addAllCryptoProtocolVersion(PBHelper.convert(supportedVersions));
+    builder.addAllCryptoProtocolVersion(PBHelperClient.convert(supportedVersions));
     CreateRequestProto req = builder.build();
     try {
       CreateResponseProto res = rpcProxy.create(null, req);
-      return res.hasFs() ? PBHelper.convert(res.getFs()) : null;
+      return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null;
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -323,13 +323,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
       DSQuotaExceededException, FileNotFoundException, SafeModeException,
       UnresolvedLinkException, IOException {
     AppendRequestProto req = AppendRequestProto.newBuilder().setSrc(src)
-        .setClientName(clientName).setFlag(PBHelper.convertCreateFlag(flag))
+        .setClientName(clientName).setFlag(PBHelperClient.convertCreateFlag(flag))
         .build();
     try {
       AppendResponseProto res = rpcProxy.append(null, req);
-      LocatedBlock lastBlock = res.hasBlock() ? PBHelper
+      LocatedBlock lastBlock = res.hasBlock() ? PBHelperClient
           .convert(res.getBlock()) : null;
-      HdfsFileStatus stat = (res.hasStat()) ? PBHelper.convert(res.getStat())
+      HdfsFileStatus stat = (res.hasStat()) ? PBHelperClient.convert(res.getStat())
           : null;
       return new LastBlockWithStatus(lastBlock, stat);
     } catch (ServiceException e) {
@@ -359,7 +359,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       UnresolvedLinkException, IOException {
     SetPermissionRequestProto req = SetPermissionRequestProto.newBuilder()
         .setSrc(src)
-        .setPermission(PBHelper.convert(permission))
+        .setPermission(PBHelperClient.convert(permission))
         .build();
     try {
       rpcProxy.setPermission(null, req);
@@ -416,7 +416,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       req.addAllFavoredNodes(Arrays.asList(favoredNodes));
     }
     try {
-      return PBHelper.convert(rpcProxy.addBlock(null, req.build()).getBlock());
+      return PBHelperClient.convert(rpcProxy.addBlock(null, req.build()).getBlock());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -441,7 +441,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setClientName(clientName)
         .build();
     try {
-      return PBHelper.convert(rpcProxy.getAdditionalDatanode(null, req)
+      return PBHelperClient.convert(rpcProxy.getAdditionalDatanode(null, req)
           .getBlock());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
@@ -469,7 +469,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
     ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder()
-        .addAllBlocks(Arrays.asList(PBHelper.convertLocatedBlock(blocks)))
+        .addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlock(blocks)))
         .build();
     try {
       rpcProxy.reportBadBlocks(null, req);
@@ -552,7 +552,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       IOException {
     MkdirsRequestProto req = MkdirsRequestProto.newBuilder()
         .setSrc(src)
-        .setMasked(PBHelper.convert(masked))
+        .setMasked(PBHelperClient.convert(masked))
         .setCreateParent(createParent).build();
 
     try {
@@ -574,7 +574,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       GetListingResponseProto result = rpcProxy.getListing(null, req);
       
       if (result.hasDirList()) {
-        return PBHelper.convert(result.getDirList());
+        return PBHelperClient.convert(result.getDirList());
       }
       return null;
     } catch (ServiceException e) {
@@ -610,7 +610,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   public long[] getStats() throws IOException {
     try {
-      return PBHelper.convert(rpcProxy.getFsStats(null,
+      return PBHelperClient.convert(rpcProxy.getFsStats(null,
           VOID_GET_FSSTATUS_REQUEST));
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
@@ -622,9 +622,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throws IOException {
     GetDatanodeReportRequestProto req = GetDatanodeReportRequestProto
         .newBuilder()
-        .setType(PBHelper.convert(type)).build();
+        .setType(PBHelperClient.convert(type)).build();
     try {
-      return PBHelper.convert(
+      return PBHelperClient.convert(
           rpcProxy.getDatanodeReport(null, req).getDiList());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
@@ -636,9 +636,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throws IOException {
     final GetDatanodeStorageReportRequestProto req
         = GetDatanodeStorageReportRequestProto.newBuilder()
-            .setType(PBHelper.convert(type)).build();
+            .setType(PBHelperClient.convert(type)).build();
     try {
-      return PBHelper.convertDatanodeStorageReports(
+      return PBHelperClient.convertDatanodeStorageReports(
           rpcProxy.getDatanodeStorageReport(null, req).getDatanodeStorageReportsList());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
@@ -662,7 +662,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException {
     SetSafeModeRequestProto req = SetSafeModeRequestProto.newBuilder()
-        .setAction(PBHelper.convert(action)).setChecked(isChecked).build();
+        .setAction(PBHelperClient.convert(action)).setChecked(isChecked).build();
     try {
       return rpcProxy.setSafeMode(null, req).getResult();
     } catch (ServiceException e) {
@@ -726,11 +726,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
     final RollingUpgradeRequestProto r = RollingUpgradeRequestProto.newBuilder()
-        .setAction(PBHelper.convert(action)).build();
+        .setAction(PBHelperClient.convert(action)).build();
     try {
       final RollingUpgradeResponseProto proto = rpcProxy.rollingUpgrade(null, r);
       if (proto.hasRollingUpgradeInfo()) {
-        return PBHelper.convert(proto.getRollingUpgradeInfo());
+        return PBHelperClient.convert(proto.getRollingUpgradeInfo());
       }
       return null;
     } catch (ServiceException e) {
@@ -746,7 +746,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     if (cookie != null) 
       req.setCookie(cookie);
     try {
-      return PBHelper.convert(
+      return PBHelperClient.convert(
           rpcProxy.listCorruptFileBlocks(null, req.build()).getCorrupt());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
@@ -772,7 +772,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setSrc(src).build();
     try {
       GetFileInfoResponseProto res = rpcProxy.getFileInfo(null, req);
-      return res.hasFs() ? PBHelper.convert(res.getFs()) : null;
+      return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null;
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -786,7 +786,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     try {
       GetFileLinkInfoResponseProto result = rpcProxy.getFileLinkInfo(null, req);
       return result.hasFs() ?  
-          PBHelper.convert(rpcProxy.getFileLinkInfo(null, req).getFs()) : null;
+          PBHelperClient.convert(rpcProxy.getFileLinkInfo(null, req).getFs()) : null;
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -801,7 +801,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setPath(path)
         .build();
     try {
-      return PBHelper.convert(rpcProxy.getContentSummary(null, req)
+      return PBHelperClient.convert(rpcProxy.getContentSummary(null, req)
           .getSummary());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
@@ -869,7 +869,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     CreateSymlinkRequestProto req = CreateSymlinkRequestProto.newBuilder()
         .setTarget(target)
         .setLink(link)
-        .setDirPerm(PBHelper.convert(dirPerm))
+        .setDirPerm(PBHelperClient.convert(dirPerm))
         .setCreateParent(createParent)
         .build();
     try {
@@ -901,7 +901,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setClientName(clientName)
         .build();
     try {
-      return PBHelper.convert(
+      return PBHelperClient.convert(
           rpcProxy.updateBlockForPipeline(null, req).getBlock());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
@@ -915,7 +915,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setClientName(clientName)
         .setOldBlock(PBHelperClient.convert(oldBlock))
         .setNewBlock(PBHelperClient.convert(newBlock))
-        .addAllNewNodes(Arrays.asList(PBHelper.convert(newNodes)))
+        .addAllNewNodes(Arrays.asList(PBHelperClient.convert(newNodes)))
         .addAllStorageIDs(storageIDs == null ? null : Arrays.asList(storageIDs))
         .build();
     try {
@@ -934,7 +934,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .build();
     try {
       GetDelegationTokenResponseProto resp = rpcProxy.getDelegationToken(null, req);
-      return resp.hasToken() ? PBHelper.convertDelegationToken(resp.getToken())
+      return resp.hasToken() ? PBHelperClient.convertDelegationToken(resp.getToken())
           : null;
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
@@ -993,7 +993,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       GetDataEncryptionKeyResponseProto rsp = rpcProxy.getDataEncryptionKey(
           null, VOID_GET_DATA_ENCRYPTIONKEY_REQUEST);
      return rsp.hasDataEncryptionKey() ? 
-          PBHelper.convert(rsp.getDataEncryptionKey()) : null;
+          PBHelperClient.convert(rsp.getDataEncryptionKey()) : null;
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -1090,7 +1090,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
           .getSnapshottableDirListing(null, req);
       
       if (result.hasSnapshottableDirList()) {
-        return PBHelper.convert(result.getSnapshottableDirList());
+        return PBHelperClient.convert(result.getSnapshottableDirList());
       }
       return null;
     } catch (ServiceException e) {
@@ -1108,7 +1108,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       GetSnapshotDiffReportResponseProto result = 
           rpcProxy.getSnapshotDiffReport(null, req);
     
-      return PBHelper.convert(result.getDiffReport());
+      return PBHelperClient.convert(result.getDiffReport());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -1120,9 +1120,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
     try {
       AddCacheDirectiveRequestProto.Builder builder =
           AddCacheDirectiveRequestProto.newBuilder().
-              setInfo(PBHelper.convert(directive));
+              setInfo(PBHelperClient.convert(directive));
       if (!flags.isEmpty()) {
-        builder.setCacheFlags(PBHelper.convertCacheFlags(flags));
+        builder.setCacheFlags(PBHelperClient.convertCacheFlags(flags));
       }
       return rpcProxy.addCacheDirective(null, builder.build()).getId();
     } catch (ServiceException e) {
@@ -1136,9 +1136,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
     try {
       ModifyCacheDirectiveRequestProto.Builder builder =
           ModifyCacheDirectiveRequestProto.newBuilder().
-              setInfo(PBHelper.convert(directive));
+              setInfo(PBHelperClient.convert(directive));
       if (!flags.isEmpty()) {
-        builder.setCacheFlags(PBHelper.convertCacheFlags(flags));
+        builder.setCacheFlags(PBHelperClient.convertCacheFlags(flags));
       }
       rpcProxy.modifyCacheDirective(null, builder.build());
     } catch (ServiceException e) {
@@ -1169,7 +1169,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
     @Override
     public CacheDirectiveEntry get(int i) {
-      return PBHelper.convert(response.getElements(i));
+      return PBHelperClient.convert(response.getElements(i));
     }
 
     @Override
@@ -1195,7 +1195,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
         rpcProxy.listCacheDirectives(null,
           ListCacheDirectivesRequestProto.newBuilder().
             setPrevId(prevId).
-            setFilter(PBHelper.convert(filter)).
+            setFilter(PBHelperClient.convert(filter)).
             build()));
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
@@ -1206,7 +1206,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   public void addCachePool(CachePoolInfo info) throws IOException {
     AddCachePoolRequestProto.Builder builder = 
         AddCachePoolRequestProto.newBuilder();
-    builder.setInfo(PBHelper.convert(info));
+    builder.setInfo(PBHelperClient.convert(info));
     try {
       rpcProxy.addCachePool(null, builder.build());
     } catch (ServiceException e) {
@@ -1218,7 +1218,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   public void modifyCachePool(CachePoolInfo req) throws IOException {
     ModifyCachePoolRequestProto.Builder builder = 
         ModifyCachePoolRequestProto.newBuilder();
-    builder.setInfo(PBHelper.convert(req));
+    builder.setInfo(PBHelperClient.convert(req));
     try {
       rpcProxy.modifyCachePool(null, builder.build());
     } catch (ServiceException e) {
@@ -1248,7 +1248,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     @Override
     public CachePoolEntry get(int i) {
       CachePoolEntryProto elem = proto.getEntries(i);
-      return PBHelper.convert(elem);
+      return PBHelperClient.convert(elem);
     }
 
     @Override
@@ -1280,7 +1280,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throws IOException {
     ModifyAclEntriesRequestProto req = ModifyAclEntriesRequestProto
         .newBuilder().setSrc(src)
-        .addAllAclSpec(PBHelper.convertAclEntryProto(aclSpec)).build();
+        .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)).build();
     try {
       rpcProxy.modifyAclEntries(null, req);
     } catch (ServiceException e) {
@@ -1293,7 +1293,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throws IOException {
     RemoveAclEntriesRequestProto req = RemoveAclEntriesRequestProto
         .newBuilder().setSrc(src)
-        .addAllAclSpec(PBHelper.convertAclEntryProto(aclSpec)).build();
+        .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)).build();
     try {
       rpcProxy.removeAclEntries(null, req);
     } catch (ServiceException e) {
@@ -1327,7 +1327,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   public void setAcl(String src, List<AclEntry> aclSpec) throws IOException {
     SetAclRequestProto req = SetAclRequestProto.newBuilder()
         .setSrc(src)
-        .addAllAclSpec(PBHelper.convertAclEntryProto(aclSpec))
+        .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec))
         .build();
     try {
       rpcProxy.setAcl(null, req);
@@ -1341,7 +1341,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     GetAclStatusRequestProto req = GetAclStatusRequestProto.newBuilder()
         .setSrc(src).build();
     try {
-      return PBHelper.convert(rpcProxy.getAclStatus(null, req));
+      return PBHelperClient.convert(rpcProxy.getAclStatus(null, req));
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -1375,7 +1375,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       final EncryptionZonesProtos.GetEZForPathResponseProto response =
           rpcProxy.getEZForPath(null, req);
       if (response.hasZone()) {
-        return PBHelper.convert(response.getZone());
+        return PBHelperClient.convert(response.getZone());
       } else {
         return null;
       }
@@ -1397,7 +1397,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       List<EncryptionZone> elements =
           Lists.newArrayListWithCapacity(response.getZonesCount());
       for (EncryptionZoneProto p : response.getZonesList()) {
-        elements.add(PBHelper.convert(p));
+        elements.add(PBHelperClient.convert(p));
       }
       return new BatchedListEntries<EncryptionZone>(elements,
           response.getHasMore());
@@ -1411,8 +1411,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throws IOException {
     SetXAttrRequestProto req = SetXAttrRequestProto.newBuilder()
         .setSrc(src)
-        .setXAttr(PBHelper.convertXAttrProto(xAttr))
-        .setFlag(PBHelper.convert(flag))
+        .setXAttr(PBHelperClient.convertXAttrProto(xAttr))
+        .setFlag(PBHelperClient.convert(flag))
         .build();
     try {
       rpcProxy.setXAttr(null, req);
@@ -1422,16 +1422,16 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
   
   @Override
-  public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs) 
+  public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs)
       throws IOException {
     GetXAttrsRequestProto.Builder builder = GetXAttrsRequestProto.newBuilder();
     builder.setSrc(src);
     if (xAttrs != null) {
-      builder.addAllXAttrs(PBHelper.convertXAttrProto(xAttrs));
+      builder.addAllXAttrs(PBHelperClient.convertXAttrProto(xAttrs));
     }
     GetXAttrsRequestProto req = builder.build();
     try {
-      return PBHelper.convert(rpcProxy.getXAttrs(null, req));
+      return PBHelperClient.convert(rpcProxy.getXAttrs(null, req));
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -1444,7 +1444,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     builder.setSrc(src);
     ListXAttrsRequestProto req = builder.build();
     try {
-      return PBHelper.convert(rpcProxy.listXAttrs(null, req));
+      return PBHelperClient.convert(rpcProxy.listXAttrs(null, req));
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -1454,7 +1454,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   public void removeXAttr(String src, XAttr xAttr) throws IOException {
     RemoveXAttrRequestProto req = RemoveXAttrRequestProto
         .newBuilder().setSrc(src)
-        .setXAttr(PBHelper.convertXAttrProto(xAttr)).build();
+        .setXAttr(PBHelperClient.convertXAttrProto(xAttr)).build();
     try {
       rpcProxy.removeXAttr(null, req);
     } catch (ServiceException e) {
@@ -1465,7 +1465,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   public void checkAccess(String path, FsAction mode) throws IOException {
     CheckAccessRequestProto req = CheckAccessRequestProto.newBuilder()
-        .setPath(path).setMode(PBHelper.convert(mode)).build();
+        .setPath(path).setMode(PBHelperClient.convert(mode)).build();
     try {
       rpcProxy.checkAccess(null, req);
     } catch (ServiceException e) {
@@ -1490,7 +1490,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     GetStoragePolicyRequestProto request = GetStoragePolicyRequestProto
         .newBuilder().setPath(path).build();
     try {
-      return PBHelper.convert(rpcProxy.getStoragePolicy(null, request)
+      return PBHelperClient.convert(rpcProxy.getStoragePolicy(null, request)
           .getStoragePolicy());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
@@ -1502,7 +1502,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     try {
       GetStoragePoliciesResponseProto response = rpcProxy
           .getStoragePolicies(null, VOID_GET_STORAGE_POLICIES_REQUEST);
-      return PBHelper.convertStoragePolicies(response.getPoliciesList());
+      return PBHelperClient.convertStoragePolicies(response.getPoliciesList());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -1523,7 +1523,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     GetEditsFromTxidRequestProto req = GetEditsFromTxidRequestProto.newBuilder()
         .setTxid(txid).build();
     try {
-      return PBHelper.convert(rpcProxy.getEditsFromTxid(null, req));
+      return PBHelperClient.convert(rpcProxy.getEditsFromTxid(null, req));
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
index 0b46927..18f89f8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
@@ -139,7 +139,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
         .setXmitsInProgress(xmitsInProgress).setXceiverCount(xceiverCount)
         .setFailedVolumes(failedVolumes)
         .setRequestFullBlockReportLease(requestFullBlockReportLease);
-    builder.addAllReports(PBHelper.convertStorageReports(reports));
+    builder.addAllReports(PBHelperClient.convertStorageReports(reports));
     if (cacheCapacity != 0) {
       builder.setCacheCapacity(cacheCapacity);
     }
@@ -164,7 +164,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
     }
     RollingUpgradeStatus rollingUpdateStatus = null;
     if (resp.hasRollingUpgradeStatus()) {
-      rollingUpdateStatus = PBHelper.convert(resp.getRollingUpgradeStatus());
+      rollingUpdateStatus = PBHelperClient.convert(resp.getRollingUpgradeStatus());
     }
     return new HeartbeatResponse(cmds, PBHelper.convert(resp.getHaStatus()),
         rollingUpdateStatus, resp.getFullBlockReportLeaseId());
@@ -183,7 +183,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
 
     for (StorageBlockReport r : reports) {
       StorageBlockReportProto.Builder reportBuilder = StorageBlockReportProto
-          .newBuilder().setStorage(PBHelper.convert(r.getStorage()));
+          .newBuilder().setStorage(PBHelperClient.convert(r.getStorage()));
       BlockListAsLongs blocks = r.getBlocks();
       if (useBlocksBuffer) {
         reportBuilder.setNumberOfBlocks(blocks.getNumberOfBlocks());
@@ -240,7 +240,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
       StorageReceivedDeletedBlocksProto.Builder repBuilder = 
           StorageReceivedDeletedBlocksProto.newBuilder();
       repBuilder.setStorageUuid(storageBlock.getStorage().getStorageID());  // Set for wire compatibility.
-      repBuilder.setStorage(PBHelper.convert(storageBlock.getStorage()));
+      repBuilder.setStorage(PBHelperClient.convert(storageBlock.getStorage()));
       for (ReceivedDeletedBlockInfo rdBlock : storageBlock.getBlocks()) {
         repBuilder.addBlocks(PBHelper.convert(rdBlock));
       }
@@ -281,7 +281,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
     ReportBadBlocksRequestProto.Builder builder = ReportBadBlocksRequestProto
         .newBuilder();
     for (int i = 0; i < blocks.length; i++) {
-      builder.addBlocks(i, PBHelper.convert(blocks[i]));
+      builder.addBlocks(i, PBHelperClient.convert(blocks[i]));
     }
     ReportBadBlocksRequestProto req = builder.build();
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
index 5964e15..94d1f0c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
@@ -105,7 +105,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
       HeartbeatRequestProto request) throws ServiceException {
     HeartbeatResponse response;
     try {
-      final StorageReport[] report = PBHelper.convertStorageReports(
+      final StorageReport[] report = PBHelperClient.convertStorageReports(
           request.getReportsList());
       VolumeFailureSummary volumeFailureSummary =
           request.hasVolumeFailureSummary() ? PBHelper.convertVolumeFailureSummary(
@@ -132,7 +132,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
     RollingUpgradeStatus rollingUpdateStatus = response
         .getRollingUpdateStatus();
     if (rollingUpdateStatus != null) {
-      builder.setRollingUpgradeStatus(PBHelper
+      builder.setRollingUpgradeStatus(PBHelperClient
           .convertRollingUpgradeStatus(rollingUpdateStatus));
     }
     builder.setFullBlockReportLeaseId(response.getFullBlockReportLeaseId());
@@ -157,7 +157,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
       } else {
         blocks = BlockListAsLongs.decodeLongs(s.getBlocksList());
       }
-      report[index++] = new StorageBlockReport(PBHelper.convert(s.getStorage()),
+      report[index++] = new StorageBlockReport(PBHelperClient.convert(s.getStorage()),
           blocks);
     }
     try {
@@ -214,7 +214,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
       }
       if (sBlock.hasStorage()) {
         info[i] = new StorageReceivedDeletedBlocks(
-            PBHelper.convert(sBlock.getStorage()), rdBlocks);
+            PBHelperClient.convert(sBlock.getStorage()), rdBlocks);
       } else {
         info[i] = new StorageReceivedDeletedBlocks(sBlock.getStorageUuid(), rdBlocks);
       }
@@ -259,7 +259,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
     List<LocatedBlockProto> lbps = request.getBlocksList();
     LocatedBlock [] blocks = new LocatedBlock [lbps.size()];
     for(int i=0; i<lbps.size(); i++) {
-      blocks[i] = PBHelper.convert(lbps.get(i));
+      blocks[i] = PBHelperClient.convert(lbps.get(i));
     }
     try {
       impl.reportBadBlocks(blocks);
@@ -276,7 +276,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
     List<DatanodeIDProto> dnprotos = request.getNewTaragetsList();
     DatanodeID[] dns = new DatanodeID[dnprotos.size()];
     for (int i = 0; i < dnprotos.size(); i++) {
-      dns[i] = PBHelper.convert(dnprotos.get(i));
+      dns[i] = PBHelperClient.convert(dnprotos.get(i));
     }
     final List<String> sidprotos = request.getNewTargetStoragesList();
     final String[] storageIDs = sidprotos.toArray(new String[sidprotos.size()]);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java
index fb67e36..ff4277c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java
@@ -64,7 +64,7 @@ public class InterDatanodeProtocolServerSideTranslatorPB implements
     } else {
       return InitReplicaRecoveryResponseProto.newBuilder()
           .setReplicaFound(true)
-          .setBlock(PBHelper.convert(r))
+          .setBlock(PBHelperClient.convert(r))
           .setState(PBHelper.convert(r.getOriginalReplicaState())).build();
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java
index 49fdf5d..44e814e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java
@@ -67,7 +67,7 @@ public class JournalProtocolTranslatorPB implements ProtocolMetaInterface,
         .setEpoch(epoch)
         .setFirstTxnId(firstTxnId)
         .setNumTxns(numTxns)
-        .setRecords(PBHelper.getByteString(records))
+        .setRecords(PBHelperClient.getByteString(records))
         .build();
     try {
       rpcProxy.journal(NULL_CONTROLLER, req);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
index dbacded..91ffb1b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto;
@@ -79,7 +78,7 @@ public class NamenodeProtocolServerSideTranslatorPB implements
   @Override
   public GetBlocksResponseProto getBlocks(RpcController unused,
       GetBlocksRequestProto request) throws ServiceException {
-    DatanodeInfo dnInfo = new DatanodeInfo(PBHelper.convert(request
+    DatanodeInfo dnInfo = new DatanodeInfo(PBHelperClient.convert(request
         .getDatanode()));
     BlocksWithLocations blocks;
     try {


[04/50] [abbrv] hadoop git commit: HADOOP-12416. Trash messages should be handled by Logger instead of being delivered on System.out. Contributed by Mingliang Liu.

Posted by zh...@apache.org.
HADOOP-12416. Trash messages should be handled by Logger instead of being delivered on System.out. Contributed by Mingliang Liu.


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

Branch: refs/heads/HDFS-7285
Commit: 77aaf4cf7dd344eb094d9c079e5fb993c6696ebf
Parents: 6c52be7
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Sep 16 10:20:24 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Sep 16 10:20:24 2015 +0900

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/77aaf4cf/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 fe09120..b637358 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -521,6 +521,9 @@ Release 2.8.0 - UNRELEASED
 
     HADOOP-11746. rewrite test-patch.sh (aw)
 
+    HADOOP-12416. Trash messages should be handled by Logger instead of being
+    delivered on System.out. (Mingliang Liu via aajisaka)
+
   NEW FEATURES
 
     HADOOP-11226. Add a configuration to set ipc.Client's traffic class with

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77aaf4cf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Trash.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Trash.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Trash.java
index aae5cf7..13d0ec3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Trash.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Trash.java
@@ -94,8 +94,7 @@ public class Trash extends Configured {
     Trash trash = new Trash(fullyResolvedFs, conf);
     boolean success = trash.moveToTrash(fullyResolvedPath);
     if (success) {
-      System.out.println("Moved: '" + p + "' to trash at: " +
-          trash.getCurrentTrashDir() );
+      LOG.info("Moved: '" + p + "' to trash at: " + trash.getCurrentTrashDir());
     }
     return success;
   }


[26/50] [abbrv] hadoop git commit: YARN-3212. RMNode State Transition Update with DECOMMISSIONING state. (Junping Du via wangda)

Posted by zh...@apache.org.
YARN-3212. RMNode State Transition Update with DECOMMISSIONING state. (Junping Du via wangda)


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

Branch: refs/heads/HDFS-7285
Commit: 9bc913a35c46e65d373c3ae3f01a377e16e8d0ca
Parents: 3f42753
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Sep 18 10:04:17 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Sep 18 10:04:17 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../resourcemanager/NodesListManager.java       |   2 +-
 .../resourcemanager/ResourceTrackerService.java |  20 +-
 .../resourcemanager/rmnode/RMNodeEventType.java |   2 +-
 .../resourcemanager/rmnode/RMNodeImpl.java      | 456 +++++++++++++------
 .../resourcemanager/TestRMNodeTransitions.java  | 196 +++++++-
 .../resourcemanager/webapp/TestNodesPage.java   |   3 +-
 7 files changed, 513 insertions(+), 169 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9bc913a3/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 8fe686d..32bf7dc 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -197,6 +197,9 @@ Release 2.8.0 - UNRELEASED
     YARN-4034. Render cluster Max Priority in scheduler metrics in RM web
     UI. (Rohith Sharma K S via jianhe)
 
+    YARN-3212. RMNode State Transition Update with DECOMMISSIONING state.
+    (Junping Du via wangda)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9bc913a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.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/NodesListManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java
index abea85e..1e8b98a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java
@@ -399,7 +399,7 @@ public class NodesListManager extends CompositeService implements
       NodeId nodeId = entry.getKey();
       if (!isValidNode(nodeId.getHost())) {
         this.rmContext.getDispatcher().getEventHandler().handle(
-            new RMNodeEvent(nodeId, RMNodeEventType.DECOMMISSION_WITH_TIMEOUT));
+            new RMNodeEvent(nodeId, RMNodeEventType.GRACEFUL_DECOMMISSION));
       } else {
         // Recommissioning the nodes
         if (entry.getValue().getState() == NodeState.DECOMMISSIONING

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9bc913a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.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/ResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
index 100e991..7e774c5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
@@ -44,6 +44,7 @@ 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.NodeLabel;
+import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -399,8 +400,10 @@ public class ResourceTrackerService extends AbstractService implements
 
     NodeId nodeId = remoteNodeStatus.getNodeId();
 
-    // 1. Check if it's a valid (i.e. not excluded) node
-    if (!this.nodesListManager.isValidNode(nodeId.getHost())) {
+    // 1. Check if it's a valid (i.e. not excluded) node, if not, see if it is
+    // in decommissioning.
+    if (!this.nodesListManager.isValidNode(nodeId.getHost())
+        && !isNodeInDecommissioning(nodeId)) {
       String message =
           "Disallowed NodeManager nodeId: " + nodeId + " hostname: "
               + nodeId.getHost();
@@ -486,6 +489,19 @@ public class ResourceTrackerService extends AbstractService implements
     return nodeHeartBeatResponse;
   }
 
+  /**
+   * Check if node in decommissioning state.
+   * @param nodeId
+   */
+  private boolean isNodeInDecommissioning(NodeId nodeId) {
+    RMNode rmNode = this.rmContext.getRMNodes().get(nodeId);
+    if (rmNode != null &&
+        rmNode.getState().equals(NodeState.DECOMMISSIONING)) {
+      return true;
+    }
+    return false;
+  }
+
   @SuppressWarnings("unchecked")
   @Override
   public UnRegisterNodeManagerResponse unRegisterNodeManager(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9bc913a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.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/RMNodeEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.java
index 27ba1c0..ad36036 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.java
@@ -24,7 +24,7 @@ public enum RMNodeEventType {
   
   // Source: AdminService
   DECOMMISSION,
-  DECOMMISSION_WITH_TIMEOUT,
+  GRACEFUL_DECOMMISSION,
   RECOMMISSION,
   
   // Source: AdminService, ResourceTrackerService

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9bc913a3/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 7a1ba74..391b6ff 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
@@ -144,101 +144,150 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
                                            RMNodeEventType,
                                            RMNodeEvent>(NodeState.NEW)
 
-     //Transitions from NEW state
-     .addTransition(NodeState.NEW, NodeState.RUNNING, 
-         RMNodeEventType.STARTED, new AddNodeTransition())
-     .addTransition(NodeState.NEW, NodeState.NEW,
-         RMNodeEventType.RESOURCE_UPDATE, 
-         new UpdateNodeResourceWhenUnusableTransition())
-
-     //Transitions from RUNNING state
-     .addTransition(NodeState.RUNNING,
-         EnumSet.of(NodeState.RUNNING, NodeState.UNHEALTHY),
-         RMNodeEventType.STATUS_UPDATE, new StatusUpdateWhenHealthyTransition())
-     .addTransition(NodeState.RUNNING, NodeState.DECOMMISSIONED,
-         RMNodeEventType.DECOMMISSION,
-         new DeactivateNodeTransition(NodeState.DECOMMISSIONED))
-     .addTransition(NodeState.RUNNING, NodeState.LOST,
-         RMNodeEventType.EXPIRE,
-         new DeactivateNodeTransition(NodeState.LOST))
-     .addTransition(NodeState.RUNNING, NodeState.REBOOTED,
-         RMNodeEventType.REBOOTING,
-         new DeactivateNodeTransition(NodeState.REBOOTED))
-     .addTransition(NodeState.RUNNING, NodeState.RUNNING,
-         RMNodeEventType.CLEANUP_APP, new CleanUpAppTransition())
-     .addTransition(NodeState.RUNNING, NodeState.RUNNING,
-         RMNodeEventType.CLEANUP_CONTAINER, new CleanUpContainerTransition())
-     .addTransition(NodeState.RUNNING, NodeState.RUNNING,
-         RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
-         new AddContainersToBeRemovedFromNMTransition())
-     .addTransition(NodeState.RUNNING, NodeState.RUNNING,
-         RMNodeEventType.RECONNECTED, new ReconnectNodeTransition())
-     .addTransition(NodeState.RUNNING, NodeState.RUNNING,
-         RMNodeEventType.RESOURCE_UPDATE, new UpdateNodeResourceWhenRunningTransition())
-     .addTransition(NodeState.RUNNING, NodeState.SHUTDOWN,
-         RMNodeEventType.SHUTDOWN,
-         new DeactivateNodeTransition(NodeState.SHUTDOWN))
-
-     //Transitions from REBOOTED state
-     .addTransition(NodeState.REBOOTED, NodeState.REBOOTED,
-         RMNodeEventType.RESOURCE_UPDATE,
-         new UpdateNodeResourceWhenUnusableTransition())
-         
-     //Transitions from DECOMMISSIONED state
-     .addTransition(NodeState.DECOMMISSIONED, NodeState.DECOMMISSIONED,
-         RMNodeEventType.RESOURCE_UPDATE,
-         new UpdateNodeResourceWhenUnusableTransition())
-     .addTransition(NodeState.DECOMMISSIONED, NodeState.DECOMMISSIONED,
-         RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
-         new AddContainersToBeRemovedFromNMTransition())
-
-     //Transitions from LOST state
-     .addTransition(NodeState.LOST, NodeState.LOST,
-         RMNodeEventType.RESOURCE_UPDATE,
-         new UpdateNodeResourceWhenUnusableTransition())
-     .addTransition(NodeState.LOST, NodeState.LOST,
-         RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
-         new AddContainersToBeRemovedFromNMTransition())
-
-     //Transitions from UNHEALTHY state
-     .addTransition(NodeState.UNHEALTHY,
-         EnumSet.of(NodeState.UNHEALTHY, NodeState.RUNNING),
-         RMNodeEventType.STATUS_UPDATE,
-         new StatusUpdateWhenUnHealthyTransition())
-     .addTransition(NodeState.UNHEALTHY, NodeState.DECOMMISSIONED,
-         RMNodeEventType.DECOMMISSION,
-         new DeactivateNodeTransition(NodeState.DECOMMISSIONED))
-     .addTransition(NodeState.UNHEALTHY, NodeState.LOST,
-         RMNodeEventType.EXPIRE,
-         new DeactivateNodeTransition(NodeState.LOST))
-     .addTransition(NodeState.UNHEALTHY, NodeState.REBOOTED,
-         RMNodeEventType.REBOOTING,
-         new DeactivateNodeTransition(NodeState.REBOOTED))
-     .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
-         RMNodeEventType.RECONNECTED, new ReconnectNodeTransition())
-     .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
-         RMNodeEventType.CLEANUP_APP, new CleanUpAppTransition())
-     .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
-         RMNodeEventType.CLEANUP_CONTAINER, new CleanUpContainerTransition())
-     .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
-         RMNodeEventType.RESOURCE_UPDATE, new UpdateNodeResourceWhenUnusableTransition())
-     .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
-         RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
-         new AddContainersToBeRemovedFromNMTransition())
-     .addTransition(NodeState.UNHEALTHY, NodeState.SHUTDOWN,
-         RMNodeEventType.SHUTDOWN,
-         new DeactivateNodeTransition(NodeState.SHUTDOWN))
-
-     //Transitions from SHUTDOWN state
-     .addTransition(NodeState.SHUTDOWN, NodeState.SHUTDOWN,
-         RMNodeEventType.RESOURCE_UPDATE,
-         new UpdateNodeResourceWhenUnusableTransition())
-     .addTransition(NodeState.SHUTDOWN, NodeState.SHUTDOWN,
-         RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
-         new AddContainersToBeRemovedFromNMTransition())
-
-     // create the topology tables
-     .installTopology(); 
+      //Transitions from NEW state
+      .addTransition(NodeState.NEW, NodeState.RUNNING,
+          RMNodeEventType.STARTED, new AddNodeTransition())
+      .addTransition(NodeState.NEW, NodeState.NEW,
+          RMNodeEventType.RESOURCE_UPDATE,
+          new UpdateNodeResourceWhenUnusableTransition())
+
+      //Transitions from RUNNING state
+      .addTransition(NodeState.RUNNING,
+          EnumSet.of(NodeState.RUNNING, NodeState.UNHEALTHY),
+          RMNodeEventType.STATUS_UPDATE,
+          new StatusUpdateWhenHealthyTransition())
+      .addTransition(NodeState.RUNNING, NodeState.DECOMMISSIONED,
+          RMNodeEventType.DECOMMISSION,
+          new DeactivateNodeTransition(NodeState.DECOMMISSIONED))
+      .addTransition(NodeState.RUNNING, NodeState.DECOMMISSIONING,
+          RMNodeEventType.GRACEFUL_DECOMMISSION,
+          new DecommissioningNodeTransition(NodeState.RUNNING,
+              NodeState.DECOMMISSIONING))
+      .addTransition(NodeState.RUNNING, NodeState.LOST,
+          RMNodeEventType.EXPIRE,
+          new DeactivateNodeTransition(NodeState.LOST))
+      .addTransition(NodeState.RUNNING, NodeState.REBOOTED,
+          RMNodeEventType.REBOOTING,
+          new DeactivateNodeTransition(NodeState.REBOOTED))
+      .addTransition(NodeState.RUNNING, NodeState.RUNNING,
+          RMNodeEventType.CLEANUP_APP, new CleanUpAppTransition())
+      .addTransition(NodeState.RUNNING, NodeState.RUNNING,
+          RMNodeEventType.CLEANUP_CONTAINER, new CleanUpContainerTransition())
+      .addTransition(NodeState.RUNNING, NodeState.RUNNING,
+          RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
+          new AddContainersToBeRemovedFromNMTransition())
+      .addTransition(NodeState.RUNNING, EnumSet.of(NodeState.RUNNING),
+          RMNodeEventType.RECONNECTED, new ReconnectNodeTransition())
+      .addTransition(NodeState.RUNNING, NodeState.RUNNING,
+          RMNodeEventType.RESOURCE_UPDATE, new UpdateNodeResourceWhenRunningTransition())
+      .addTransition(NodeState.RUNNING, NodeState.SHUTDOWN,
+          RMNodeEventType.SHUTDOWN,
+          new DeactivateNodeTransition(NodeState.SHUTDOWN))
+
+      //Transitions from REBOOTED state
+      .addTransition(NodeState.REBOOTED, NodeState.REBOOTED,
+          RMNodeEventType.RESOURCE_UPDATE,
+          new UpdateNodeResourceWhenUnusableTransition())
+
+      //Transitions from DECOMMISSIONED state
+      .addTransition(NodeState.DECOMMISSIONED, NodeState.DECOMMISSIONED,
+          RMNodeEventType.RESOURCE_UPDATE,
+          new UpdateNodeResourceWhenUnusableTransition())
+      .addTransition(NodeState.DECOMMISSIONED, NodeState.DECOMMISSIONED,
+          RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
+          new AddContainersToBeRemovedFromNMTransition())
+
+       //Transitions from DECOMMISSIONING state
+      .addTransition(NodeState.DECOMMISSIONING, NodeState.DECOMMISSIONED,
+          RMNodeEventType.DECOMMISSION,
+          new DeactivateNodeTransition(NodeState.DECOMMISSIONED))
+      .addTransition(NodeState.DECOMMISSIONING, NodeState.RUNNING,
+          RMNodeEventType.RECOMMISSION,
+          new RecommissionNodeTransition(NodeState.RUNNING))
+      .addTransition(NodeState.DECOMMISSIONING,
+          EnumSet.of(NodeState.DECOMMISSIONING, NodeState.DECOMMISSIONED),
+          RMNodeEventType.STATUS_UPDATE,
+          new StatusUpdateWhenHealthyTransition())
+      .addTransition(NodeState.DECOMMISSIONING, NodeState.DECOMMISSIONING,
+          RMNodeEventType.GRACEFUL_DECOMMISSION,
+          new DecommissioningNodeTransition(NodeState.DECOMMISSIONING,
+              NodeState.DECOMMISSIONING))
+      .addTransition(NodeState.DECOMMISSIONING, NodeState.LOST,
+          RMNodeEventType.EXPIRE,
+          new DeactivateNodeTransition(NodeState.LOST))
+      .addTransition(NodeState.DECOMMISSIONING, NodeState.REBOOTED,
+          RMNodeEventType.REBOOTING,
+          new DeactivateNodeTransition(NodeState.REBOOTED))
+
+      .addTransition(NodeState.DECOMMISSIONING, NodeState.DECOMMISSIONING,
+          RMNodeEventType.CLEANUP_APP, new CleanUpAppTransition())
+
+      // TODO (in YARN-3223) update resource when container finished.
+      .addTransition(NodeState.DECOMMISSIONING, NodeState.DECOMMISSIONING,
+          RMNodeEventType.CLEANUP_CONTAINER, new CleanUpContainerTransition())
+      // TODO (in YARN-3223) update resource when container finished.
+      .addTransition(NodeState.DECOMMISSIONING, NodeState.DECOMMISSIONING,
+          RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
+          new AddContainersToBeRemovedFromNMTransition())
+      .addTransition(NodeState.DECOMMISSIONING, EnumSet.of(
+          NodeState.DECOMMISSIONING, NodeState.DECOMMISSIONED),
+          RMNodeEventType.RECONNECTED, new ReconnectNodeTransition())
+      .addTransition(NodeState.DECOMMISSIONING, NodeState.DECOMMISSIONING,
+          RMNodeEventType.RESOURCE_UPDATE,
+          new UpdateNodeResourceWhenRunningTransition())
+
+      //Transitions from LOST state
+      .addTransition(NodeState.LOST, NodeState.LOST,
+          RMNodeEventType.RESOURCE_UPDATE,
+          new UpdateNodeResourceWhenUnusableTransition())
+      .addTransition(NodeState.LOST, NodeState.LOST,
+          RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
+          new AddContainersToBeRemovedFromNMTransition())
+
+      //Transitions from UNHEALTHY state
+      .addTransition(NodeState.UNHEALTHY,
+          EnumSet.of(NodeState.UNHEALTHY, NodeState.RUNNING),
+          RMNodeEventType.STATUS_UPDATE,
+          new StatusUpdateWhenUnHealthyTransition())
+      .addTransition(NodeState.UNHEALTHY, NodeState.DECOMMISSIONED,
+          RMNodeEventType.DECOMMISSION,
+          new DeactivateNodeTransition(NodeState.DECOMMISSIONED))
+      .addTransition(NodeState.UNHEALTHY, NodeState.DECOMMISSIONING,
+          RMNodeEventType.GRACEFUL_DECOMMISSION,
+          new DecommissioningNodeTransition(NodeState.UNHEALTHY,
+              NodeState.DECOMMISSIONING))
+      .addTransition(NodeState.UNHEALTHY, NodeState.LOST,
+          RMNodeEventType.EXPIRE,
+          new DeactivateNodeTransition(NodeState.LOST))
+      .addTransition(NodeState.UNHEALTHY, NodeState.REBOOTED,
+          RMNodeEventType.REBOOTING,
+          new DeactivateNodeTransition(NodeState.REBOOTED))
+      .addTransition(NodeState.UNHEALTHY, EnumSet.of(NodeState.UNHEALTHY),
+          RMNodeEventType.RECONNECTED, new ReconnectNodeTransition())
+      .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
+          RMNodeEventType.CLEANUP_APP, new CleanUpAppTransition())
+      .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
+          RMNodeEventType.CLEANUP_CONTAINER, new CleanUpContainerTransition())
+      .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
+          RMNodeEventType.RESOURCE_UPDATE,
+          new UpdateNodeResourceWhenUnusableTransition())
+      .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
+          RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
+          new AddContainersToBeRemovedFromNMTransition())
+      .addTransition(NodeState.UNHEALTHY, NodeState.SHUTDOWN,
+          RMNodeEventType.SHUTDOWN,
+          new DeactivateNodeTransition(NodeState.SHUTDOWN))
+
+      //Transitions from SHUTDOWN state
+      .addTransition(NodeState.SHUTDOWN, NodeState.SHUTDOWN,
+          RMNodeEventType.RESOURCE_UPDATE,
+          new UpdateNodeResourceWhenUnusableTransition())
+      .addTransition(NodeState.SHUTDOWN, NodeState.SHUTDOWN,
+          RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
+          new AddContainersToBeRemovedFromNMTransition())
+
+      // create the topology tables
+      .installTopology();
 
   private final StateMachine<NodeState, RMNodeEventType,
                              RMNodeEvent> stateMachine;
@@ -265,7 +314,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
     this.writeLock = lock.writeLock();
 
     this.stateMachine = stateMachineFactory.make(this);
-    
+
     this.nodeUpdateQueue = new ConcurrentLinkedQueue<UpdatedContainerInfo>();
 
     this.containerAllocationExpirer = context.getContainerAllocationExpirer();
@@ -291,6 +340,11 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
     return httpPort;
   }
 
+  // Test only
+  public void setHttpPort(int port) {
+    this.httpPort = port;
+  }
+
   @Override
   public NodeId getNodeID() {
     return this.nodeId;
@@ -497,23 +551,35 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
       metrics.decrNumShutdownNMs();
       break;
     default:
-      LOG.debug("Unexpected previous node state");    
+      LOG.debug("Unexpected previous node state");
     }
   }
 
+  // Treats nodes in decommissioning as active nodes
+  // TODO we may want to differentiate active nodes and decommissioning node in
+  // metrics later.
+  private void updateMetricsForGracefulDecommissionOnUnhealthyNode() {
+    ClusterMetrics metrics = ClusterMetrics.getMetrics();
+    metrics.incrNumActiveNodes();
+    metrics.decrNumUnhealthyNMs();
+  }
+
   private void updateMetricsForDeactivatedNode(NodeState initialState,
                                                NodeState finalState) {
     ClusterMetrics metrics = ClusterMetrics.getMetrics();
 
     switch (initialState) {
-      case RUNNING:
-        metrics.decrNumActiveNodes();
-        break;
-      case UNHEALTHY:
-        metrics.decrNumUnhealthyNMs();
-        break;
-      default:
-        LOG.debug("Unexpected inital state");
+    case RUNNING:
+      metrics.decrNumActiveNodes();
+      break;
+    case DECOMMISSIONING:
+      metrics.decrNumActiveNodes();
+      break;
+    case UNHEALTHY:
+      metrics.decrNumUnhealthyNMs();
+      break;
+    default:
+      LOG.debug("Unexpected inital state");
     }
 
     switch (finalState) {
@@ -608,10 +674,10 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
   }
 
   public static class ReconnectNodeTransition implements
-      SingleArcTransition<RMNodeImpl, RMNodeEvent> {
+      MultipleArcTransition<RMNodeImpl, RMNodeEvent, NodeState> {
 
     @Override
-    public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
+    public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) {
       RMNodeReconnectEvent reconnectEvent = (RMNodeReconnectEvent) event;
       RMNode newNode = reconnectEvent.getReconnectedNode();
       rmNode.nodeManagerVersion = newNode.getNodeManagerVersion();
@@ -622,6 +688,12 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
       // No application running on the node, so send node-removal event with 
       // cleaning up old container info.
       if (noRunningApps) {
+        if (rmNode.getState() == NodeState.DECOMMISSIONING) {
+          // When node in decommissioning, and no running apps on this node,
+          // it will return as decommissioned state.
+          deactivateNode(rmNode, NodeState.DECOMMISSIONED);
+          return NodeState.DECOMMISSIONED;
+        }
         rmNode.nodeUpdateQueue.clear();
         rmNode.context.getDispatcher().getEventHandler().handle(
             new NodeRemovedSchedulerEvent(rmNode));
@@ -652,6 +724,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
             rmNode.context.getDispatcher().getEventHandler().handle(
                 new RMNodeStartedEvent(newNode.getNodeID(), null, null));
         }
+
       } else {
         rmNode.httpPort = newNode.getHttpPort();
         rmNode.httpAddress = newNode.getHttpAddress();
@@ -678,17 +751,21 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
                   new NodeResourceUpdateSchedulerEvent(rmNode, ResourceOption
                       .newInstance(newNode.getTotalCapability(), -1)));
         }
+
       }
+      return rmNode.getState();
     }
 
     private void handleNMContainerStatus(
         List<NMContainerStatus> nmContainerStatuses, RMNodeImpl rmnode) {
-      List<ContainerStatus> containerStatuses =
-          new ArrayList<ContainerStatus>();
-      for (NMContainerStatus nmContainerStatus : nmContainerStatuses) {
-        containerStatuses.add(createContainerStatus(nmContainerStatus));
+      if (nmContainerStatuses != null) {
+        List<ContainerStatus> containerStatuses =
+            new ArrayList<ContainerStatus>();
+        for (NMContainerStatus nmContainerStatus : nmContainerStatuses) {
+          containerStatuses.add(createContainerStatus(nmContainerStatus));
+        }
+        rmnode.handleContainerStatus(containerStatuses);
       }
-      rmnode.handleContainerStatus(containerStatuses);
     }
 
     private ContainerStatus createContainerStatus(
@@ -770,31 +847,94 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
 
     @Override
     public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
-      // Inform the scheduler
-      rmNode.nodeUpdateQueue.clear();
-      // If the current state is NodeState.UNHEALTHY
-      // Then node is already been removed from the
-      // Scheduler
-      NodeState initialState = rmNode.getState();
-      if (!initialState.equals(NodeState.UNHEALTHY)) {
-        rmNode.context.getDispatcher().getEventHandler()
-          .handle(new NodeRemovedSchedulerEvent(rmNode));
+      RMNodeImpl.deactivateNode(rmNode, finalState);
+    }
+  }
+
+  /**
+   * Put a node in deactivated (decommissioned) status.
+   * @param rmNode
+   * @param finalState
+   */
+  public static void deactivateNode(RMNodeImpl rmNode, NodeState finalState) {
+
+    reportNodeUnusable(rmNode, finalState);
+
+    // Deactivate the node
+    rmNode.context.getRMNodes().remove(rmNode.nodeId);
+    LOG.info("Deactivating Node " + rmNode.nodeId + " as it is now "
+        + finalState);
+    rmNode.context.getInactiveRMNodes().put(rmNode.nodeId, rmNode);
+  }
+
+  /**
+   * Report node is UNUSABLE and update metrics.
+   * @param rmNode
+   * @param finalState
+   */
+  public static void reportNodeUnusable(RMNodeImpl rmNode,
+      NodeState finalState) {
+    // Inform the scheduler
+    rmNode.nodeUpdateQueue.clear();
+    // If the current state is NodeState.UNHEALTHY
+    // Then node is already been removed from the
+    // Scheduler
+    NodeState initialState = rmNode.getState();
+    if (!initialState.equals(NodeState.UNHEALTHY)) {
+      rmNode.context.getDispatcher().getEventHandler()
+        .handle(new NodeRemovedSchedulerEvent(rmNode));
+    }
+    rmNode.context.getDispatcher().getEventHandler().handle(
+        new NodesListManagerEvent(
+            NodesListManagerEventType.NODE_UNUSABLE, rmNode));
+
+    //Update the metrics
+    rmNode.updateMetricsForDeactivatedNode(initialState, finalState);
+  }
+
+  /**
+   * The transition to put node in decommissioning state.
+   */
+  public static class DecommissioningNodeTransition
+      implements SingleArcTransition<RMNodeImpl, RMNodeEvent> {
+    private final NodeState initState;
+    private final NodeState finalState;
+
+    public DecommissioningNodeTransition(NodeState initState,
+        NodeState finalState) {
+      this.initState = initState;
+      this.finalState = finalState;
+    }
+
+    @Override
+    public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
+      LOG.info("Put Node " + rmNode.nodeId + " in DECOMMISSIONING.");
+      if (initState.equals(NodeState.UNHEALTHY)) {
+        rmNode.updateMetricsForGracefulDecommissionOnUnhealthyNode();
       }
-      rmNode.context.getDispatcher().getEventHandler().handle(
-          new NodesListManagerEvent(
-              NodesListManagerEventType.NODE_UNUSABLE, rmNode));
+      // TODO (in YARN-3223) Keep NM's available resource to be 0
+    }
+  }
+
+  public static class RecommissionNodeTransition
+      implements SingleArcTransition<RMNodeImpl, RMNodeEvent> {
 
-      // Deactivate the node
-      rmNode.context.getRMNodes().remove(rmNode.nodeId);
-      LOG.info("Deactivating Node " + rmNode.nodeId + " as it is now "
-          + finalState);
-      rmNode.context.getInactiveRMNodes().put(rmNode.nodeId, rmNode);
+    private final NodeState finalState;
+    public RecommissionNodeTransition(NodeState finalState) {
+      this.finalState = finalState;
+    }
 
-      //Update the metrics
-      rmNode.updateMetricsForDeactivatedNode(initialState, finalState);
+    @Override
+    public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
+      LOG.info("Node " + rmNode.nodeId + " in DECOMMISSIONING is " +
+          "recommissioned back to RUNNING.");
+      // TODO handle NM resource resume in YARN-3223.
     }
   }
 
+  /**
+   * Status update transition when node is healthy.
+   */
   public static class StatusUpdateWhenHealthyTransition implements
       MultipleArcTransition<RMNodeImpl, RMNodeEvent, NodeState> {
     @Override
@@ -805,25 +945,44 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
       // Switch the last heartbeatresponse.
       rmNode.latestNodeHeartBeatResponse = statusEvent.getLatestResponse();
 
-      NodeHealthStatus remoteNodeHealthStatus = 
+      NodeHealthStatus remoteNodeHealthStatus =
           statusEvent.getNodeHealthStatus();
       rmNode.setHealthReport(remoteNodeHealthStatus.getHealthReport());
       rmNode.setLastHealthReportTime(
           remoteNodeHealthStatus.getLastHealthReportTime());
+      NodeState initialState = rmNode.getState();
+      boolean isNodeDecommissioning =
+          initialState.equals(NodeState.DECOMMISSIONING);
       if (!remoteNodeHealthStatus.getIsNodeHealthy()) {
-        LOG.info("Node " + rmNode.nodeId + " reported UNHEALTHY with details: "
-            + remoteNodeHealthStatus.getHealthReport());
-        rmNode.nodeUpdateQueue.clear();
-        // Inform the scheduler
-        rmNode.context.getDispatcher().getEventHandler().handle(
-            new NodeRemovedSchedulerEvent(rmNode));
-        rmNode.context.getDispatcher().getEventHandler().handle(
-            new NodesListManagerEvent(
-                NodesListManagerEventType.NODE_UNUSABLE, rmNode));
-        // Update metrics
-        rmNode.updateMetricsForDeactivatedNode(rmNode.getState(),
-            NodeState.UNHEALTHY);
-        return NodeState.UNHEALTHY;
+        LOG.info("Node " + rmNode.nodeId +
+            " reported UNHEALTHY with details: " +
+            remoteNodeHealthStatus.getHealthReport());
+        // if a node in decommissioning receives an unhealthy report,
+        // it will keep decommissioning.
+        if (isNodeDecommissioning) {
+          return NodeState.DECOMMISSIONING;
+        } else {
+          reportNodeUnusable(rmNode, NodeState.UNHEALTHY);
+          return NodeState.UNHEALTHY;
+        }
+      }
+      if (isNodeDecommissioning) {
+        List<ApplicationId> runningApps = rmNode.getRunningApps();
+
+        List<ApplicationId> keepAliveApps = statusEvent.getKeepAliveAppIds();
+
+        // no running (and keeping alive) app on this node, get it
+        // decommissioned.
+        // TODO may need to check no container is being scheduled on this node
+        // as well.
+        if ((runningApps == null || runningApps.size() == 0)
+            && (keepAliveApps == null || keepAliveApps.size() == 0)) {
+          RMNodeImpl.deactivateNode(rmNode, NodeState.DECOMMISSIONED);
+          return NodeState.DECOMMISSIONED;
+        }
+
+        // TODO (in YARN-3223) if node in decommissioning, get node resource
+        // updated if container get finished (keep available resource to be 0)
       }
 
       rmNode.handleContainerStatus(statusEvent.getContainers());
@@ -848,7 +1007,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
           statusEvent.getKeepAliveAppIds());
       }
 
-      return NodeState.RUNNING;
+      return initialState;
     }
   }
 
@@ -857,11 +1016,12 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
 
     @Override
     public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) {
-      RMNodeStatusEvent statusEvent = (RMNodeStatusEvent) event;
+      RMNodeStatusEvent statusEvent = (RMNodeStatusEvent)event;
 
       // Switch the last heartbeatresponse.
       rmNode.latestNodeHeartBeatResponse = statusEvent.getLatestResponse();
-      NodeHealthStatus remoteNodeHealthStatus = statusEvent.getNodeHealthStatus();
+      NodeHealthStatus remoteNodeHealthStatus =
+          statusEvent.getNodeHealthStatus();
       rmNode.setHealthReport(remoteNodeHealthStatus.getHealthReport());
       rmNode.setLastHealthReportTime(
           remoteNodeHealthStatus.getLastHealthReportTime());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9bc913a3/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 61c6166..a6e1575 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
@@ -29,7 +29,9 @@ import static org.mockito.Mockito.when;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Random;
 
+import org.apache.hadoop.net.Node;
 import org.apache.hadoop.util.HostsFileReader;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -75,7 +77,7 @@ import org.mockito.stubbing.Answer;
 public class TestRMNodeTransitions {
 
   RMNodeImpl node;
-  
+
   private RMContext rmContext;
   private YarnScheduler scheduler;
 
@@ -168,6 +170,42 @@ public class TestRMNodeTransitions {
     return event;
   }
   
+  private RMNodeStatusEvent getMockRMNodeStatusEventWithRunningApps() {
+    NodeHeartbeatResponse response = mock(NodeHeartbeatResponse.class);
+
+    NodeHealthStatus healthStatus = mock(NodeHealthStatus.class);
+    Boolean yes = new Boolean(true);
+    doReturn(yes).when(healthStatus).getIsNodeHealthy();
+
+    RMNodeStatusEvent event = mock(RMNodeStatusEvent.class);
+    doReturn(healthStatus).when(event).getNodeHealthStatus();
+    doReturn(response).when(event).getLatestResponse();
+    doReturn(RMNodeEventType.STATUS_UPDATE).when(event).getType();
+    doReturn(getAppIdList()).when(event).getKeepAliveAppIds();
+    return event;
+  }
+
+  private List<ApplicationId> getAppIdList() {
+    List<ApplicationId> appIdList = new ArrayList<ApplicationId>();
+    appIdList.add(BuilderUtils.newApplicationId(0, 0));
+    return appIdList;
+  }
+
+  private RMNodeStatusEvent getMockRMNodeStatusEventWithoutRunningApps() {
+    NodeHeartbeatResponse response = mock(NodeHeartbeatResponse.class);
+
+    NodeHealthStatus healthStatus = mock(NodeHealthStatus.class);
+    Boolean yes = new Boolean(true);
+    doReturn(yes).when(healthStatus).getIsNodeHealthy();
+
+    RMNodeStatusEvent event = mock(RMNodeStatusEvent.class);
+    doReturn(healthStatus).when(event).getNodeHealthStatus();
+    doReturn(response).when(event).getLatestResponse();
+    doReturn(RMNodeEventType.STATUS_UPDATE).when(event).getType();
+    doReturn(null).when(event).getKeepAliveAppIds();
+    return event;
+  }
+
   @Test (timeout = 5000)
   public void testExpiredContainer() {
     // Start the node
@@ -195,7 +233,33 @@ public class TestRMNodeTransitions {
      */
     verify(scheduler,times(2)).handle(any(NodeUpdateSchedulerEvent.class));     
   }
-  
+
+  @Test
+  public void testStatusUpdateOnDecommissioningNode(){
+    RMNodeImpl node = getDecommissioningNode();
+    Assert.assertEquals(NodeState.DECOMMISSIONING, node.getState());
+    // Verify node in DECOMMISSIONING won't be changed by status update
+    // with running apps
+    RMNodeStatusEvent statusEvent = getMockRMNodeStatusEventWithRunningApps();
+    node.handle(statusEvent);
+    Assert.assertEquals(NodeState.DECOMMISSIONING, node.getState());
+
+    // Verify node in DECOMMISSIONING will be changed by status update
+    // without running apps
+    statusEvent = getMockRMNodeStatusEventWithoutRunningApps();
+    node.handle(statusEvent);
+    Assert.assertEquals(NodeState.DECOMMISSIONED, node.getState());
+  }
+
+  @Test
+  public void testRecommissionNode(){
+    RMNodeImpl node = getDecommissioningNode();
+    Assert.assertEquals(NodeState.DECOMMISSIONING, node.getState());
+    node.handle(new RMNodeEvent(node.getNodeID(),
+        RMNodeEventType.RECOMMISSION));
+    Assert.assertEquals(NodeState.RUNNING, node.getState());
+  }
+
   @Test (timeout = 5000)
   public void testContainerUpdate() throws InterruptedException{
     //Start the node
@@ -253,9 +317,9 @@ public class TestRMNodeTransitions {
     Assert.assertEquals(completedContainerIdFromNode2_1,completedContainers.get(0)
         .getContainerId()); 
     Assert.assertEquals(completedContainerIdFromNode2_2,completedContainers.get(1)
-        .getContainerId());   
+        .getContainerId());
   }
-  
+
   @Test (timeout = 5000)
   public void testStatusChange(){
     //Start the node
@@ -292,7 +356,7 @@ public class TestRMNodeTransitions {
     node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.EXPIRE));
     Assert.assertEquals(0, node.getQueueSize());
   }
-  
+
   @Test
   public void testRunningExpire() {
     RMNodeImpl node = getRunningNode();
@@ -375,7 +439,7 @@ public class TestRMNodeTransitions {
         initialRebooted, cm.getNumRebootedNMs());
     Assert.assertEquals(NodeState.LOST, node.getState());
   }
-  
+
   @Test
   public void testUnhealthyExpireForSchedulerRemove() {
     RMNodeImpl node = getUnhealthyNode();
@@ -408,6 +472,28 @@ public class TestRMNodeTransitions {
   }
 
   @Test
+  public void testDecommissionOnDecommissioningNode() {
+    RMNodeImpl node = getDecommissioningNode();
+    ClusterMetrics cm = ClusterMetrics.getMetrics();
+    int initialActive = cm.getNumActiveNMs();
+    int initialLost = cm.getNumLostNMs();
+    int initialUnhealthy = cm.getUnhealthyNMs();
+    int initialDecommissioned = cm.getNumDecommisionedNMs();
+    int initialRebooted = cm.getNumRebootedNMs();
+    node.handle(new RMNodeEvent(node.getNodeID(),
+        RMNodeEventType.DECOMMISSION));
+    Assert.assertEquals("Active Nodes", initialActive - 1, cm.getNumActiveNMs());
+    Assert.assertEquals("Lost Nodes", initialLost, cm.getNumLostNMs());
+    Assert.assertEquals("Unhealthy Nodes",
+        initialUnhealthy, cm.getUnhealthyNMs());
+    Assert.assertEquals("Decommissioned Nodes",
+        initialDecommissioned + 1, cm.getNumDecommisionedNMs());
+    Assert.assertEquals("Rebooted Nodes",
+        initialRebooted, cm.getNumRebootedNMs());
+    Assert.assertEquals(NodeState.DECOMMISSIONED, node.getState());
+  }
+
+  @Test
   public void testUnhealthyDecommission() {
     RMNodeImpl node = getUnhealthyNode();
     ClusterMetrics cm = ClusterMetrics.getMetrics();
@@ -429,6 +515,30 @@ public class TestRMNodeTransitions {
     Assert.assertEquals(NodeState.DECOMMISSIONED, node.getState());
   }
 
+  // Test Decommissioning on a unhealthy node will make it decommissioning.
+  @Test
+  public void testUnhealthyDecommissioning() {
+    RMNodeImpl node = getUnhealthyNode();
+    ClusterMetrics cm = ClusterMetrics.getMetrics();
+    int initialActive = cm.getNumActiveNMs();
+    int initialLost = cm.getNumLostNMs();
+    int initialUnhealthy = cm.getUnhealthyNMs();
+    int initialDecommissioned = cm.getNumDecommisionedNMs();
+    int initialRebooted = cm.getNumRebootedNMs();
+    node.handle(new RMNodeEvent(node.getNodeID(),
+        RMNodeEventType.GRACEFUL_DECOMMISSION));
+    Assert.assertEquals("Active Nodes", initialActive + 1,
+        cm.getNumActiveNMs());
+    Assert.assertEquals("Lost Nodes", initialLost, cm.getNumLostNMs());
+    Assert.assertEquals("Unhealthy Nodes",
+        initialUnhealthy - 1, cm.getUnhealthyNMs());
+    Assert.assertEquals("Decommissioned Nodes", initialDecommissioned,
+        cm.getNumDecommisionedNMs());
+    Assert.assertEquals("Rebooted Nodes",
+        initialRebooted, cm.getNumRebootedNMs());
+    Assert.assertEquals(NodeState.DECOMMISSIONING, node.getState());
+  }
+
   @Test
   public void testRunningRebooting() {
     RMNodeImpl node = getRunningNode();
@@ -567,6 +677,14 @@ public class TestRMNodeTransitions {
     return node;
   }
 
+  private RMNodeImpl getDecommissioningNode() {
+    RMNodeImpl node = getRunningNode();
+    node.handle(new RMNodeEvent(node.getNodeID(),
+        RMNodeEventType.GRACEFUL_DECOMMISSION));
+    Assert.assertEquals(NodeState.DECOMMISSIONING, node.getState());
+    return node;
+  }
+
   private RMNodeImpl getUnhealthyNode() {
     RMNodeImpl node = getRunningNode();
     NodeHealthStatus status = NodeHealthStatus.newInstance(false, "sick",
@@ -577,20 +695,19 @@ public class TestRMNodeTransitions {
     return node;
   }
 
-
   private RMNodeImpl getNewNode() {
     NodeId nodeId = BuilderUtils.newNodeId("localhost", 0);
     RMNodeImpl node = new RMNodeImpl(nodeId, rmContext, null, 0, 0, null, null, null);
     return node;
   }
-  
+
   private RMNodeImpl getNewNode(Resource capability) {
     NodeId nodeId = BuilderUtils.newNodeId("localhost", 0);
     RMNodeImpl node = new RMNodeImpl(nodeId, rmContext, null, 0, 0, null, 
         capability, null);
     return node;
   }
-  
+
   private RMNodeImpl getRebootedNode() {
     NodeId nodeId = BuilderUtils.newNodeId("localhost", 0);
     Resource capability = Resource.newInstance(4096, 4);
@@ -650,7 +767,39 @@ public class TestRMNodeTransitions {
     Assert.assertEquals(NodesListManagerEventType.NODE_USABLE,
         nodesListManagerEvent.getType());
   }
-  
+
+  @Test
+  public void testReconnectOnDecommissioningNode() {
+    RMNodeImpl node = getDecommissioningNode();
+
+    // Reconnect event with running app
+    node.handle(new RMNodeReconnectEvent(node.getNodeID(), node,
+        getAppIdList(), null));
+    // still decommissioning
+    Assert.assertEquals(NodeState.DECOMMISSIONING, node.getState());
+
+    // Reconnect event without any running app
+    node.handle(new RMNodeReconnectEvent(node.getNodeID(), node, null, null));
+    Assert.assertEquals(NodeState.DECOMMISSIONED, node.getState());
+  }
+
+  @Test
+  public void testReconnectWithNewPortOnDecommissioningNode() {
+    RMNodeImpl node = getDecommissioningNode();
+    Random r= new Random();
+    node.setHttpPort(r.nextInt(10000));
+    // Reconnect event with running app
+    node.handle(new RMNodeReconnectEvent(node.getNodeID(), node,
+        getAppIdList(), null));
+    // still decommissioning
+    Assert.assertEquals(NodeState.DECOMMISSIONING, node.getState());
+
+    node.setHttpPort(r.nextInt(10000));
+    // Reconnect event without any running app
+    node.handle(new RMNodeReconnectEvent(node.getNodeID(), node, null, null));
+    Assert.assertEquals(NodeState.DECOMMISSIONED, node.getState());
+  }
+
   @Test
   public void testResourceUpdateOnRunningNode() {
     RMNodeImpl node = getRunningNode();
@@ -658,18 +807,23 @@ public class TestRMNodeTransitions {
     assertEquals("Memory resource is not match.", oldCapacity.getMemory(), 4096);
     assertEquals("CPU resource is not match.", oldCapacity.getVirtualCores(), 4);
     node.handle(new RMNodeResourceUpdateEvent(node.getNodeID(),
-        ResourceOption.newInstance(Resource.newInstance(2048, 2), 
+        ResourceOption.newInstance(Resource.newInstance(2048, 2),
             ResourceOption.OVER_COMMIT_TIMEOUT_MILLIS_DEFAULT)));
     Resource newCapacity = node.getTotalCapability();
     assertEquals("Memory resource is not match.", newCapacity.getMemory(), 2048);
     assertEquals("CPU resource is not match.", newCapacity.getVirtualCores(), 2);
-    
+
     Assert.assertEquals(NodeState.RUNNING, node.getState());
     Assert.assertNotNull(nodesListManagerEvent);
     Assert.assertEquals(NodesListManagerEventType.NODE_USABLE,
         nodesListManagerEvent.getType());
   }
-  
+
+  @Test
+  public void testDecommissioningOnRunningNode(){
+    getDecommissioningNode();
+  }
+
   @Test
   public void testResourceUpdateOnNewNode() {
     RMNodeImpl node = getNewNode(Resource.newInstance(4096, 4));
@@ -682,10 +836,10 @@ public class TestRMNodeTransitions {
     Resource newCapacity = node.getTotalCapability();
     assertEquals("Memory resource is not match.", newCapacity.getMemory(), 2048);
     assertEquals("CPU resource is not match.", newCapacity.getVirtualCores(), 2);
-    
+
     Assert.assertEquals(NodeState.NEW, node.getState());
   }
-  
+
   @Test
   public void testResourceUpdateOnRebootedNode() {
     RMNodeImpl node = getRebootedNode();
@@ -702,6 +856,18 @@ public class TestRMNodeTransitions {
     Assert.assertEquals(NodeState.REBOOTED, node.getState());
   }
 
+  // Test unhealthy report on a decommissioning node will make it
+  // keep decommissioning.
+  @Test
+  public void testDecommissioningUnhealthy() {
+    RMNodeImpl node = getDecommissioningNode();
+    NodeHealthStatus status = NodeHealthStatus.newInstance(false, "sick",
+        System.currentTimeMillis());
+    node.handle(new RMNodeStatusEvent(node.getNodeID(), status,
+        new ArrayList<ContainerStatus>(), null, null));
+    Assert.assertEquals(NodeState.DECOMMISSIONING, node.getState());
+  }
+
   @Test
   public void testReconnnectUpdate() {
     final String nmVersion1 = "nm version 1";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9bc913a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java
index b70fdc1..458b240 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java
@@ -43,8 +43,7 @@ public class TestNodesPage {
   final int numberOfNodesPerRack = 8;
   // The following is because of the way TestRMWebApp.mockRMContext creates
   // nodes.
-  final int numberOfLostNodesPerRack = numberOfNodesPerRack
-      / NodeState.values().length;
+  final int numberOfLostNodesPerRack = 1;
 
   // Number of Actual Table Headers for NodesPage.NodesBlock might change in
   // future. In that case this value should be adjusted to the new value.


[41/50] [abbrv] hadoop git commit: HDFS-9111. Move hdfs-client protobuf convert methods from PBHelper to PBHelperClient. Contributed by Mingliang Liu.

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/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 cf55445..3de4513 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
@@ -17,97 +17,21 @@
  */
 package org.apache.hadoop.hdfs.protocolPB;
 
-import static com.google.common.base.Preconditions.checkNotNull;
-import static org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos
-    .EncryptionZoneProto;
-import static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CryptoProtocolVersionProto;
-
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.EnumSet;
 import java.util.List;
 
-import org.apache.hadoop.fs.CacheFlag;
-import org.apache.hadoop.fs.ContentSummary;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FsServerDefaults;
-import org.apache.hadoop.fs.Path;
+import com.google.protobuf.ByteString;
+
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.fs.XAttr;
-import org.apache.hadoop.fs.XAttrSetFlag;
-import org.apache.hadoop.fs.permission.AclEntry;
-import org.apache.hadoop.fs.permission.AclEntryScope;
-import org.apache.hadoop.fs.permission.AclEntryType;
-import org.apache.hadoop.fs.permission.AclStatus;
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.proto.HAServiceProtocolProtos;
 import org.apache.hadoop.hdfs.DFSUtilClient;
-import org.apache.hadoop.hdfs.inotify.EventBatch;
-import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
-import org.apache.hadoop.hdfs.inotify.Event;
-import org.apache.hadoop.hdfs.inotify.EventBatchList;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
-import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
-import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
-import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
-import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
-import org.apache.hadoop.hdfs.protocol.CachePoolStats;
-import org.apache.hadoop.crypto.CipherOption;
-import org.apache.hadoop.crypto.CipherSuite;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
-import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
-import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo;
-import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.fs.FileEncryptionInfo;
-import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
-import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
-import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
-import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
-import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
-import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
-import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto;
-import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryScopeProto;
-import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryTypeProto;
-import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.FsActionProto;
-import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclStatusProto;
-import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveEntryProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveStatsProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheFlagProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolStatsProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateFlagProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeReportTypeProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeStorageReportProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeActionProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockIdCommandProto;
@@ -121,34 +45,15 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDele
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlocksWithLocationsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CipherOptionProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ContentSummaryProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CorruptFileBlocksProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DataEncryptionKeyProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfosProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeLocalInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto.StorageState;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsPermissionProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsServerDefaultsProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto.FileType;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlocksProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.NamenodeRoleProto;
@@ -157,28 +62,13 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RollingUpgradeStatusProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportEntryProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryListingProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryStatusProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypesProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
-import org.apache.hadoop.hdfs.protocol.proto.InotifyProtos;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrProto;
-import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrProto.XAttrNamespaceProto;
-import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrSetFlagProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockKey;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
@@ -196,9 +86,6 @@ import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
 import org.apache.hadoop.hdfs.server.protocol.JournalInfo;
 import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
@@ -211,18 +98,7 @@ import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStat
 import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
-import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
-import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
-import org.apache.hadoop.io.EnumSetWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
-import org.apache.hadoop.security.token.Token;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.primitives.Shorts;
-import com.google.protobuf.ByteString;
 
 /**
  * Utilities for converting protobuf classes to and from implementation classes
@@ -231,33 +107,19 @@ import com.google.protobuf.ByteString;
  * Note that when converting from an internal type to protobuf type, the
  * converter never return null for protobuf type. The check for internal type
  * being null must be done before calling the convert() method.
+ *
+ * For those helper methods that convert HDFS client-side data structures from
+ * and to protobuf, see {@link PBHelperClient}.
  */
 public class PBHelper {
   private static final RegisterCommandProto REG_CMD_PROTO = 
       RegisterCommandProto.newBuilder().build();
   private static final RegisterCommand REG_CMD = new RegisterCommand();
 
-  private static final AclEntryScope[] ACL_ENTRY_SCOPE_VALUES =
-      AclEntryScope.values();
-  private static final AclEntryType[] ACL_ENTRY_TYPE_VALUES =
-      AclEntryType.values();
-  private static final FsAction[] FSACTION_VALUES =
-      FsAction.values();
-  private static final XAttr.NameSpace[] XATTR_NAMESPACE_VALUES = 
-      XAttr.NameSpace.values();
-
   private PBHelper() {
     /** Hidden constructor */
   }
 
-  public static ByteString getByteString(byte[] bytes) {
-    return ByteString.copyFrom(bytes);
-  }
-
-  private static <T extends Enum<T>, U extends Enum<U>> U castEnum(T from, U[] to) {
-    return to[from.ordinal()];
-  }
-
   public static NamenodeRole convert(NamenodeRoleProto role) {
     switch (role) {
     case NAMENODE:
@@ -282,65 +144,6 @@ public class PBHelper {
     return null;
   }
 
-  public static BlockStoragePolicy[] convertStoragePolicies(
-      List<BlockStoragePolicyProto> policyProtos) {
-    if (policyProtos == null || policyProtos.size() == 0) {
-      return new BlockStoragePolicy[0];
-    }
-    BlockStoragePolicy[] policies = new BlockStoragePolicy[policyProtos.size()];
-    int i = 0;
-    for (BlockStoragePolicyProto proto : policyProtos) {
-      policies[i++] = convert(proto);
-    }
-    return policies;
-  }
-
-  public static BlockStoragePolicy convert(BlockStoragePolicyProto proto) {
-    List<StorageTypeProto> cList = proto.getCreationPolicy()
-        .getStorageTypesList();
-    StorageType[] creationTypes = convertStorageTypes(cList, cList.size());
-    List<StorageTypeProto> cfList = proto.hasCreationFallbackPolicy() ? proto
-        .getCreationFallbackPolicy().getStorageTypesList() : null;
-    StorageType[] creationFallbackTypes = cfList == null ? StorageType
-        .EMPTY_ARRAY : convertStorageTypes(cfList, cfList.size());
-    List<StorageTypeProto> rfList = proto.hasReplicationFallbackPolicy() ?
-        proto.getReplicationFallbackPolicy().getStorageTypesList() : null;
-    StorageType[] replicationFallbackTypes = rfList == null ? StorageType
-        .EMPTY_ARRAY : convertStorageTypes(rfList, rfList.size());
-    return new BlockStoragePolicy((byte) proto.getPolicyId(), proto.getName(),
-        creationTypes, creationFallbackTypes, replicationFallbackTypes);
-  }
-
-  public static BlockStoragePolicyProto convert(BlockStoragePolicy policy) {
-    BlockStoragePolicyProto.Builder builder = BlockStoragePolicyProto
-        .newBuilder().setPolicyId(policy.getId()).setName(policy.getName());
-    // creation storage types
-    StorageTypesProto creationProto = convert(policy.getStorageTypes());
-    Preconditions.checkArgument(creationProto != null);
-    builder.setCreationPolicy(creationProto);
-    // creation fallback
-    StorageTypesProto creationFallbackProto = convert(
-        policy.getCreationFallbacks());
-    if (creationFallbackProto != null) {
-      builder.setCreationFallbackPolicy(creationFallbackProto);
-    }
-    // replication fallback
-    StorageTypesProto replicationFallbackProto = convert(
-        policy.getReplicationFallbacks());
-    if (replicationFallbackProto != null) {
-      builder.setReplicationFallbackPolicy(replicationFallbackProto);
-    }
-    return builder.build();
-  }
-
-  public static StorageTypesProto convert(StorageType[] types) {
-    if (types == null || types.length == 0) {
-      return null;
-    }
-    List<StorageTypeProto> list = PBHelperClient.convertStorageTypes(types);
-    return StorageTypesProto.newBuilder().addAllStorageTypes(list).build();
-  }
-
   public static StorageInfoProto convert(StorageInfo info) {
     return StorageInfoProto.newBuilder().setClusterID(info.getClusterID())
         .setCTime(info.getCTime()).setLayoutVersion(info.getLayoutVersion())
@@ -365,49 +168,9 @@ public class PBHelper {
         si, convert(reg.getRole()));
   }
 
-  // DatanodeId
-  public static DatanodeID convert(DatanodeIDProto dn) {
-    return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getDatanodeUuid(),
-        dn.getXferPort(), dn.getInfoPort(), dn.hasInfoSecurePort() ? dn
-        .getInfoSecurePort() : 0, dn.getIpcPort());
-  }
-
-  // Arrays of DatanodeId
-  public static DatanodeIDProto[] convert(DatanodeID[] did) {
-    if (did == null)
-      return null;
-    final int len = did.length;
-    DatanodeIDProto[] result = new DatanodeIDProto[len];
-    for (int i = 0; i < len; ++i) {
-      result[i] = PBHelperClient.convert(did[i]);
-    }
-    return result;
-  }
-  
-  public static DatanodeID[] convert(DatanodeIDProto[] did) {
-    if (did == null) return null;
-    final int len = did.length;
-    DatanodeID[] result = new DatanodeID[len];
-    for (int i = 0; i < len; ++i) {
-      result[i] = convert(did[i]);
-    }
-    return result;
-  }
-  
-  // Block
-  public static BlockProto convert(Block b) {
-    return BlockProto.newBuilder().setBlockId(b.getBlockId())
-        .setGenStamp(b.getGenerationStamp()).setNumBytes(b.getNumBytes())
-        .build();
-  }
-
-  public static Block convert(BlockProto b) {
-    return new Block(b.getBlockId(), b.getNumBytes(), b.getGenStamp());
-  }
-
   public static BlockWithLocationsProto convert(BlockWithLocations blk) {
     return BlockWithLocationsProto.newBuilder()
-        .setBlock(convert(blk.getBlock()))
+        .setBlock(PBHelperClient.convert(blk.getBlock()))
         .addAllDatanodeUuids(Arrays.asList(blk.getDatanodeUuids()))
         .addAllStorageUuids(Arrays.asList(blk.getStorageIDs()))
         .addAllStorageTypes(PBHelperClient.convertStorageTypes(blk.getStorageTypes()))
@@ -418,10 +181,10 @@ public class PBHelper {
     final List<String> datanodeUuids = b.getDatanodeUuidsList();
     final List<String> storageUuids = b.getStorageUuidsList();
     final List<StorageTypeProto> storageTypes = b.getStorageTypesList();
-    return new BlockWithLocations(convert(b.getBlock()),
+    return new BlockWithLocations(PBHelperClient.convert(b.getBlock()),
         datanodeUuids.toArray(new String[datanodeUuids.size()]),
         storageUuids.toArray(new String[storageUuids.size()]),
-        convertStorageTypes(storageTypes, storageUuids.size()));
+        PBHelperClient.convertStorageTypes(storageTypes, storageUuids.size()));
   }
 
   public static BlocksWithLocationsProto convert(BlocksWithLocations blks) {
@@ -445,7 +208,7 @@ public class PBHelper {
 
   public static BlockKeyProto convert(BlockKey key) {
     byte[] encodedKey = key.getEncodedKey();
-    ByteString keyBytes = ByteString.copyFrom(encodedKey == null ? 
+    ByteString keyBytes = ByteString.copyFrom(encodedKey == null ?
         DFSUtilClient.EMPTY_BYTES : encodedKey);
     return BlockKeyProto.newBuilder().setKeyId(key.getKeyId())
         .setKeyBytes(keyBytes).setExpiryDate(key.getExpiryDate()).build();
@@ -570,185 +333,22 @@ public class PBHelper {
     if (b == null) {
       return null;
     }
-    LocatedBlockProto lb = PBHelper.convert((LocatedBlock)b);
+    LocatedBlockProto lb = PBHelperClient.convert((LocatedBlock) b);
     RecoveringBlockProto.Builder builder = RecoveringBlockProto.newBuilder();
     builder.setBlock(lb).setNewGenStamp(b.getNewGenerationStamp());
     if(b.getNewBlock() != null)
-      builder.setTruncateBlock(PBHelper.convert(b.getNewBlock()));
+      builder.setTruncateBlock(PBHelperClient.convert(b.getNewBlock()));
     return builder.build();
   }
 
   public static RecoveringBlock convert(RecoveringBlockProto b) {
     ExtendedBlock block = PBHelperClient.convert(b.getBlock().getB());
-    DatanodeInfo[] locs = convert(b.getBlock().getLocsList());
+    DatanodeInfo[] locs = PBHelperClient.convert(b.getBlock().getLocsList());
     return (b.hasTruncateBlock()) ?
-        new RecoveringBlock(block, locs, PBHelper.convert(b.getTruncateBlock())) :
+        new RecoveringBlock(block, locs, PBHelperClient.convert(b.getTruncateBlock())) :
         new RecoveringBlock(block, locs, b.getNewGenStamp());
   }
   
-  static public DatanodeInfo convert(DatanodeInfoProto di) {
-    if (di == null) return null;
-    return new DatanodeInfo(
-        PBHelper.convert(di.getId()),
-        di.hasLocation() ? di.getLocation() : null , 
-        di.getCapacity(),  di.getDfsUsed(),  di.getRemaining(),
-        di.getBlockPoolUsed(), di.getCacheCapacity(), di.getCacheUsed(),
-        di.getLastUpdate(), di.getLastUpdateMonotonic(),
-        di.getXceiverCount(), PBHelper.convert(di.getAdminState()),
-        di.hasUpgradeDomain() ? di.getUpgradeDomain() : null);
-  }
-  
-  static public DatanodeInfo[] convert(DatanodeInfoProto di[]) {
-    if (di == null) return null;
-    DatanodeInfo[] result = new DatanodeInfo[di.length];
-    for (int i = 0; i < di.length; i++) {
-      result[i] = convert(di[i]);
-    }    
-    return result;
-  }
-
-  public static DatanodeInfo[] convert(List<DatanodeInfoProto> list) {
-    DatanodeInfo[] info = new DatanodeInfo[list.size()];
-    for (int i = 0; i < info.length; i++) {
-      info[i] = convert(list.get(i));
-    }
-    return info;
-  }
-
-  public static DatanodeStorageReportProto convertDatanodeStorageReport(
-      DatanodeStorageReport report) {
-    return DatanodeStorageReportProto.newBuilder()
-        .setDatanodeInfo(PBHelperClient.convert(report.getDatanodeInfo()))
-        .addAllStorageReports(convertStorageReports(report.getStorageReports()))
-        .build();
-  }
-
-  public static List<DatanodeStorageReportProto> convertDatanodeStorageReports(
-      DatanodeStorageReport[] reports) {
-    final List<DatanodeStorageReportProto> protos
-        = new ArrayList<DatanodeStorageReportProto>(reports.length);
-    for(int i = 0; i < reports.length; i++) {
-      protos.add(convertDatanodeStorageReport(reports[i]));
-    }
-    return protos;
-  }
-
-  public static DatanodeStorageReport convertDatanodeStorageReport(
-      DatanodeStorageReportProto proto) {
-    return new DatanodeStorageReport(
-        convert(proto.getDatanodeInfo()),
-        convertStorageReports(proto.getStorageReportsList()));
-  }
-
-  public static DatanodeStorageReport[] convertDatanodeStorageReports(
-      List<DatanodeStorageReportProto> protos) {
-    final DatanodeStorageReport[] reports
-        = new DatanodeStorageReport[protos.size()];
-    for(int i = 0; i < reports.length; i++) {
-      reports[i] = convertDatanodeStorageReport(protos.get(i));
-    }
-    return reports;
-  }
-
-  public static AdminStates convert(AdminState adminState) {
-    switch(adminState) {
-    case DECOMMISSION_INPROGRESS:
-      return AdminStates.DECOMMISSION_INPROGRESS;
-    case DECOMMISSIONED:
-      return AdminStates.DECOMMISSIONED;
-    case NORMAL:
-    default:
-      return AdminStates.NORMAL;
-    }
-  }
-  
-  public static LocatedBlockProto convert(LocatedBlock b) {
-    if (b == null) return null;
-    Builder builder = LocatedBlockProto.newBuilder();
-    DatanodeInfo[] locs = b.getLocations();
-    List<DatanodeInfo> cachedLocs =
-        Lists.newLinkedList(Arrays.asList(b.getCachedLocations()));
-    for (int i = 0; i < locs.length; i++) {
-      DatanodeInfo loc = locs[i];
-      builder.addLocs(i, PBHelperClient.convert(loc));
-      boolean locIsCached = cachedLocs.contains(loc);
-      builder.addIsCached(locIsCached);
-      if (locIsCached) {
-        cachedLocs.remove(loc);
-      }
-    }
-    Preconditions.checkArgument(cachedLocs.size() == 0,
-        "Found additional cached replica locations that are not in the set of"
-        + " storage-backed locations!");
-
-    StorageType[] storageTypes = b.getStorageTypes();
-    if (storageTypes != null) {
-      for (int i = 0; i < storageTypes.length; ++i) {
-        builder.addStorageTypes(PBHelperClient.convertStorageType(storageTypes[i]));
-      }
-    }
-    final String[] storageIDs = b.getStorageIDs();
-    if (storageIDs != null) {
-      builder.addAllStorageIDs(Arrays.asList(storageIDs));
-    }
-
-    return builder.setB(PBHelperClient.convert(b.getBlock()))
-        .setBlockToken(PBHelperClient.convert(b.getBlockToken()))
-        .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build();
-  }
-  
-  public static LocatedBlock convert(LocatedBlockProto proto) {
-    if (proto == null) return null;
-    List<DatanodeInfoProto> locs = proto.getLocsList();
-    DatanodeInfo[] targets = new DatanodeInfo[locs.size()];
-    for (int i = 0; i < locs.size(); i++) {
-      targets[i] = PBHelper.convert(locs.get(i));
-    }
-
-    final StorageType[] storageTypes = convertStorageTypes(
-        proto.getStorageTypesList(), locs.size());
-
-    final int storageIDsCount = proto.getStorageIDsCount();
-    final String[] storageIDs;
-    if (storageIDsCount == 0) {
-      storageIDs = null;
-    } else {
-      Preconditions.checkState(storageIDsCount == locs.size());
-      storageIDs = proto.getStorageIDsList().toArray(new String[storageIDsCount]);
-    }
-
-    // Set values from the isCached list, re-using references from loc
-    List<DatanodeInfo> cachedLocs = new ArrayList<DatanodeInfo>(locs.size());
-    List<Boolean> isCachedList = proto.getIsCachedList();
-    for (int i=0; i<isCachedList.size(); i++) {
-      if (isCachedList.get(i)) {
-        cachedLocs.add(targets[i]);
-      }
-    }
-
-    LocatedBlock lb = new LocatedBlock(PBHelperClient.convert(proto.getB()), targets,
-        storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(),
-        cachedLocs.toArray(new DatanodeInfo[0]));
-    lb.setBlockToken(PBHelper.convert(proto.getBlockToken()));
-
-    return lb;
-  }
-
-  public static Token<BlockTokenIdentifier> convert(
-      TokenProto blockToken) {
-    return new Token<BlockTokenIdentifier>(blockToken.getIdentifier()
-        .toByteArray(), blockToken.getPassword().toByteArray(), new Text(
-        blockToken.getKind()), new Text(blockToken.getService()));
-  }
-
-  
-  public static Token<DelegationTokenIdentifier> convertDelegationToken(
-      TokenProto blockToken) {
-    return new Token<DelegationTokenIdentifier>(blockToken.getIdentifier()
-        .toByteArray(), blockToken.getPassword().toByteArray(), new Text(
-        blockToken.getKind()), new Text(blockToken.getService()));
-  }
-
   public static ReplicaState convert(ReplicaStateProto state) {
     switch (state) {
     case RBW:
@@ -786,15 +386,15 @@ public class PBHelper {
     DatanodeRegistrationProto.Builder builder = DatanodeRegistrationProto
         .newBuilder();
     return builder.setDatanodeID(PBHelperClient.convert((DatanodeID) registration))
-        .setStorageInfo(PBHelper.convert(registration.getStorageInfo()))
-        .setKeys(PBHelper.convert(registration.getExportedKeys()))
+        .setStorageInfo(convert(registration.getStorageInfo()))
+        .setKeys(convert(registration.getExportedKeys()))
         .setSoftwareVersion(registration.getSoftwareVersion()).build();
   }
 
   public static DatanodeRegistration convert(DatanodeRegistrationProto proto) {
     StorageInfo si = convert(proto.getStorageInfo(), NodeType.DATA_NODE);
-    return new DatanodeRegistration(PBHelper.convert(proto.getDatanodeID()),
-        si, PBHelper.convert(proto.getKeys()), proto.getSoftwareVersion());
+    return new DatanodeRegistration(PBHelperClient.convert(proto.getDatanodeID()),
+        si, convert(proto.getKeys()), proto.getSoftwareVersion());
   }
 
   public static DatanodeCommand convert(DatanodeCommandProto proto) {
@@ -826,7 +426,7 @@ public class PBHelper {
 
   public static KeyUpdateCommandProto convert(KeyUpdateCommand cmd) {
     return KeyUpdateCommandProto.newBuilder()
-        .setKeys(PBHelper.convert(cmd.getExportedKeys())).build();
+        .setKeys(convert(cmd.getExportedKeys())).build();
   }
 
   public static BlockRecoveryCommandProto convert(BlockRecoveryCommand cmd) {
@@ -861,29 +461,17 @@ public class PBHelper {
     }
     Block[] blocks = cmd.getBlocks();
     for (int i = 0; i < blocks.length; i++) {
-      builder.addBlocks(PBHelper.convert(blocks[i]));
+      builder.addBlocks(PBHelperClient.convert(blocks[i]));
     }
-    builder.addAllTargets(convert(cmd.getTargets()))
+    builder.addAllTargets(PBHelperClient.convert(cmd.getTargets()))
            .addAllTargetStorageUuids(convert(cmd.getTargetStorageIDs()));
     StorageType[][] types = cmd.getTargetStorageTypes();
     if (types != null) {
-      builder.addAllTargetStorageTypes(convert(types));
+      builder.addAllTargetStorageTypes(PBHelperClient.convert(types));
     }
     return builder.build();
   }
 
-  private static List<StorageTypesProto> convert(StorageType[][] types) {
-    List<StorageTypesProto> list = Lists.newArrayList();
-    if (types != null) {
-      for (StorageType[] ts : types) {
-        StorageTypesProto.Builder builder = StorageTypesProto.newBuilder();
-        builder.addAllStorageTypes(PBHelperClient.convertStorageTypes(ts));
-        list.add(builder.build());
-      }
-    }
-    return list;
-  }
-
   public static BlockIdCommandProto convert(BlockIdCommand cmd) {
     BlockIdCommandProto.Builder builder = BlockIdCommandProto.newBuilder()
         .setBlockPoolId(cmd.getBlockPoolId());
@@ -904,15 +492,6 @@ public class PBHelper {
     return builder.build();
   }
 
-  private static List<DatanodeInfosProto> convert(DatanodeInfo[][] targets) {
-    DatanodeInfosProto[] ret = new DatanodeInfosProto[targets.length];
-    for (int i = 0; i < targets.length; i++) {
-      ret[i] = DatanodeInfosProto.newBuilder()
-          .addAllDatanodes(PBHelperClient.convert(targets[i])).build();
-    }
-    return Arrays.asList(ret);
-  }
-
   private static List<StorageUuidsProto> convert(String[][] targetStorageUuids) {
     StorageUuidsProto[] ret = new StorageUuidsProto[targetStorageUuids.length];
     for (int i = 0; i < targetStorageUuids.length; i++) {
@@ -971,7 +550,7 @@ public class PBHelper {
   }
 
   public static KeyUpdateCommand convert(KeyUpdateCommandProto keyUpdateCmd) {
-    return new KeyUpdateCommand(PBHelper.convert(keyUpdateCmd.getKeys()));
+    return new KeyUpdateCommand(convert(keyUpdateCmd.getKeys()));
   }
 
   public static FinalizeCommand convert(FinalizeCommandProto finalizeCmd) {
@@ -994,12 +573,12 @@ public class PBHelper {
     List<BlockProto> blockProtoList = blkCmd.getBlocksList();
     Block[] blocks = new Block[blockProtoList.size()];
     for (int i = 0; i < blockProtoList.size(); i++) {
-      blocks[i] = PBHelper.convert(blockProtoList.get(i));
+      blocks[i] = PBHelperClient.convert(blockProtoList.get(i));
     }
     List<DatanodeInfosProto> targetList = blkCmd.getTargetsList();
     DatanodeInfo[][] targets = new DatanodeInfo[targetList.size()][];
     for (int i = 0; i < targetList.size(); i++) {
-      targets[i] = PBHelper.convert(targetList.get(i));
+      targets[i] = PBHelperClient.convert(targetList.get(i));
     }
 
     StorageType[][] targetStorageTypes = new StorageType[targetList.size()][];
@@ -1012,7 +591,7 @@ public class PBHelper {
     } else {
       for(int i = 0; i < targetStorageTypes.length; i++) {
         List<StorageTypeProto> p = targetStorageTypesList.get(i).getStorageTypesList();
-        targetStorageTypes[i] = convertStorageTypes(p, targets[i].length);
+        targetStorageTypes[i] = PBHelperClient.convertStorageTypes(p, targets[i].length);
       }
     }
 
@@ -1061,15 +640,6 @@ public class PBHelper {
     return new BlockIdCommand(action, blkIdCmd.getBlockPoolId(), blockIds);
   }
 
-  public static DatanodeInfo[] convert(DatanodeInfosProto datanodeInfosProto) {
-    List<DatanodeInfoProto> proto = datanodeInfosProto.getDatanodesList();
-    DatanodeInfo[] infos = new DatanodeInfo[proto.size()];
-    for (int i = 0; i < infos.length; i++) {
-      infos[i] = PBHelper.convert(proto.get(i));
-    }
-    return infos;
-  }
-
   public static BalancerBandwidthCommand convert(
       BalancerBandwidthCommandProto balancerCmd) {
     return new BalancerBandwidthCommand(balancerCmd.getBandwidth());
@@ -1100,8 +670,8 @@ public class PBHelper {
     if (receivedDeletedBlockInfo.getDelHints() != null) {
       builder.setDeleteHint(receivedDeletedBlockInfo.getDelHints());
     }
-    return builder.setBlock(PBHelper.convert(receivedDeletedBlockInfo.getBlock()))
-        .build();
+    return builder.setBlock(
+        PBHelperClient.convert(receivedDeletedBlockInfo.getBlock())).build();
   }
 
   public static ReceivedDeletedBlockInfo convert(
@@ -1119,7 +689,7 @@ public class PBHelper {
       break;
     }
     return new ReceivedDeletedBlockInfo(
-        PBHelper.convert(proto.getBlock()),
+        PBHelperClient.convert(proto.getBlock()),
         status,
         proto.hasDeleteHint() ? proto.getDeleteHint() : null);
   }
@@ -1134,539 +704,7 @@ public class PBHelper {
         .setCapabilities(info.getCapabilities())
         .build();
   }
-  
-  // Located Block Arrays and Lists
-  public static LocatedBlockProto[] convertLocatedBlock(LocatedBlock[] lb) {
-    if (lb == null) return null;
-    return convertLocatedBlock2(Arrays.asList(lb)).toArray(
-        new LocatedBlockProto[lb.length]);
-  }
-  
-  public static LocatedBlock[] convertLocatedBlock(LocatedBlockProto[] lb) {
-    if (lb == null) return null;
-    return convertLocatedBlock(Arrays.asList(lb)).toArray(
-        new LocatedBlock[lb.length]);
-  }
-  
-  public static List<LocatedBlock> convertLocatedBlock(
-      List<LocatedBlockProto> lb) {
-    if (lb == null) return null;
-    final int len = lb.size();
-    List<LocatedBlock> result = 
-        new ArrayList<LocatedBlock>(len);
-    for (int i = 0; i < len; ++i) {
-      result.add(PBHelper.convert(lb.get(i)));
-    }
-    return result;
-  }
-  
-  public static List<LocatedBlockProto> convertLocatedBlock2(List<LocatedBlock> lb) {
-    if (lb == null) return null;
-    final int len = lb.size();
-    List<LocatedBlockProto> result = new ArrayList<LocatedBlockProto>(len);
-    for (int i = 0; i < len; ++i) {
-      result.add(PBHelper.convert(lb.get(i)));
-    }
-    return result;
-  }
-  
-  
-  // LocatedBlocks
-  public static LocatedBlocks convert(LocatedBlocksProto lb) {
-    return new LocatedBlocks(
-        lb.getFileLength(), lb.getUnderConstruction(),
-        PBHelper.convertLocatedBlock(lb.getBlocksList()),
-        lb.hasLastBlock() ? PBHelper.convert(lb.getLastBlock()) : null,
-        lb.getIsLastBlockComplete(),
-        lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) :
-            null);
-  }
-  
-  public static LocatedBlocksProto convert(LocatedBlocks lb) {
-    if (lb == null) {
-      return null;
-    }
-    LocatedBlocksProto.Builder builder = 
-        LocatedBlocksProto.newBuilder();
-    if (lb.getLastLocatedBlock() != null) {
-      builder.setLastBlock(PBHelper.convert(lb.getLastLocatedBlock()));
-    }
-    if (lb.getFileEncryptionInfo() != null) {
-      builder.setFileEncryptionInfo(convert(lb.getFileEncryptionInfo()));
-    }
-    return builder.setFileLength(lb.getFileLength())
-        .setUnderConstruction(lb.isUnderConstruction())
-        .addAllBlocks(PBHelper.convertLocatedBlock2(lb.getLocatedBlocks()))
-        .setIsLastBlockComplete(lb.isLastBlockComplete()).build();
-  }
-  
-  // DataEncryptionKey
-  public static DataEncryptionKey convert(DataEncryptionKeyProto bet) {
-    String encryptionAlgorithm = bet.getEncryptionAlgorithm();
-    return new DataEncryptionKey(bet.getKeyId(),
-        bet.getBlockPoolId(),
-        bet.getNonce().toByteArray(),
-        bet.getEncryptionKey().toByteArray(),
-        bet.getExpiryDate(),
-        encryptionAlgorithm.isEmpty() ? null : encryptionAlgorithm);
-  }
-  
-  public static DataEncryptionKeyProto convert(DataEncryptionKey bet) {
-    DataEncryptionKeyProto.Builder b = DataEncryptionKeyProto.newBuilder()
-        .setKeyId(bet.keyId)
-        .setBlockPoolId(bet.blockPoolId)
-        .setNonce(ByteString.copyFrom(bet.nonce))
-        .setEncryptionKey(ByteString.copyFrom(bet.encryptionKey))
-        .setExpiryDate(bet.expiryDate);
-    if (bet.encryptionAlgorithm != null) {
-      b.setEncryptionAlgorithm(bet.encryptionAlgorithm);
-    }
-    return b.build();
-  }
-  
-  public static FsServerDefaults convert(FsServerDefaultsProto fs) {
-    if (fs == null) return null;
-    return new FsServerDefaults(
-        fs.getBlockSize(), fs.getBytesPerChecksum(), 
-        fs.getWritePacketSize(), (short) fs.getReplication(),
-        fs.getFileBufferSize(),
-        fs.getEncryptDataTransfer(),
-        fs.getTrashInterval(),
-        PBHelperClient.convert(fs.getChecksumType()));
-  }
-  
-  public static FsServerDefaultsProto convert(FsServerDefaults fs) {
-    if (fs == null) return null;
-    return FsServerDefaultsProto.newBuilder().
-      setBlockSize(fs.getBlockSize()).
-      setBytesPerChecksum(fs.getBytesPerChecksum()).
-      setWritePacketSize(fs.getWritePacketSize())
-      .setReplication(fs.getReplication())
-      .setFileBufferSize(fs.getFileBufferSize())
-      .setEncryptDataTransfer(fs.getEncryptDataTransfer())
-      .setTrashInterval(fs.getTrashInterval())
-      .setChecksumType(PBHelperClient.convert(fs.getChecksumType()))
-      .build();
-  }
-  
-  public static FsPermissionProto convert(FsPermission p) {
-    return FsPermissionProto.newBuilder().setPerm(p.toExtendedShort()).build();
-  }
-  
-  public static FsPermission convert(FsPermissionProto p) {
-    return new FsPermissionExtension((short)p.getPerm());
-  }
-  
-  
-  // The creatFlag field in PB is a bitmask whose values are the same a the 
-  // emum values of CreateFlag
-  public static int convertCreateFlag(EnumSetWritable<CreateFlag> flag) {
-    int value = 0;
-    if (flag.contains(CreateFlag.APPEND)) {
-      value |= CreateFlagProto.APPEND.getNumber();
-    }
-    if (flag.contains(CreateFlag.CREATE)) {
-      value |= CreateFlagProto.CREATE.getNumber();
-    }
-    if (flag.contains(CreateFlag.OVERWRITE)) {
-      value |= CreateFlagProto.OVERWRITE.getNumber();
-    }
-    if (flag.contains(CreateFlag.LAZY_PERSIST)) {
-      value |= CreateFlagProto.LAZY_PERSIST.getNumber();
-    }
-    if (flag.contains(CreateFlag.NEW_BLOCK)) {
-      value |= CreateFlagProto.NEW_BLOCK.getNumber();
-    }
-    return value;
-  }
-  
-  public static EnumSetWritable<CreateFlag> convertCreateFlag(int flag) {
-    EnumSet<CreateFlag> result = 
-       EnumSet.noneOf(CreateFlag.class);   
-    if ((flag & CreateFlagProto.APPEND_VALUE) == CreateFlagProto.APPEND_VALUE) {
-      result.add(CreateFlag.APPEND);
-    }
-    if ((flag & CreateFlagProto.CREATE_VALUE) == CreateFlagProto.CREATE_VALUE) {
-      result.add(CreateFlag.CREATE);
-    }
-    if ((flag & CreateFlagProto.OVERWRITE_VALUE) 
-        == CreateFlagProto.OVERWRITE_VALUE) {
-      result.add(CreateFlag.OVERWRITE);
-    }
-    if ((flag & CreateFlagProto.LAZY_PERSIST_VALUE)
-        == CreateFlagProto.LAZY_PERSIST_VALUE) {
-      result.add(CreateFlag.LAZY_PERSIST);
-    }
-    if ((flag & CreateFlagProto.NEW_BLOCK_VALUE)
-        == CreateFlagProto.NEW_BLOCK_VALUE) {
-      result.add(CreateFlag.NEW_BLOCK);
-    }
-    return new EnumSetWritable<CreateFlag>(result, CreateFlag.class);
-  }
-
-  public static int convertCacheFlags(EnumSet<CacheFlag> flags) {
-    int value = 0;
-    if (flags.contains(CacheFlag.FORCE)) {
-      value |= CacheFlagProto.FORCE.getNumber();
-    }
-    return value;
-  }
-
-  public static EnumSet<CacheFlag> convertCacheFlags(int flags) {
-    EnumSet<CacheFlag> result = EnumSet.noneOf(CacheFlag.class);
-    if ((flags & CacheFlagProto.FORCE_VALUE) == CacheFlagProto.FORCE_VALUE) {
-      result.add(CacheFlag.FORCE);
-    }
-    return result;
-  }
-
-  public static HdfsFileStatus convert(HdfsFileStatusProto fs) {
-    if (fs == null)
-      return null;
-    return new HdfsLocatedFileStatus(
-        fs.getLength(), fs.getFileType().equals(FileType.IS_DIR), 
-        fs.getBlockReplication(), fs.getBlocksize(),
-        fs.getModificationTime(), fs.getAccessTime(),
-        PBHelper.convert(fs.getPermission()), fs.getOwner(), fs.getGroup(), 
-        fs.getFileType().equals(FileType.IS_SYMLINK) ? 
-            fs.getSymlink().toByteArray() : null,
-        fs.getPath().toByteArray(),
-        fs.hasFileId()? fs.getFileId(): HdfsConstants.GRANDFATHER_INODE_ID,
-        fs.hasLocations() ? PBHelper.convert(fs.getLocations()) : null,
-        fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
-        fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,
-        fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy()
-            : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
-  }
-
-  public static SnapshottableDirectoryStatus convert(
-      SnapshottableDirectoryStatusProto sdirStatusProto) {
-    if (sdirStatusProto == null) {
-      return null;
-    }
-    final HdfsFileStatusProto status = sdirStatusProto.getDirStatus();
-    return new SnapshottableDirectoryStatus(
-        status.getModificationTime(),
-        status.getAccessTime(),
-        PBHelper.convert(status.getPermission()),
-        status.getOwner(),
-        status.getGroup(),
-        status.getPath().toByteArray(),
-        status.getFileId(),
-        status.getChildrenNum(),
-        sdirStatusProto.getSnapshotNumber(),
-        sdirStatusProto.getSnapshotQuota(),
-        sdirStatusProto.getParentFullpath().toByteArray());
-  }
-  
-  public static HdfsFileStatusProto convert(HdfsFileStatus fs) {
-    if (fs == null)
-      return null;
-    FileType fType = FileType.IS_FILE;
-    if (fs.isDir()) {
-      fType = FileType.IS_DIR;
-    } else if (fs.isSymlink()) {
-      fType = FileType.IS_SYMLINK;
-    }
-
-    HdfsFileStatusProto.Builder builder = 
-     HdfsFileStatusProto.newBuilder().
-      setLength(fs.getLen()).
-      setFileType(fType).
-      setBlockReplication(fs.getReplication()).
-      setBlocksize(fs.getBlockSize()).
-      setModificationTime(fs.getModificationTime()).
-      setAccessTime(fs.getAccessTime()).
-      setPermission(PBHelper.convert(fs.getPermission())).
-      setOwner(fs.getOwner()).
-      setGroup(fs.getGroup()).
-      setFileId(fs.getFileId()).
-      setChildrenNum(fs.getChildrenNum()).
-      setPath(ByteString.copyFrom(fs.getLocalNameInBytes())).
-      setStoragePolicy(fs.getStoragePolicy());
-    if (fs.isSymlink())  {
-      builder.setSymlink(ByteString.copyFrom(fs.getSymlinkInBytes()));
-    }
-    if (fs.getFileEncryptionInfo() != null) {
-      builder.setFileEncryptionInfo(convert(fs.getFileEncryptionInfo()));
-    }
-    if (fs instanceof HdfsLocatedFileStatus) {
-      final HdfsLocatedFileStatus lfs = (HdfsLocatedFileStatus) fs;
-      LocatedBlocks locations = lfs.getBlockLocations();
-      if (locations != null) {
-        builder.setLocations(PBHelper.convert(locations));
-      }
-    }
-    return builder.build();
-  }
-  
-  public static SnapshottableDirectoryStatusProto convert(
-      SnapshottableDirectoryStatus status) {
-    if (status == null) {
-      return null;
-    }
-    int snapshotNumber = status.getSnapshotNumber();
-    int snapshotQuota = status.getSnapshotQuota();
-    byte[] parentFullPath = status.getParentFullPath();
-    ByteString parentFullPathBytes = ByteString.copyFrom(
-        parentFullPath == null ? DFSUtilClient.EMPTY_BYTES : parentFullPath);
-    HdfsFileStatusProto fs = convert(status.getDirStatus());
-    SnapshottableDirectoryStatusProto.Builder builder = 
-        SnapshottableDirectoryStatusProto
-        .newBuilder().setSnapshotNumber(snapshotNumber)
-        .setSnapshotQuota(snapshotQuota).setParentFullpath(parentFullPathBytes)
-        .setDirStatus(fs);
-    return builder.build();
-  }
-  
-  public static HdfsFileStatusProto[] convert(HdfsFileStatus[] fs) {
-    if (fs == null) return null;
-    final int len = fs.length;
-    HdfsFileStatusProto[] result = new HdfsFileStatusProto[len];
-    for (int i = 0; i < len; ++i) {
-      result[i] = PBHelper.convert(fs[i]);
-    }
-    return result;
-  }
-  
-  public static HdfsFileStatus[] convert(HdfsFileStatusProto[] fs) {
-    if (fs == null) return null;
-    final int len = fs.length;
-    HdfsFileStatus[] result = new HdfsFileStatus[len];
-    for (int i = 0; i < len; ++i) {
-      result[i] = PBHelper.convert(fs[i]);
-    }
-    return result;
-  }
-  
-  public static DirectoryListing convert(DirectoryListingProto dl) {
-    if (dl == null)
-      return null;
-    List<HdfsFileStatusProto> partList =  dl.getPartialListingList();
-    return new DirectoryListing( 
-        partList.isEmpty() ? new HdfsLocatedFileStatus[0] 
-          : PBHelper.convert(
-              partList.toArray(new HdfsFileStatusProto[partList.size()])),
-        dl.getRemainingEntries());
-  }
-
-  public static DirectoryListingProto convert(DirectoryListing d) {
-    if (d == null)
-      return null;
-    return DirectoryListingProto.newBuilder().
-        addAllPartialListing(Arrays.asList(
-            PBHelper.convert(d.getPartialListing()))).
-        setRemainingEntries(d.getRemainingEntries()).
-        build();
-  }
-
-  public static long[] convert(GetFsStatsResponseProto res) {
-    long[] result = new long[7];
-    result[ClientProtocol.GET_STATS_CAPACITY_IDX] = res.getCapacity();
-    result[ClientProtocol.GET_STATS_USED_IDX] = res.getUsed();
-    result[ClientProtocol.GET_STATS_REMAINING_IDX] = res.getRemaining();
-    result[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX] = res.getUnderReplicated();
-    result[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX] = res.getCorruptBlocks();
-    result[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX] = res.getMissingBlocks();
-    result[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX] =
-        res.getMissingReplOneBlocks();
-    return result;
-  }
-  
-  public static GetFsStatsResponseProto convert(long[] fsStats) {
-    GetFsStatsResponseProto.Builder result = GetFsStatsResponseProto
-        .newBuilder();
-    if (fsStats.length >= ClientProtocol.GET_STATS_CAPACITY_IDX + 1)
-      result.setCapacity(fsStats[ClientProtocol.GET_STATS_CAPACITY_IDX]);
-    if (fsStats.length >= ClientProtocol.GET_STATS_USED_IDX + 1)
-      result.setUsed(fsStats[ClientProtocol.GET_STATS_USED_IDX]);
-    if (fsStats.length >= ClientProtocol.GET_STATS_REMAINING_IDX + 1)
-      result.setRemaining(fsStats[ClientProtocol.GET_STATS_REMAINING_IDX]);
-    if (fsStats.length >= ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX + 1)
-      result.setUnderReplicated(
-              fsStats[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX]);
-    if (fsStats.length >= ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX + 1)
-      result.setCorruptBlocks(
-          fsStats[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX]);
-    if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX + 1)
-      result.setMissingBlocks(
-          fsStats[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX]);
-    if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX + 1)
-      result.setMissingReplOneBlocks(
-          fsStats[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX]);
-    return result.build();
-  }
-  
-  public static DatanodeReportTypeProto
-    convert(DatanodeReportType t) {
-    switch (t) {
-    case ALL: return DatanodeReportTypeProto.ALL;
-    case LIVE: return DatanodeReportTypeProto.LIVE;
-    case DEAD: return DatanodeReportTypeProto.DEAD;
-    case DECOMMISSIONING: return DatanodeReportTypeProto.DECOMMISSIONING;
-    default: 
-      throw new IllegalArgumentException("Unexpected data type report:" + t);
-    }
-  }
-  
-  public static DatanodeReportType 
-    convert(DatanodeReportTypeProto t) {
-    switch (t) {
-    case ALL: return DatanodeReportType.ALL;
-    case LIVE: return DatanodeReportType.LIVE;
-    case DEAD: return DatanodeReportType.DEAD;
-    case DECOMMISSIONING: return DatanodeReportType.DECOMMISSIONING;
-    default: 
-      throw new IllegalArgumentException("Unexpected data type report:" + t);
-    }
-  }
-
-  public static SafeModeActionProto convert(
-      SafeModeAction a) {
-    switch (a) {
-    case SAFEMODE_LEAVE:
-      return SafeModeActionProto.SAFEMODE_LEAVE;
-    case SAFEMODE_ENTER:
-      return SafeModeActionProto.SAFEMODE_ENTER;
-    case SAFEMODE_GET:
-      return SafeModeActionProto.SAFEMODE_GET;
-    default:
-      throw new IllegalArgumentException("Unexpected SafeModeAction :" + a);
-    }
-  }
-  
-  public static SafeModeAction convert(
-      ClientNamenodeProtocolProtos.SafeModeActionProto a) {
-    switch (a) {
-    case SAFEMODE_LEAVE:
-      return SafeModeAction.SAFEMODE_LEAVE;
-    case SAFEMODE_ENTER:
-      return SafeModeAction.SAFEMODE_ENTER;
-    case SAFEMODE_GET:
-      return SafeModeAction.SAFEMODE_GET;
-    default:
-      throw new IllegalArgumentException("Unexpected SafeModeAction :" + a);
-    }
-  }
-  
-  public static RollingUpgradeActionProto convert(RollingUpgradeAction a) {
-    switch (a) {
-    case QUERY:
-      return RollingUpgradeActionProto.QUERY;
-    case PREPARE:
-      return RollingUpgradeActionProto.START;
-    case FINALIZE:
-      return RollingUpgradeActionProto.FINALIZE;
-    default:
-      throw new IllegalArgumentException("Unexpected value: " + a);
-    }
-  }
-  
-  public static RollingUpgradeAction convert(RollingUpgradeActionProto a) {
-    switch (a) {
-    case QUERY:
-      return RollingUpgradeAction.QUERY;
-    case START:
-      return RollingUpgradeAction.PREPARE;
-    case FINALIZE:
-      return RollingUpgradeAction.FINALIZE;
-    default:
-      throw new IllegalArgumentException("Unexpected value: " + a);
-    }
-  }
-
-  public static RollingUpgradeStatusProto convertRollingUpgradeStatus(
-      RollingUpgradeStatus status) {
-    return RollingUpgradeStatusProto.newBuilder()
-        .setBlockPoolId(status.getBlockPoolId())
-        .setFinalized(status.isFinalized())
-        .build();
-  }
-
-  public static RollingUpgradeStatus convert(RollingUpgradeStatusProto proto) {
-    return new RollingUpgradeStatus(proto.getBlockPoolId(),
-        proto.getFinalized());
-  }
-
-  public static RollingUpgradeInfoProto convert(RollingUpgradeInfo info) {
-    return RollingUpgradeInfoProto.newBuilder()
-        .setStatus(convertRollingUpgradeStatus(info))
-        .setCreatedRollbackImages(info.createdRollbackImages())
-        .setStartTime(info.getStartTime())
-        .setFinalizeTime(info.getFinalizeTime())
-        .build();
-  }
-
-  public static RollingUpgradeInfo convert(RollingUpgradeInfoProto proto) {
-    RollingUpgradeStatusProto status = proto.getStatus();
-    return new RollingUpgradeInfo(status.getBlockPoolId(),
-        proto.getCreatedRollbackImages(),
-        proto.getStartTime(), proto.getFinalizeTime());
-  }
-
-  public static CorruptFileBlocks convert(CorruptFileBlocksProto c) {
-    if (c == null)
-      return null;
-    List<String> fileList = c.getFilesList();
-    return new CorruptFileBlocks(fileList.toArray(new String[fileList.size()]),
-        c.getCookie());
-  }
-
-  public static CorruptFileBlocksProto convert(CorruptFileBlocks c) {
-    if (c == null)
-      return null;
-    return CorruptFileBlocksProto.newBuilder().
-        addAllFiles(Arrays.asList(c.getFiles())).
-        setCookie(c.getCookie()).
-        build();
-  }
-  
-  public static ContentSummary convert(ContentSummaryProto cs) {
-    if (cs == null) return null;
-    ContentSummary.Builder builder = new ContentSummary.Builder();
-    builder.length(cs.getLength()).
-        fileCount(cs.getFileCount()).
-        directoryCount(cs.getDirectoryCount()).
-        quota(cs.getQuota()).
-        spaceConsumed(cs.getSpaceConsumed()).
-        spaceQuota(cs.getSpaceQuota());
-    if (cs.hasTypeQuotaInfos()) {
-      for (HdfsProtos.StorageTypeQuotaInfoProto info :
-          cs.getTypeQuotaInfos().getTypeQuotaInfoList()) {
-        StorageType type = PBHelperClient.convertStorageType(info.getType());
-        builder.typeConsumed(type, info.getConsumed());
-        builder.typeQuota(type, info.getQuota());
-      }
-    }
-    return builder.build();
-  }
-  
-  public static ContentSummaryProto convert(ContentSummary cs) {
-    if (cs == null) return null;
-    ContentSummaryProto.Builder builder = ContentSummaryProto.newBuilder();
-        builder.setLength(cs.getLength()).
-        setFileCount(cs.getFileCount()).
-        setDirectoryCount(cs.getDirectoryCount()).
-        setQuota(cs.getQuota()).
-        setSpaceConsumed(cs.getSpaceConsumed()).
-        setSpaceQuota(cs.getSpaceQuota());
 
-    if (cs.isTypeQuotaSet() || cs.isTypeConsumedAvailable()) {
-      HdfsProtos.StorageTypeQuotaInfosProto.Builder isb =
-          HdfsProtos.StorageTypeQuotaInfosProto.newBuilder();
-      for (StorageType t: StorageType.getTypesSupportingQuota()) {
-        HdfsProtos.StorageTypeQuotaInfoProto info =
-            HdfsProtos.StorageTypeQuotaInfoProto.newBuilder().
-                setType(PBHelperClient.convertStorageType(t)).
-                setConsumed(cs.getTypeConsumed(t)).
-                setQuota(cs.getTypeQuota(t)).
-                build();
-        isb.addTypeQuotaInfo(info);
-      }
-      builder.setTypeQuotaInfos(isb);
-    }
-    return builder.build();
-  }
 
   public static NNHAStatusHeartbeat convert(NNHAStatusHeartbeatProto s) {
     if (s == null) return null;
@@ -1699,95 +737,12 @@ public class PBHelper {
     return builder.build();
   }
 
-  public static DatanodeStorageProto convert(DatanodeStorage s) {
-    return DatanodeStorageProto.newBuilder()
-        .setState(PBHelper.convertState(s.getState()))
-        .setStorageType(PBHelperClient.convertStorageType(s.getStorageType()))
-        .setStorageUuid(s.getStorageID()).build();
-  }
-
-  private static StorageState convertState(State state) {
-    switch(state) {
-    case READ_ONLY_SHARED:
-      return StorageState.READ_ONLY_SHARED;
-    case NORMAL:
-    default:
-      return StorageState.NORMAL;
-    }
-  }
-
-  public static DatanodeStorage convert(DatanodeStorageProto s) {
-    return new DatanodeStorage(s.getStorageUuid(),
-                               PBHelper.convertState(s.getState()),
-                               PBHelperClient.convertStorageType(s.getStorageType()));
-  }
-
-  private static State convertState(StorageState state) {
-    switch(state) {
-    case READ_ONLY_SHARED:
-      return DatanodeStorage.State.READ_ONLY_SHARED;
-    case NORMAL:
-    default:
-      return DatanodeStorage.State.NORMAL;
-    }
-  }
-
-  public static StorageType[] convertStorageTypes(
-      List<StorageTypeProto> storageTypesList, int expectedSize) {
-    final StorageType[] storageTypes = new StorageType[expectedSize];
-    if (storageTypesList.size() != expectedSize) { // missing storage types
-      Preconditions.checkState(storageTypesList.isEmpty());
-      Arrays.fill(storageTypes, StorageType.DEFAULT);
-    } else {
-      for (int i = 0; i < storageTypes.length; ++i) {
-        storageTypes[i] = PBHelperClient.convertStorageType(storageTypesList.get(i));
-      }
-    }
-    return storageTypes;
-  }
-
-  public static StorageReportProto convert(StorageReport r) {
-    StorageReportProto.Builder builder = StorageReportProto.newBuilder()
-        .setBlockPoolUsed(r.getBlockPoolUsed()).setCapacity(r.getCapacity())
-        .setDfsUsed(r.getDfsUsed()).setRemaining(r.getRemaining())
-        .setStorageUuid(r.getStorage().getStorageID())
-        .setStorage(convert(r.getStorage()));
-    return builder.build();
-  }
-
-  public static StorageReport convert(StorageReportProto p) {
-    return new StorageReport(
-        p.hasStorage() ?
-            convert(p.getStorage()) :
-            new DatanodeStorage(p.getStorageUuid()),
-        p.getFailed(), p.getCapacity(), p.getDfsUsed(), p.getRemaining(),
-        p.getBlockPoolUsed());
-  }
-
-  public static StorageReport[] convertStorageReports(
-      List<StorageReportProto> list) {
-    final StorageReport[] report = new StorageReport[list.size()];
-    for (int i = 0; i < report.length; i++) {
-      report[i] = convert(list.get(i));
-    }
-    return report;
-  }
-
-  public static List<StorageReportProto> convertStorageReports(StorageReport[] storages) {
-    final List<StorageReportProto> protos = new ArrayList<StorageReportProto>(
-        storages.length);
-    for(int i = 0; i < storages.length; i++) {
-      protos.add(convert(storages[i]));
-    }
-    return protos;
-  }
-
-  public static VolumeFailureSummary convertVolumeFailureSummary(
-      VolumeFailureSummaryProto proto) {
-    List<String> failedStorageLocations = proto.getFailedStorageLocationsList();
-    return new VolumeFailureSummary(
-        failedStorageLocations.toArray(new String[failedStorageLocations.size()]),
-        proto.getLastVolumeFailureDate(), proto.getEstimatedCapacityLostTotal());
+  public static VolumeFailureSummary convertVolumeFailureSummary(
+      VolumeFailureSummaryProto proto) {
+    List<String> failedStorageLocations = proto.getFailedStorageLocationsList();
+    return new VolumeFailureSummary(
+        failedStorageLocations.toArray(new String[failedStorageLocations.size()]),
+        proto.getLastVolumeFailureDate(), proto.getEstimatedCapacityLostTotal());
   }
 
   public static VolumeFailureSummaryProto convertVolumeFailureSummary(
@@ -1819,933 +774,8 @@ public class PBHelper {
     return JournalInfoProto.newBuilder().setClusterID(j.getClusterId())
         .setLayoutVersion(j.getLayoutVersion())
         .setNamespaceID(j.getNamespaceId()).build();
-  } 
-  
-  public static SnapshottableDirectoryStatus[] convert(
-      SnapshottableDirectoryListingProto sdlp) {
-    if (sdlp == null)
-      return null;
-    List<SnapshottableDirectoryStatusProto> list = sdlp
-        .getSnapshottableDirListingList();
-    if (list.isEmpty()) {
-      return new SnapshottableDirectoryStatus[0];
-    } else {
-      SnapshottableDirectoryStatus[] result = 
-          new SnapshottableDirectoryStatus[list.size()];
-      for (int i = 0; i < list.size(); i++) {
-        result[i] = PBHelper.convert(list.get(i));
-      }
-      return result;
-    }
-  }
-  
-  public static SnapshottableDirectoryListingProto convert(
-      SnapshottableDirectoryStatus[] status) {
-    if (status == null)
-      return null;
-    SnapshottableDirectoryStatusProto[] protos = 
-        new SnapshottableDirectoryStatusProto[status.length];
-    for (int i = 0; i < status.length; i++) {
-      protos[i] = PBHelper.convert(status[i]);
-    }
-    List<SnapshottableDirectoryStatusProto> protoList = Arrays.asList(protos);
-    return SnapshottableDirectoryListingProto.newBuilder()
-        .addAllSnapshottableDirListing(protoList).build();
-  }
-  
-  public static DiffReportEntry convert(SnapshotDiffReportEntryProto entry) {
-    if (entry == null) {
-      return null;
-    }
-    DiffType type = DiffType.getTypeFromLabel(entry
-        .getModificationLabel());
-    return type == null ? null : new DiffReportEntry(type, entry.getFullpath()
-        .toByteArray(), entry.hasTargetPath() ? entry.getTargetPath()
-        .toByteArray() : null);
-  }
-  
-  public static SnapshotDiffReportEntryProto convert(DiffReportEntry entry) {
-    if (entry == null) {
-      return null;
-    }
-    ByteString sourcePath = ByteString
-        .copyFrom(entry.getSourcePath() == null ? DFSUtilClient.EMPTY_BYTES : entry
-            .getSourcePath());
-    String modification = entry.getType().getLabel();
-    SnapshotDiffReportEntryProto.Builder builder = SnapshotDiffReportEntryProto
-        .newBuilder().setFullpath(sourcePath)
-        .setModificationLabel(modification);
-    if (entry.getType() == DiffType.RENAME) {
-      ByteString targetPath = ByteString
-          .copyFrom(entry.getTargetPath() == null ? DFSUtilClient.EMPTY_BYTES : entry
-              .getTargetPath());
-      builder.setTargetPath(targetPath);
-    }
-    return builder.build();
-  }
-  
-  public static SnapshotDiffReport convert(SnapshotDiffReportProto reportProto) {
-    if (reportProto == null) {
-      return null;
-    }
-    String snapshotDir = reportProto.getSnapshotRoot();
-    String fromSnapshot = reportProto.getFromSnapshot();
-    String toSnapshot = reportProto.getToSnapshot();
-    List<SnapshotDiffReportEntryProto> list = reportProto
-        .getDiffReportEntriesList();
-    List<DiffReportEntry> entries = new ArrayList<DiffReportEntry>();
-    for (SnapshotDiffReportEntryProto entryProto : list) {
-      DiffReportEntry entry = convert(entryProto);
-      if (entry != null)
-        entries.add(entry);
-    }
-    return new SnapshotDiffReport(snapshotDir, fromSnapshot, toSnapshot,
-        entries);
-  }
-  
-  public static SnapshotDiffReportProto convert(SnapshotDiffReport report) {
-    if (report == null) {
-      return null;
-    }
-    List<DiffReportEntry> entries = report.getDiffList();
-    List<SnapshotDiffReportEntryProto> entryProtos = 
-        new ArrayList<SnapshotDiffReportEntryProto>();
-    for (DiffReportEntry entry : entries) {
-      SnapshotDiffReportEntryProto entryProto = convert(entry);
-      if (entryProto != null)
-        entryProtos.add(entryProto);
-    }
-    
-    SnapshotDiffReportProto reportProto = SnapshotDiffReportProto.newBuilder()
-        .setSnapshotRoot(report.getSnapshotRoot())
-        .setFromSnapshot(report.getFromSnapshot())
-        .setToSnapshot(report.getLaterSnapshotName())
-        .addAllDiffReportEntries(entryProtos).build();
-    return reportProto;
-  }
-
-  public static CacheDirectiveInfoProto convert
-      (CacheDirectiveInfo info) {
-    CacheDirectiveInfoProto.Builder builder = 
-        CacheDirectiveInfoProto.newBuilder();
-    if (info.getId() != null) {
-      builder.setId(info.getId());
-    }
-    if (info.getPath() != null) {
-      builder.setPath(info.getPath().toUri().getPath());
-    }
-    if (info.getReplication() != null) {
-      builder.setReplication(info.getReplication());
-    }
-    if (info.getPool() != null) {
-      builder.setPool(info.getPool());
-    }
-    if (info.getExpiration() != null) {
-      builder.setExpiration(convert(info.getExpiration()));
-    }
-    return builder.build();
-  }
-
-  public static CacheDirectiveInfo convert
-      (CacheDirectiveInfoProto proto) {
-    CacheDirectiveInfo.Builder builder =
-        new CacheDirectiveInfo.Builder();
-    if (proto.hasId()) {
-      builder.setId(proto.getId());
-    }
-    if (proto.hasPath()) {
-      builder.setPath(new Path(proto.getPath()));
-    }
-    if (proto.hasReplication()) {
-      builder.setReplication(Shorts.checkedCast(
-          proto.getReplication()));
-    }
-    if (proto.hasPool()) {
-      builder.setPool(proto.getPool());
-    }
-    if (proto.hasExpiration()) {
-      builder.setExpiration(convert(proto.getExpiration()));
-    }
-    return builder.build();
-  }
-
-  public static CacheDirectiveInfoExpirationProto convert(
-      CacheDirectiveInfo.Expiration expiration) {
-    return CacheDirectiveInfoExpirationProto.newBuilder()
-        .setIsRelative(expiration.isRelative())
-        .setMillis(expiration.getMillis())
-        .build();
-  }
-
-  public static CacheDirectiveInfo.Expiration convert(
-      CacheDirectiveInfoExpirationProto proto) {
-    if (proto.getIsRelative()) {
-      return CacheDirectiveInfo.Expiration.newRelative(proto.getMillis());
-    }
-    return CacheDirectiveInfo.Expiration.newAbsolute(proto.getMillis());
-  }
-
-  public static CacheDirectiveStatsProto convert(CacheDirectiveStats stats) {
-    CacheDirectiveStatsProto.Builder builder = 
-        CacheDirectiveStatsProto.newBuilder();
-    builder.setBytesNeeded(stats.getBytesNeeded());
-    builder.setBytesCached(stats.getBytesCached());
-    builder.setFilesNeeded(stats.getFilesNeeded());
-    builder.setFilesCached(stats.getFilesCached());
-    builder.setHasExpired(stats.hasExpired());
-    return builder.build();
-  }
-  
-  public static CacheDirectiveStats convert(CacheDirectiveStatsProto proto) {
-    CacheDirectiveStats.Builder builder = new CacheDirectiveStats.Builder();
-    builder.setBytesNeeded(proto.getBytesNeeded());
-    builder.setBytesCached(proto.getBytesCached());
-    builder.setFilesNeeded(proto.getFilesNeeded());
-    builder.setFilesCached(proto.getFilesCached());
-    builder.setHasExpired(proto.getHasExpired());
-    return builder.build();
-  }
-
-  public static CacheDirectiveEntryProto convert(CacheDirectiveEntry entry) {
-    CacheDirectiveEntryProto.Builder builder = 
-        CacheDirectiveEntryProto.newBuilder();
-    builder.setInfo(PBHelper.convert(entry.getInfo()));
-    builder.setStats(PBHelper.convert(entry.getStats()));
-    return builder.build();
-  }
-  
-  public static CacheDirectiveEntry convert(CacheDirectiveEntryProto proto) {
-    CacheDirectiveInfo info = PBHelper.convert(proto.getInfo());
-    CacheDirectiveStats stats = PBHelper.convert(proto.getStats());
-    return new CacheDirectiveEntry(info, stats);
-  }
-
-  public static CachePoolInfoProto convert(CachePoolInfo info) {
-    CachePoolInfoProto.Builder builder = CachePoolInfoProto.newBuilder();
-    builder.setPoolName(info.getPoolName());
-    if (info.getOwnerName() != null) {
-      builder.setOwnerName(info.getOwnerName());
-    }
-    if (info.getGroupName() != null) {
-      builder.setGroupName(info.getGroupName());
-    }
-    if (info.getMode() != null) {
-      builder.setMode(info.getMode().toShort());
-    }
-    if (info.getLimit() != null) {
-      builder.setLimit(info.getLimit());
-    }
-    if (info.getMaxRelativeExpiryMs() != null) {
-      builder.setMaxRelativeExpiry(info.getMaxRelativeExpiryMs());
-    }
-    return builder.build();
-  }
-
-  public static CachePoolInfo convert (CachePoolInfoProto proto) {
-    // Pool name is a required field, the rest are optional
-    String poolName = checkNotNull(proto.getPoolName());
-    CachePoolInfo info = new CachePoolInfo(poolName);
-    if (proto.hasOwnerName()) {
-        info.setOwnerName(proto.getOwnerName());
-    }
-    if (proto.hasGroupName()) {
-      info.setGroupName(proto.getGroupName());
-    }
-    if (proto.hasMode()) {
-      info.setMode(new FsPermission((short)proto.getMode()));
-    }
-    if (proto.hasLimit())  {
-      info.setLimit(proto.getLimit());
-    }
-    if (proto.hasMaxRelativeExpiry()) {
-      info.setMaxRelativeExpiryMs(proto.getMaxRelativeExpiry());
-    }
-    return info;
-  }
-
-  public static CachePoolStatsProto convert(CachePoolStats stats) {
-    CachePoolStatsProto.Builder builder = CachePoolStatsProto.newBuilder();
-    builder.setBytesNeeded(stats.getBytesNeeded());
-    builder.setBytesCached(stats.getBytesCached());
-    builder.setBytesOverlimit(stats.getBytesOverlimit());
-    builder.setFilesNeeded(stats.getFilesNeeded());
-    builder.setFilesCached(stats.getFilesCached());
-    return builder.build();
-  }
-
-  public static CachePoolStats convert (CachePoolStatsProto proto) {
-    CachePoolStats.Builder builder = new CachePoolStats.Builder();
-    builder.setBytesNeeded(proto.getBytesNeeded());
-    builder.setBytesCached(proto.getBytesCached());
-    builder.setBytesOverlimit(proto.getBytesOverlimit());
-    builder.setFilesNeeded(proto.getFilesNeeded());
-    builder.setFilesCached(proto.getFilesCached());
-    return builder.build();
-  }
-
-  public static CachePoolEntryProto convert(CachePoolEntry entry) {
-    CachePoolEntryProto.Builder builder = CachePoolEntryProto.newBuilder();
-    builder.setInfo(PBHelper.convert(entry.getInfo()));
-    builder.setStats(PBHelper.convert(entry.getStats()));
-    return builder.build();
-  }
-
-  public static CachePoolEntry convert (CachePoolEntryProto proto) {
-    CachePoolInfo info = PBHelper.convert(proto.getInfo());
-    CachePoolStats stats = PBHelper.convert(proto.getStats());
-    return new CachePoolEntry(info, stats);
-  }
-  
-
-  public static DatanodeLocalInfoProto convert(DatanodeLocalInfo info) {
-    DatanodeLocalInfoProto.Builder builder = DatanodeLocalInfoProto.newBuilder();
-    builder.setSoftwareVersion(info.getSoftwareVersion());
-    builder.setConfigVersion(info.getConfigVersion());
-    builder.setUptime(info.getUptime());
-    return builder.build();
-  }
-
-  private static AclEntryScopeProto convert(AclEntryScope v) {
-    return AclEntryScopeProto.valueOf(v.ordinal());
-  }
-
-  private static AclEntryScope convert(AclEntryScopeProto v) {
-    return castEnum(v, ACL_ENTRY_SCOPE_VALUES);
-  }
-
-  private static AclEntryTypeProto convert(AclEntryType e) {
-    return AclEntryTypeProto.valueOf(e.ordinal());
-  }
-
-  private static AclEntryType convert(AclEntryTypeProto v) {
-    return castEnum(v, ACL_ENTRY_TYPE_VALUES);
-  }
-  
-  private static XAttrNamespaceProto convert(XAttr.NameSpace v) {
-    return XAttrNamespaceProto.valueOf(v.ordinal());
-  }
-  
-  private static XAttr.NameSpace convert(XAttrNamespaceProto v) {
-    return castEnum(v, XATTR_NAMESPACE_VALUES);
-  }
-
-  public static FsActionProto convert(FsAction v) {
-    return FsActionProto.valueOf(v != null ? v.ordinal() : 0);
-  }
-
-  public static FsAction convert(FsActionProto v) {
-    return castEnum(v, FSACTION_VALUES);
   }
 
-  public static List<AclEntryProto> convertAclEntryProto(
-      List<AclEntry> aclSpec) {
-    ArrayList<AclEntryProto> r = Lists.newArrayListWithCapacity(aclSpec.size());
-    for (AclEntry e : aclSpec) {
-      AclEntryProto.Builder builder = AclEntryProto.newBuilder();
-      builder.setType(convert(e.getType()));
-      builder.setScope(convert(e.getScope()));
-      builder.setPermissions(convert(e.getPermission()));
-      if (e.getName() != null) {
-        builder.setName(e.getName());
-      }
-      r.add(builder.build());
-    }
-    return r;
-  }
-
-  public static List<AclEntry> convertAclEntry(List<AclEntryProto> aclSpec) {
-    ArrayList<AclEntry> r = Lists.newArrayListWithCapacity(aclSpec.size());
-    for (AclEntryProto e : aclSpec) {
-      AclEntry.Builder builder = new AclEntry.Builder();
-      builder.setType(convert(e.getType()));
-      builder.setScope(convert(e.getScope()));
-      builder.setPermission(convert(e.getPermissions()));
-      if (e.hasName()) {
-        builder.setName(e.getName());
-      }
-      r.add(builder.build());
-    }
-    return r;
-  }
-
-  public static AclStatus convert(GetAclStatusResponseProto e) {
-    AclStatusProto r = e.getResult();
-    AclStatus.Builder builder = new AclStatus.Builder();
-    builder.owner(r.getOwner()).group(r.getGroup()).stickyBit(r.getSticky())
-        .addEntries(convertAclEntry(r.getEntriesList()));
-    if (r.hasPermission()) {
-      builder.setPermission(convert(r.getPermission()));
-    }
-    return builder.build();
-  }
-
-  public static GetAclStatusResponseProto convert(AclStatus e) {
-    AclStatusProto.Builder builder = AclStatusProto.newBuilder();
-    builder.setOwner(e.getOwner())
-        .setGroup(e.getGroup()).setSticky(e.isStickyBit())
-        .addAllEntries(convertAclEntryProto(e.getEntries()));
-    if (e.getPermission() != null) {
-      builder.setPermission(convert(e.getPermission()));
-    }
-    AclStatusProto r = builder.build();
-    return GetAclStatusResponseProto.newBuilder().setResult(r).build();
-  }
-  
-  public static XAttrProto convertXAttrProto(XAttr a) {
-    XAttrProto.Builder builder = XAttrProto.newBuilder();
-    builder.setNamespace(convert(a.getNameSpace()));
-    if (a.getName() != null) {
-      builder.setName(a.getName());
-    }
-    if (a.getValue() != null) {
-      builder.setValue(getByteString(a.getValue()));
-    }
-    return builder.build();
-  }
-  
-  public static List<XAttrProto> convertXAttrProto(
-      List<XAttr> xAttrSpec) {
-    if (xAttrSpec == null) {
-      return Lists.newArrayListWithCapacity(0);
-    }
-    ArrayList<XAttrProto> xAttrs = Lists.newArrayListWithCapacity(
-        xAttrSpec.size());
-    for (XAttr a : xAttrSpec) {
-      XAttrProto.Builder builder = XAttrProto.newBuilder();
-      builder.setNamespace(convert(a.getNameSpace()));
-      if (a.getName() != null) {
-        builder.setName(a.getName());
-      }
-      if (a.getValue() != null) {
-        builder.setValue(getByteString(a.getValue()));
-      }
-      xAttrs.add(builder.build());
-    }
-    return xAttrs;
-  }
-  
-  /**
-   * The flag field in PB is a bitmask whose values are the same a the 
-   * emum values of XAttrSetFlag
-   */
-  public static int convert(EnumSet<XAttrSetFlag> flag) {
-    int value = 0;
-    if (flag.contains(XAttrSetFlag.CREATE)) {
-      value |= XAttrSetFlagProto.XATTR_CREATE.getNumber();
-    }
-    if (flag.contains(XAttrSetFlag.REPLACE)) {
-      value |= XAttrSetFlagProto.XATTR_REPLACE.getNumber();
-    }
-    return value;
-  }
- 
-  public static EnumSet<XAttrSetFlag> convert(int flag) {
-    EnumSet<XAttrSetFlag> result = 
-        EnumSet.noneOf(XAttrSetFlag.class);
-    if ((flag & XAttrSetFlagProto.XATTR_CREATE_VALUE) == 
-        XAttrSetFlagProto.XATTR_CREATE_VALUE) {
-      result.add(XAttrSetFlag.CREATE);
-    }
-    if ((flag & XAttrSetFlagProto.XATTR_REPLACE_VALUE) == 
-        XAttrSetFlagProto.XATTR_REPLACE_VALUE) {
-      result.add(XAttrSetFlag.REPLACE);
-    }
-    return result;
-  }
-  
-  public static XAttr convertXAttr(XAttrProto a) {
-    XAttr.Builder builder = new XAttr.Builder();
-    builder.setNameSpace(convert(a.getNamespace()));
-    if (a.hasName()) {
-      builder.setName(a.getName());
-    }
-    if (a.hasValue()) {
-      builder.setValue(a.getValue().toByteArray());
-    }
-    return builder.build();
-  }
-  
-  public static List<XAttr> convertXAttrs(List<XAttrProto> xAttrSpec) {
-    ArrayList<XAttr> xAttrs = Lists.newArrayListWithCapacity(xAttrSpec.size());
-    for (XAttrProto a : xAttrSpec) {
-      XAttr.Builder builder = new XAttr.Builder();
-      builder.setNameSpace(convert(a.getNamespace()));
-      if (a.hasName()) {
-        builder.setName(a.getName());
-      }
-      if (a.hasValue()) {
-        builder.setValue(a.getValue().toByteArray());
-      }
-      xAttrs.add(builder.build());
-    }
-    return xAttrs;
-  }
-
-  public static List<XAttr> convert(GetXAttrsResponseProto a) {
-    List<XAttrProto> xAttrs = a.getXAttrsList();
-    return convertXAttrs(xAttrs);
-  }
-
-  public static GetXAttrsResponseProto convertXAttrsResponse(
-      List<XAttr> xAttrs) {
-    GetXAttrsResponseProto.Builder builder = GetXAttrsResponseProto
-        .newBuilder();
-    if (xAttrs != null) {
-      builder.addAllXAttrs(convertXAttrProto(xAttrs));
-    }
-    return builder.build();
-  }
-
-  public static List<XAttr> convert(ListXAttrsResponseProto a) {
-    final List<XAttrProto> xAttrs = a.getXAttrsList();
-    return convertXAttrs(xAttrs);
-  }
-
-  public static ListXAttrsResponseProto convertListXAttrsResponse(
-    List<XAttr> names) {
-    ListXAttrsResponseProto.Builder builder =
-      ListXAttrsResponseProto.newBuilder();
-    if (names != null) {
-      builder.addAllXAttrs(convertXAttrProto(names));
-    }
-    return builder.build();
-  }
-
-  public static EncryptionZoneProto convert(EncryptionZone zone) {
-    return EncryptionZoneProto.newBuilder()
-        .setId(zone.getId())
-        .setPath(zone.getPath())
-        .setSuite(PBHelperClient.convert(zone.getSuite()))
-        .setCryptoProtocolVersion(convert(zone.getVersion()))
-        .setKeyName(zone.getKeyName())
-        .build();
-  }
-
-  public static EncryptionZone convert(EncryptionZoneProto proto) {
-    return new EncryptionZone(proto.getId(), proto.getPath(),
-        PBHelperClient.convert(proto.getSuite()), convert(proto.getCryptoProtocolVersion()),
-        proto.getKeyName());
-  }
-
-  public static SlotId convert(ShortCircuitShmSlotProto slotId) {
-    return new SlotId(PBHelperClient.convert(slotId.getShmId()),
-        slotId.getSlotIdx());
-  }
-
-  private static Event.CreateEvent.INodeType createTypeConvert(InotifyProtos.INodeType
-      type) {
-    switch (type) {
-    case I_TYPE_DIRECTORY:
-      return Event.CreateEvent.INodeType.DIRECTORY;
-    case I_TYPE_FILE:
-      return Event.CreateEvent.INodeType.FILE;
-    case I_TYPE_SYMLINK:
-      return Event.CreateEvent.INodeType.SYMLINK;
-    default:
-      return null;
-    }
-  }
-
-  private static InotifyProtos.MetadataUpdateType metadataUpdateTypeConvert(
-      Event.MetadataUpdateEvent.MetadataType type) {
-    switch (type) {
-    case TIMES:
-      return InotifyProtos.MetadataUpdateType.META_TYPE_TIMES;
-    case REPLICATION:
-      return InotifyProtos.MetadataUpdateType.META_TYPE_REPLICATION;
-    case OWNER:
-      return InotifyProtos.MetadataUpdateType.META_TYPE_OWNER;
-    case PERMS:
-      return InotifyProtos.MetadataUpdateType.META_TYPE_PERMS;
-    case ACLS:
-      return InotifyProtos.MetadataUpdateType.META_TYPE_ACLS;
-    case XATTRS:
-      return InotifyProtos.MetadataUpdateType.META_TYPE_XATTRS;
-    default:
-      return null;
-    }
-  }
-
-  private static Event.MetadataUpdateEvent.MetadataType metadataUpdateTypeConvert(
-      InotifyProtos.MetadataUpdateType type) {
-    switch (type) {
-    case META_TYPE_TIMES:
-      return Event.MetadataUpdateEvent.MetadataType.TIMES;
-    case META_TYPE_REPLICATION:
-      return Event.MetadataUpdateEvent.MetadataType.REPLICATION;
-    case META_TYPE_OWNER:
-      return Event.MetadataUpdateEvent.MetadataType.OWNER;
-    case META_TYPE_PERMS:
-      return Event.MetadataUpdateEvent.MetadataType.PERMS;
-    case META_TYPE_ACLS:
-      return Event.MetadataUpdateEvent.MetadataType.ACLS;
-    case META_TYPE_XATTRS:
-      return Event.MetadataUpdateEvent.MetadataType.XATTRS;
-    default:
-      return null;
-    }
-  }
-
-  private static InotifyProtos.INodeType createTypeConvert(Event.CreateEvent.INodeType
-      type) {
-    switch (type) {
-    case DIRECTORY:
-      return InotifyProtos.INodeType.I_TYPE_DIRECTORY;
-    case FILE:
-      return InotifyProtos.INodeType.I_TYPE_FILE;
-    case SYMLINK:
-      return InotifyProtos.INodeType.I_TYPE_SYMLINK;
-    default:
-      return null;
-    }
-  }
-
-  public static EventBatchList convert(GetEditsFromTxidResponseProto resp) throws
-    IOException {
-    final InotifyProtos.EventsListProto list = resp.getEventsList();
-    final long firstTxid = list.getFirstTxid();
-    final long lastTxid = list.getLastTxid();
-
-    List<EventBatch> batches = Lists.newArrayList();
-    if (list.getEventsList().size() > 0) {
-      throw new IOException("Can't handle old inotify server response.");
-    }
-    for (InotifyProtos.EventBatchProto bp : list.getBatchList()) {
-      long txid = bp.getTxid();
-      if ((txid != -1) && ((txid < firstTxid) || (txid > lastTxid))) {
-        throw new IOException("Error converting TxidResponseProto: got a " +
-            "transaction id " + txid + " that was outside the range of [" +
-            firstTxid + ", " + lastTxid + "].");
-      }
-      List<Event> events = Lists.newArrayList();
-      for (InotifyProtos.EventProto p : bp.getEventsList()) {
-        switch (p.getType()) {
-          case EVENT_CLOSE:
-            InotifyProtos.CloseEventProto close =
-                InotifyProtos.CloseEventProto.parseFrom(p.getContents());
-            events.add(new Event.CloseEvent(close.getPath(),
-                close.getFileSize(), close.getTimestamp()));
-            break;
-          case EVENT_CREATE:
-            InotifyProtos.CreateEventProto create =
-                InotifyProtos.CreateEventProto.parseFrom(p.getContents());
-            events.add(new Event.CreateEvent.Builder()
-                .iNodeType(createTypeConvert(create.getType()))
-                .path(create.getPath())
-                .ctime(create.getCtime())
-                .ownerName(create.getOwnerName())
-                .groupName(create.getGroupName())
-                .perms(convert(create.getPerms()))
-                .replication(create.getReplication())
-                .symlinkTarget(create.getSymlinkTarget().isEmpty() ? null :
-                    create.getSymlinkTarget())
-                .defaultBlockSize(create.getDefaultBlockSize())
-                .overwrite(create.getOverwrite()).build());
-            break;
-          case EVENT_METADATA:
-            InotifyProtos.MetadataUpdateEventProto meta =
-                InotifyProtos.MetadataUpdateEventProto.parseFrom(p.getContents());
-            events.add(new Event.MetadataUpdateEvent.Builder()
-                .path(meta.getPath())
-                .metadataType(metadataUpdateTypeConvert(meta.getType()))
-                .mtime(meta.getMtime())
-                .atime(meta.getAtime())
-                .replication(meta.getReplication())
-                .ownerName(
-                    meta.getOwnerName().isEmpty() ? null : meta.getOwnerName())
-                .groupName(
-                    meta.getGroupName().isEmpty() ? null : meta.getGroupName())
-                .perms(meta.hasPerms() ? convert(meta.getPerms()) : null)
-                .acls(meta.getAclsList().isEmpty() ? null : convertAclEntry(
-                    meta.getAclsList()))
-                .xAttrs(meta.getXAttrsList().isEmpty() ? null : convertXAttrs(
-                    meta.getXAttrsList()))
-                .xAttrsRemoved(meta.getXAttrsRemoved())
-                .build());
-            break;
-          case EVENT_RENAME:
-            InotifyProtos.RenameEventProto rename =
-                InotifyProtos.RenameEventProto.parseFrom(p.getContents());
-            events.add(new Event.RenameEvent.Builder()
-                  .srcPath(rename.getSrcPath())
-                  .dstPath(rename.getDestPath())
-                  .timestamp(rename.getTimestamp())
-                  .build());
-            break;
-          case EVENT_APPEND:
-            InotifyProtos.AppendEventProto append =
-                InotifyProtos.AppendEventProto.parseFrom(p.getContents());
-            events.add(new Event.AppendEvent.Builder().path(append.getPath())
-                .newBlock(append.hasNewBlock() && append.getNewBlock())
-                .build());
-            break;
-          case EVENT_UNLINK:
-            InotifyProtos.UnlinkEventProto unlink =
-                InotifyProtos.UnlinkEventProto.parseFrom(p.getContents());
-            events.add(new Event.UnlinkEvent.Builder()
-                  .path(unlink.getPath())
-                  .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());
-        }
-      }
-      batches.add(new EventBatch(txid, events.toArray(new Event[0])));
-    }
-    return new EventBatchList(batches, resp.getEventsList().getFirstTxid(),
-        resp.getEventsList().getLastTxid(), resp.getEventsList().getSyncTxid());
-  }
-
-  public static GetEditsFromTxidResponseProto convertEditsResponse(EventBatchList el) {
-    InotifyProtos.EventsListProto.Builder builder =
-        InotifyProtos.EventsListProto.newBuilder();
-    for (EventBatch b : el.getBatches()) {
-      List<InotifyProtos.EventProto> events = Lists.newArrayList();
-      for (Event e : b.getEvents()) {
-        switch (e.getEventType()) {
-          case CLOSE:
-            Event.CloseEvent ce = (Event.CloseEvent) e;
-            events.add(InotifyProtos.EventProto.newBuilder()
-                .setType(InotifyProtos.EventType.EVENT_CLOSE)
-                .setContents(
-                    InotifyProtos.CloseEventProto.newBuilder()
-                        .setPath(ce.getPath())
-                        .setFileSize(ce.getFileSize())
-                        .setTimestamp(ce.getTimestamp()).build().toByteString()
-                ).build());
-            break;
-          case CREATE:
-            Event.CreateEvent ce2 = (Event.CreateEvent) e;
-            events.add(InotifyProtos.EventProto.newBuilder()
-                .setType(InotifyProtos.EventType.EVENT_CREATE)
-                .setContents(
-                    InotifyProtos.CreateEventProto.newBuilder()
-                        .setType(createTypeConvert(ce2.getiNodeType()))
-                        .setPath(ce2.getPath())
-                        .setCtime(ce2.getCtime())
-                        .setOwnerName(ce2.getOwnerName())
-                        .setGroupName(ce2.getGroupName())
-                        .setPerms(convert(ce2.getPerms()))
-                        .setReplication(ce2.getReplication())
-                        .setSymlinkTarget(ce2.getSymlinkTarget() == null ?
-                            "" : ce2.getSymlinkTarget())
-                        .setDefaultBlockSize(ce2.getDefaultBlockSize())
-                        .setOverwrite(ce2.getOverwrite()).build().toByteString()
-                ).build());
-            break;
-          case METADATA:
-            Event.MetadataUpdateEvent me = (Event.MetadataUpdateEvent) e;
-            InotifyProtos.MetadataUpdateEventProto.Builder metaB =
-                InotifyProtos.MetadataUpdateEventProto.newBuilder()
-                    .setPath(me.getPath())
-                    .setType(metadataUpdateTypeConvert(me.getMetadataType()))
-                    .setMtime(me.getMtime())
-                    .setAtime(me.getAtime())
-                    .setReplication(me.getReplication())
-                    .setOwnerName(me.getOwnerName() == null ? "" :
-                        me.getOwnerName())
-                    .setGroupName(me.getGroupName() == null ? "" :
-                        me.getGroupName())
-                    .addAllAcls(me.getAcls() == null ?
-                        Lists.<AclEntryProto>newArrayList() :
-                        convertAclEntryProto(me.getAcls()))
-                    .addAllXAttrs(me.getxAttrs() == null ?
-                        Lists.<XAttrProto>newArrayList() :
-                        convertXAttrProto(me.getxAttrs()))
-                    .setXAttrsRemoved(me.isxAttrsRemoved());
-            if (me.getPerms() != null) {
-              metaB.setPerms(convert(me.getPerms()));
-            }
-            events.add(InotifyProtos.EventProto.newBuilder()
-                .setType(InotifyProtos.EventType.EVENT_METADATA)
-                .setContents(metaB.build().toByteString())
-                .build());
-            break;
-          case RENAME:
-            Event.RenameEvent re = (Event.RenameEvent) e;
-            events.add(InotifyProtos.EventProto.newBuilder()
-                .setType(InotifyProtos.EventType.EVENT_RENAME)
-                .setContents(
-                    InotifyProtos.RenameEventProto.newBuilder()
-                        .setSrcPath(re.getSrcPath())
-                        .setDestPath(re.getDstPath())
-                        .setTimestamp(re.getTimestamp()).build().toByteString()
-                ).build());
-            break;
-          case APPEND:
-            Event.AppendEvent re2 = (Event.AppendEvent) 

<TRUNCATED>

[17/50] [abbrv] hadoop git commit: MAPREDUCE-5982. Task attempts that fail from the ASSIGNED state can disappear. Contributed by Chang Li

Posted by zh...@apache.org.
MAPREDUCE-5982. Task attempts that fail from the ASSIGNED state can disappear. Contributed by Chang Li


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

Branch: refs/heads/HDFS-7285
Commit: ee4ee6af6a5a6299d27462adb6944206039bbbae
Parents: 9eee975
Author: Jason Lowe <jl...@apache.org>
Authored: Thu Sep 17 21:37:39 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Thu Sep 17 21:37:39 2015 +0000

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |   3 +
 .../v2/app/job/impl/TaskAttemptImpl.java        |  92 +++++------
 .../apache/hadoop/mapreduce/v2/app/MRApp.java   |  11 +-
 .../v2/app/job/impl/TestTaskAttempt.java        | 154 +++++++++++++++++++
 4 files changed, 213 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee4ee6af/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 6cf7abb..cd84a34 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -573,6 +573,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-5002. AM could potentially allocate a reduce container to a map
     attempt (Chang Li via jlowe)
 
+    MAPREDUCE-5982. Task attempts that fail from the ASSIGNED state can
+    disappear (Chang Li via jlowe)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee4ee6af/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 77a7555..a7becdb 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
@@ -1484,6 +1484,19 @@ public abstract class TaskAttemptImpl implements
     return tauce;
   }
 
+  private static void
+      sendJHStartEventForAssignedFailTask(TaskAttemptImpl taskAttempt) {
+    TaskAttemptContainerLaunchedEvent event;
+    taskAttempt.launchTime = taskAttempt.clock.getTime();
+
+    InetSocketAddress nodeHttpInetAddr =
+        NetUtils.createSocketAddr(taskAttempt.container.getNodeHttpAddress());
+    taskAttempt.trackerName = nodeHttpInetAddr.getHostName();
+    taskAttempt.httpPort = nodeHttpInetAddr.getPort();
+    taskAttempt.sendLaunchedEvents();
+  }
+
+
   @SuppressWarnings("unchecked")
   private void sendLaunchedEvents() {
     JobCounterUpdateEvent jce = new JobCounterUpdateEvent(attemptId.getTaskId()
@@ -1681,6 +1694,9 @@ public abstract class TaskAttemptImpl implements
     @Override
     public void transition(TaskAttemptImpl taskAttempt, 
         TaskAttemptEvent event) {
+      if (taskAttempt.getLaunchTime() == 0) {
+        sendJHStartEventForAssignedFailTask(taskAttempt);
+      }
       //set the finish time
       taskAttempt.setFinishTime();
 
@@ -1715,23 +1731,19 @@ public abstract class TaskAttemptImpl implements
         default:
           LOG.error("Task final state is not FAILED or KILLED: " + finalState);
       }
-      if (taskAttempt.getLaunchTime() != 0) {
-        TaskAttemptUnsuccessfulCompletionEvent tauce =
-            createTaskAttemptUnsuccessfulCompletionEvent(taskAttempt,
-                finalState);
-        if(finalState == TaskAttemptStateInternal.FAILED) {
-          taskAttempt.eventHandler
-            .handle(createJobCounterUpdateEventTAFailed(taskAttempt, false));
-        } else if(finalState == TaskAttemptStateInternal.KILLED) {
-          taskAttempt.eventHandler
-          .handle(createJobCounterUpdateEventTAKilled(taskAttempt, false));
-        }
-        taskAttempt.eventHandler.handle(new JobHistoryEvent(
-            taskAttempt.attemptId.getTaskId().getJobId(), tauce));
-      } else {
-        LOG.debug("Not generating HistoryFinish event since start event not " +
-            "generated for taskAttempt: " + taskAttempt.getID());
+
+      TaskAttemptUnsuccessfulCompletionEvent tauce =
+          createTaskAttemptUnsuccessfulCompletionEvent(taskAttempt,
+              finalState);
+      if(finalState == TaskAttemptStateInternal.FAILED) {
+        taskAttempt.eventHandler
+          .handle(createJobCounterUpdateEventTAFailed(taskAttempt, false));
+      } else if(finalState == TaskAttemptStateInternal.KILLED) {
+        taskAttempt.eventHandler
+        .handle(createJobCounterUpdateEventTAKilled(taskAttempt, false));
       }
+      taskAttempt.eventHandler.handle(new JobHistoryEvent(
+          taskAttempt.attemptId.getTaskId().getJobId(), tauce));
     }
   }
 
@@ -2023,27 +2035,25 @@ public abstract class TaskAttemptImpl implements
     @Override
     public void transition(TaskAttemptImpl taskAttempt,
         TaskAttemptEvent event) {
+      if (taskAttempt.getLaunchTime() == 0) {
+        sendJHStartEventForAssignedFailTask(taskAttempt);
+      }
       //set the finish time
       taskAttempt.setFinishTime();
-      if (taskAttempt.getLaunchTime() != 0) {
-        taskAttempt.eventHandler
-            .handle(createJobCounterUpdateEventTAKilled(taskAttempt, false));
-        TaskAttemptUnsuccessfulCompletionEvent tauce =
-            createTaskAttemptUnsuccessfulCompletionEvent(taskAttempt,
-                TaskAttemptStateInternal.KILLED);
-        taskAttempt.eventHandler.handle(new JobHistoryEvent(
-            taskAttempt.attemptId.getTaskId().getJobId(), tauce));
-      }else {
-        LOG.debug("Not generating HistoryFinish event since start event not " +
-            "generated for taskAttempt: " + taskAttempt.getID());
-      }
+
+      taskAttempt.eventHandler
+          .handle(createJobCounterUpdateEventTAKilled(taskAttempt, false));
+      TaskAttemptUnsuccessfulCompletionEvent tauce =
+          createTaskAttemptUnsuccessfulCompletionEvent(taskAttempt,
+              TaskAttemptStateInternal.KILLED);
+      taskAttempt.eventHandler.handle(new JobHistoryEvent(
+          taskAttempt.attemptId.getTaskId().getJobId(), tauce));
 
       if (event instanceof TaskAttemptKillEvent) {
         taskAttempt.addDiagnosticInfo(
             ((TaskAttemptKillEvent) event).getMessage());
       }
 
-//      taskAttempt.logAttemptFinishedEvent(TaskAttemptStateInternal.KILLED); Not logging Map/Reduce attempts in case of failure.
       taskAttempt.eventHandler.handle(new TaskTAttemptEvent(
           taskAttempt.attemptId,
           TaskEventType.T_ATTEMPT_KILLED));
@@ -2178,23 +2188,19 @@ public abstract class TaskAttemptImpl implements
 
   @SuppressWarnings("unchecked")
   private static void notifyTaskAttemptFailed(TaskAttemptImpl taskAttempt) {
+    if (taskAttempt.getLaunchTime() == 0) {
+      sendJHStartEventForAssignedFailTask(taskAttempt);
+    }
     // set the finish time
     taskAttempt.setFinishTime();
+    taskAttempt.eventHandler
+        .handle(createJobCounterUpdateEventTAFailed(taskAttempt, false));
+    TaskAttemptUnsuccessfulCompletionEvent tauce =
+        createTaskAttemptUnsuccessfulCompletionEvent(taskAttempt,
+            TaskAttemptStateInternal.FAILED);
+    taskAttempt.eventHandler.handle(new JobHistoryEvent(
+        taskAttempt.attemptId.getTaskId().getJobId(), tauce));
 
-    if (taskAttempt.getLaunchTime() != 0) {
-      taskAttempt.eventHandler
-          .handle(createJobCounterUpdateEventTAFailed(taskAttempt, false));
-      TaskAttemptUnsuccessfulCompletionEvent tauce =
-          createTaskAttemptUnsuccessfulCompletionEvent(taskAttempt,
-              TaskAttemptStateInternal.FAILED);
-      taskAttempt.eventHandler.handle(new JobHistoryEvent(
-          taskAttempt.attemptId.getTaskId().getJobId(), tauce));
-      // taskAttempt.logAttemptFinishedEvent(TaskAttemptStateInternal.FAILED); Not
-      // handling failed map/reduce events.
-    }else {
-      LOG.debug("Not generating HistoryFinish event since start event not " +
-          "generated for taskAttempt: " + taskAttempt.getID());
-    }
     taskAttempt.eventHandler.handle(new TaskTAttemptEvent(
         taskAttempt.attemptId, TaskEventType.T_ATTEMPT_FAILED));
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee4ee6af/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
index b51adf2..f0c10d3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
@@ -544,10 +544,7 @@ public class MRApp extends MRAppMaster {
     public void handle(ContainerLauncherEvent event) {
       switch (event.getType()) {
       case CONTAINER_REMOTE_LAUNCH:
-        getContext().getEventHandler().handle(
-            new TaskAttemptContainerLaunchedEvent(event.getTaskAttemptID(),
-                shufflePort));
-        
+        containerLaunched(event.getTaskAttemptID(), shufflePort);
         attemptLaunched(event.getTaskAttemptID());
         break;
       case CONTAINER_REMOTE_CLEANUP:
@@ -561,6 +558,12 @@ public class MRApp extends MRAppMaster {
     }
   }
 
+  protected void containerLaunched(TaskAttemptId attemptID, int shufflePort) {
+    getContext().getEventHandler().handle(
+      new TaskAttemptContainerLaunchedEvent(attemptID,
+          shufflePort));
+  }
+
   protected void attemptLaunched(TaskAttemptId attemptID) {
     if (autoComplete) {
       // send the done event

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee4ee6af/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
index a88a935..6b4656a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
@@ -115,6 +115,69 @@ public class TestTaskAttempt{
   }
 
   @Test
+  public void testMRAppHistoryForTAFailedInAssigned() throws Exception {
+    // test TA_CONTAINER_LAUNCH_FAILED for map
+    FailingAttemptsDuringAssignedMRApp app =
+        new FailingAttemptsDuringAssignedMRApp(1, 0,
+            TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED);
+    testTaskAttemptAssignedFailHistory(app);
+
+    // test TA_CONTAINER_LAUNCH_FAILED for reduce
+    app =
+        new FailingAttemptsDuringAssignedMRApp(0, 1,
+            TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED);
+    testTaskAttemptAssignedFailHistory(app);
+
+    // test TA_CONTAINER_COMPLETED for map
+    app =
+        new FailingAttemptsDuringAssignedMRApp(1, 0,
+            TaskAttemptEventType.TA_CONTAINER_COMPLETED);
+    testTaskAttemptAssignedFailHistory(app);
+
+    // test TA_CONTAINER_COMPLETED for reduce
+    app =
+        new FailingAttemptsDuringAssignedMRApp(0, 1,
+            TaskAttemptEventType.TA_CONTAINER_COMPLETED);
+    testTaskAttemptAssignedFailHistory(app);
+
+    // test TA_FAILMSG for map
+    app =
+        new FailingAttemptsDuringAssignedMRApp(1, 0,
+            TaskAttemptEventType.TA_FAILMSG);
+    testTaskAttemptAssignedFailHistory(app);
+
+    // test TA_FAILMSG for reduce
+    app =
+        new FailingAttemptsDuringAssignedMRApp(0, 1,
+            TaskAttemptEventType.TA_FAILMSG);
+    testTaskAttemptAssignedFailHistory(app);
+
+    // test TA_FAILMSG_BY_CLIENT for map
+    app =
+        new FailingAttemptsDuringAssignedMRApp(1, 0,
+            TaskAttemptEventType.TA_FAILMSG_BY_CLIENT);
+    testTaskAttemptAssignedFailHistory(app);
+
+    // test TA_FAILMSG_BY_CLIENT for reduce
+    app =
+        new FailingAttemptsDuringAssignedMRApp(0, 1,
+            TaskAttemptEventType.TA_FAILMSG_BY_CLIENT);
+    testTaskAttemptAssignedFailHistory(app);
+
+    // test TA_KILL for map
+    app =
+        new FailingAttemptsDuringAssignedMRApp(1, 0,
+            TaskAttemptEventType.TA_KILL);
+    testTaskAttemptAssignedKilledHistory(app);
+
+    // test TA_KILL for reduce
+    app =
+        new FailingAttemptsDuringAssignedMRApp(0, 1,
+            TaskAttemptEventType.TA_KILL);
+    testTaskAttemptAssignedKilledHistory(app);
+  }
+
+  @Test
   public void testSingleRackRequest() throws Exception {
     TaskAttemptImpl.RequestContainerTransition rct =
         new TaskAttemptImpl.RequestContainerTransition(false);
@@ -301,6 +364,31 @@ public class TestTaskAttempt{
         report.getTaskAttemptState());
   }
 
+  private void testTaskAttemptAssignedFailHistory
+      (FailingAttemptsDuringAssignedMRApp app) throws Exception {
+    Configuration conf = new Configuration();
+    Job job = app.submit(conf);
+    app.waitForState(job, JobState.FAILED);
+    Map<TaskId, Task> tasks = job.getTasks();
+    Assert.assertTrue("No Ta Started JH Event", app.getTaStartJHEvent());
+    Assert.assertTrue("No Ta Failed JH Event", app.getTaFailedJHEvent());
+  }
+
+  private void testTaskAttemptAssignedKilledHistory
+      (FailingAttemptsDuringAssignedMRApp app) throws Exception {
+    Configuration conf = new Configuration();
+    Job job = app.submit(conf);
+    app.waitForState(job, JobState.RUNNING);
+    Map<TaskId, Task> tasks = job.getTasks();
+    Task task = tasks.values().iterator().next();
+    app.waitForState(task, TaskState.SCHEDULED);
+    Map<TaskAttemptId, TaskAttempt> attempts = task.getAttempts();
+    TaskAttempt attempt = attempts.values().iterator().next();
+    app.waitForState(attempt, TaskAttemptState.KILLED);
+    Assert.assertTrue("No Ta Started JH Event", app.getTaStartJHEvent());
+    Assert.assertTrue("No Ta Killed JH Event", app.getTaKilledJHEvent());
+  }
+
   static class FailingAttemptsMRApp extends MRApp {
     FailingAttemptsMRApp(int maps, int reduces) {
       super(maps, reduces, true, "FailingAttemptsMRApp", true);
@@ -331,6 +419,72 @@ public class TestTaskAttempt{
     }
   }
 
+  static class FailingAttemptsDuringAssignedMRApp extends MRApp {
+    FailingAttemptsDuringAssignedMRApp(int maps, int reduces,
+        TaskAttemptEventType event) {
+      super(maps, reduces, true, "FailingAttemptsMRApp", true);
+      sendFailEvent = event;
+    }
+
+   TaskAttemptEventType sendFailEvent;
+
+   @Override
+    protected void containerLaunched(TaskAttemptId attemptID,
+        int shufflePort) {
+      //do nothing, not send TA_CONTAINER_LAUNCHED event
+    }
+
+    @Override
+    protected void attemptLaunched(TaskAttemptId attemptID) {
+      getContext().getEventHandler().handle(
+          new TaskAttemptEvent(attemptID, sendFailEvent));
+    }
+
+    private boolean receiveTaStartJHEvent = false;
+    private boolean receiveTaFailedJHEvent = false;
+    private boolean receiveTaKilledJHEvent = false;
+
+    public boolean getTaStartJHEvent(){
+      return receiveTaStartJHEvent;
+    }
+
+    public boolean getTaFailedJHEvent(){
+      return receiveTaFailedJHEvent;
+    }
+
+    public boolean getTaKilledJHEvent(){
+        return receiveTaKilledJHEvent;
+    }
+
+    protected EventHandler<JobHistoryEvent> createJobHistoryHandler(
+        AppContext context) {
+      return new EventHandler<JobHistoryEvent>() {
+        @Override
+        public void handle(JobHistoryEvent event) {
+          if (event.getType() == org.apache.hadoop.mapreduce.jobhistory.
+              EventType.MAP_ATTEMPT_FAILED) {
+            receiveTaFailedJHEvent = true;
+          } else if (event.getType() == org.apache.hadoop.mapreduce.
+              jobhistory.EventType.MAP_ATTEMPT_KILLED) {
+            receiveTaKilledJHEvent = true;
+          } else if (event.getType() == org.apache.hadoop.mapreduce.
+              jobhistory.EventType.MAP_ATTEMPT_STARTED) {
+            receiveTaStartJHEvent = true;
+          } else if (event.getType() == org.apache.hadoop.mapreduce.
+              jobhistory.EventType.REDUCE_ATTEMPT_FAILED) {
+            receiveTaFailedJHEvent = true;
+          } else if (event.getType() == org.apache.hadoop.mapreduce.
+                  jobhistory.EventType.REDUCE_ATTEMPT_KILLED) {
+            receiveTaKilledJHEvent = true;
+          } else if (event.getType() == org.apache.hadoop.mapreduce.
+              jobhistory.EventType.REDUCE_ATTEMPT_STARTED) {
+            receiveTaStartJHEvent = true;
+          }
+        }
+      };
+    }
+  }
+
   @Test
   public void testLaunchFailedWhileKilling() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(1, 2);


[11/50] [abbrv] hadoop git commit: MAPREDUCE-6477. Replace usage of deprecated NameNode.DEFAULT_PORT in TestFileSystem. Contributed by Mingliang Liu.

Posted by zh...@apache.org.
MAPREDUCE-6477. Replace usage of deprecated NameNode.DEFAULT_PORT in TestFileSystem. Contributed by Mingliang Liu.


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

Branch: refs/heads/HDFS-7285
Commit: 941001bfa63a2ed32b062aa639d72e76672952ee
Parents: 07f3044
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Sep 16 13:06:26 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Wed Sep 16 13:06:26 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/941001bf/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index dd7a34a..669fee5 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -381,6 +381,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6443. Add JvmPauseMonitor to JobHistoryServer. (Robert Kanter
     via junping_du)
 
+    MAPREDUCE-6477. Replace usage of deprecated NameNode.DEFAULT_PORT in
+    TestFileSystem. (Mingliang Liu via wheat9)
+
   OPTIMIZATIONS
 
     MAPREDUCE-6376. Add avro binary support for jhist files (Ray Chiang via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/941001bf/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestFileSystem.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestFileSystem.java
index 92441ab..4146b13 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestFileSystem.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestFileSystem.java
@@ -40,7 +40,7 @@ import org.apache.commons.logging.Log;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.fs.shell.CommandFormat;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.SequenceFile;
@@ -510,10 +510,10 @@ public class TestFileSystem extends TestCase {
     
     {
       try {
-        runTestCache(NameNode.DEFAULT_PORT);
+        runTestCache(HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT);
       } catch(java.net.BindException be) {
-        LOG.warn("Cannot test NameNode.DEFAULT_PORT (="
-            + NameNode.DEFAULT_PORT + ")", be);
+        LOG.warn("Cannot test HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT (="
+            + HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT + ")", be);
       }
 
       runTestCache(0);
@@ -537,11 +537,11 @@ public class TestFileSystem extends TestCase {
         }
       }
       
-      if (port == NameNode.DEFAULT_PORT) {
+      if (port == HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT) {
         //test explicit default port
         URI uri2 = new URI(uri.getScheme(), uri.getUserInfo(),
-            uri.getHost(), NameNode.DEFAULT_PORT, uri.getPath(),
-            uri.getQuery(), uri.getFragment());  
+            uri.getHost(), HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT,
+            uri.getPath(), uri.getQuery(), uri.getFragment());
         LOG.info("uri2=" + uri2);
         FileSystem fs = FileSystem.get(uri2, conf);
         checkPath(cluster, fs);


[32/50] [abbrv] hadoop git commit: YARN-3920. FairScheduler container reservation on a node should be configurable to limit it to large containers (adhoot via asuresh)

Posted by zh...@apache.org.
YARN-3920. FairScheduler container reservation on a node should be configurable to limit it to large containers (adhoot via asuresh)


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

Branch: refs/heads/HDFS-7285
Commit: 94dec5a9164cd9bc573fbf74e76bcff9e7c5c637
Parents: 602335d
Author: Arun Suresh <as...@apache.org>
Authored: Fri Sep 18 14:00:49 2015 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Fri Sep 18 14:02:55 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../scheduler/fair/FSAppAttempt.java            | 19 ++++-
 .../scheduler/fair/FairScheduler.java           | 21 ++++-
 .../fair/FairSchedulerConfiguration.java        | 22 ++++-
 .../scheduler/fair/FairSchedulerTestBase.java   |  6 ++
 .../scheduler/fair/TestFairScheduler.java       | 89 +++++++++++++++++++-
 6 files changed, 148 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/94dec5a9/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 822624f..7487f71 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -454,6 +454,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6478. Add an option to skip cleanupJob stage or ignore cleanup
     failure during commitJob. (Junping Du via wangda)
 
+    YARN-3920. FairScheduler container reservation on a node should be
+    configurable to limit it to large containers (adhoot via asuresh)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94dec5a9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.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/FSAppAttempt.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/FSAppAttempt.java
index cfec915..7af1891 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/FSAppAttempt.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/FSAppAttempt.java
@@ -543,10 +543,23 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
       return container.getResource();
     }
 
-    // The desired container won't fit here, so reserve
-    reserve(request.getPriority(), node, container, reserved);
+    if (isReservable(container)) {
+      // The desired container won't fit here, so reserve
+      reserve(request.getPriority(), node, container, reserved);
 
-    return FairScheduler.CONTAINER_RESERVED;
+      return FairScheduler.CONTAINER_RESERVED;
+    } else {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Not creating reservation as container " + container.getId()
+            + " is not reservable");
+      }
+      return Resources.none();
+    }
+  }
+
+  private boolean isReservable(Container container) {
+    return scheduler.isAtLeastReservationThreshold(
+      getQueue().getPolicy().getResourceCalculator(), container.getResource());
   }
 
   private boolean hasNodeOrRackLocalRequests(Priority priority) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94dec5a9/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 3a39799..a083272 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
@@ -194,7 +194,11 @@ public class FairScheduler extends
   private AllocationFileLoaderService allocsLoader;
   @VisibleForTesting
   AllocationConfiguration allocConf;
-  
+
+  // Container size threshold for making a reservation.
+  @VisibleForTesting
+  Resource reservationThreshold;
+
   public FairScheduler() {
     super(FairScheduler.class.getName());
     clock = new SystemClock();
@@ -203,6 +207,12 @@ public class FairScheduler extends
     maxRunningEnforcer = new MaxRunningAppsEnforcer(this);
   }
 
+  public boolean isAtLeastReservationThreshold(
+      ResourceCalculator resourceCalculator, Resource resource) {
+    return Resources.greaterThanOrEqual(
+        resourceCalculator, clusterResource, resource, reservationThreshold);
+  }
+
   private void validateConf(Configuration conf) {
     // validate scheduler memory allocation setting
     int minMem = conf.getInt(
@@ -1325,6 +1335,7 @@ public class FairScheduler extends
       minimumAllocation = this.conf.getMinimumAllocation();
       initMaximumResourceCapability(this.conf.getMaximumAllocation());
       incrAllocation = this.conf.getIncrementAllocation();
+      updateReservationThreshold();
       continuousSchedulingEnabled = this.conf.isContinuousSchedulingEnabled();
       continuousSchedulingSleepMs =
           this.conf.getContinuousSchedulingSleepMs();
@@ -1391,6 +1402,14 @@ public class FairScheduler extends
     }
   }
 
+  private void updateReservationThreshold() {
+    Resource newThreshold = Resources.multiply(
+        getIncrementResourceCapability(),
+        this.conf.getReservationThresholdIncrementMultiple());
+
+    reservationThreshold = newThreshold;
+  }
+
   private synchronized void startSchedulerThreads() {
     Preconditions.checkNotNull(updateThread, "updateThread is null");
     Preconditions.checkNotNull(allocsLoader, "allocsLoader is null");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94dec5a9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.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/FairSchedulerConfiguration.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/FairSchedulerConfiguration.java
index e477e6e..892484d 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/FairSchedulerConfiguration.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/FairSchedulerConfiguration.java
@@ -18,8 +18,6 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
 import java.io.File;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -49,7 +47,17 @@ public class FairSchedulerConfiguration extends Configuration {
   public static final String RM_SCHEDULER_INCREMENT_ALLOCATION_VCORES =
     YarnConfiguration.YARN_PREFIX + "scheduler.increment-allocation-vcores";
   public static final int DEFAULT_RM_SCHEDULER_INCREMENT_ALLOCATION_VCORES = 1;
-  
+
+  /** Threshold for container size for making a container reservation as a
+   * multiple of increment allocation. Only container sizes above this are
+   * allowed to reserve a node */
+  public static final String
+      RM_SCHEDULER_RESERVATION_THRESHOLD_INCERMENT_MULTIPLE =
+      YarnConfiguration.YARN_PREFIX +
+          "scheduler.reservation-threshold.increment-multiple";
+  public static final float
+      DEFAULT_RM_SCHEDULER_RESERVATION_THRESHOLD_INCREMENT_MULTIPLE = 2f;
+
   private static final String CONF_PREFIX =  "yarn.scheduler.fair.";
 
   public static final String ALLOCATION_FILE = CONF_PREFIX + "allocation.file";
@@ -166,7 +174,13 @@ public class FairSchedulerConfiguration extends Configuration {
       DEFAULT_RM_SCHEDULER_INCREMENT_ALLOCATION_VCORES);
     return Resources.createResource(incrementMemory, incrementCores);
   }
-  
+
+  public float getReservationThresholdIncrementMultiple() {
+    return getFloat(
+      RM_SCHEDULER_RESERVATION_THRESHOLD_INCERMENT_MULTIPLE,
+      DEFAULT_RM_SCHEDULER_RESERVATION_THRESHOLD_INCREMENT_MULTIPLE);
+  }
+
   public float getLocalityThresholdNode() {
     return getFloat(LOCALITY_THRESHOLD_NODE, DEFAULT_LOCALITY_THRESHOLD_NODE);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94dec5a9/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 1c9801d..dd7ed41 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
@@ -64,6 +64,7 @@ public class FairSchedulerTestBase {
   protected Configuration conf;
   protected FairScheduler scheduler;
   protected ResourceManager resourceManager;
+  public static final float TEST_RESERVATION_THRESHOLD = 0.09f;
 
   // Helper methods
   public Configuration createConfiguration() {
@@ -76,6 +77,11 @@ public class FairSchedulerTestBase {
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 10240);
     conf.setBoolean(FairSchedulerConfiguration.ASSIGN_MULTIPLE, false);
     conf.setFloat(FairSchedulerConfiguration.PREEMPTION_THRESHOLD, 0f);
+
+    conf.setFloat(
+        FairSchedulerConfiguration
+           .RM_SCHEDULER_RESERVATION_THRESHOLD_INCERMENT_MULTIPLE,
+        TEST_RESERVATION_THRESHOLD);
     return conf;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94dec5a9/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 a02cf18..ad54616 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
@@ -710,9 +710,10 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     scheduler.handle(updateEvent);
 
     // Asked for less than increment allocation.
-    assertEquals(FairSchedulerConfiguration.DEFAULT_RM_SCHEDULER_INCREMENT_ALLOCATION_MB,
+    assertEquals(
+        FairSchedulerConfiguration.DEFAULT_RM_SCHEDULER_INCREMENT_ALLOCATION_MB,
         scheduler.getQueueManager().getQueue("queue1").
-        getResourceUsage().getMemory());
+            getResourceUsage().getMemory());
 
     NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2);
     scheduler.handle(updateEvent2);
@@ -764,7 +765,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
 
     // Make sure queue 2 is waiting with a reservation
     assertEquals(0, scheduler.getQueueManager().getQueue("queue2").
-      getResourceUsage().getMemory());
+        getResourceUsage().getMemory());
     assertEquals(1024, scheduler.getSchedulerApp(attId).getCurrentReservation().getMemory());
 
     // Now another node checks in with capacity
@@ -939,8 +940,88 @@ public class TestFairScheduler extends FairSchedulerTestBase {
         getResourceUsage().getMemory());
   }
 
-  
+  @Test
+  public void testReservationThresholdGatesReservations() 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("<defaultQueueSchedulingPolicy>drf" +
+        "</defaultQueueSchedulingPolicy>");
+    out.println("</allocations>");
+    out.close();
 
+    // Set threshold to 2 * 1024 ==> 2048 MB & 2 * 1 ==> 2 vcores (test will
+    // use vcores)
+    conf.setFloat(FairSchedulerConfiguration.
+            RM_SCHEDULER_RESERVATION_THRESHOLD_INCERMENT_MULTIPLE,
+        2f);
+    scheduler.init(conf);
+    scheduler.start();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
+    // Add a node
+    RMNode node1 =
+        MockNodes
+            .newNodeInfo(1, Resources.createResource(4096, 4), 1, "127.0.0.1");
+    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
+    scheduler.handle(nodeEvent1);
+
+    // Queue 1 requests full capacity of node
+    createSchedulingRequest(4096, 4, "queue1", "user1", 1, 1);
+    scheduler.update();
+    NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
+
+    scheduler.handle(updateEvent);
+
+    // Make sure queue 1 is allocated app capacity
+    assertEquals(4096, scheduler.getQueueManager().getQueue("queue1").
+        getResourceUsage().getMemory());
+
+    // Now queue 2 requests below threshold
+    ApplicationAttemptId attId = createSchedulingRequest(1024, "queue2", "user1", 1);
+    scheduler.update();
+    scheduler.handle(updateEvent);
+
+    // Make sure queue 2 has no reservation
+    assertEquals(0, scheduler.getQueueManager().getQueue("queue2").
+        getResourceUsage().getMemory());
+    assertEquals(0,
+        scheduler.getSchedulerApp(attId).getReservedContainers().size());
+
+    // Now queue requests CPU above threshold
+    createSchedulingRequestExistingApplication(1024, 3, 1, attId);
+    scheduler.update();
+    scheduler.handle(updateEvent);
+
+    // Make sure queue 2 is waiting with a reservation
+    assertEquals(0, scheduler.getQueueManager().getQueue("queue2").
+        getResourceUsage().getMemory());
+    assertEquals(3, scheduler.getSchedulerApp(attId).getCurrentReservation()
+        .getVirtualCores());
+
+    // Now another node checks in with capacity
+    RMNode node2 =
+        MockNodes
+            .newNodeInfo(1, Resources.createResource(1024, 4), 2, "127.0.0.2");
+    NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
+    NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2);
+    scheduler.handle(nodeEvent2);
+    scheduler.handle(updateEvent2);
+
+    // Make sure this goes to queue 2
+    assertEquals(3, scheduler.getQueueManager().getQueue("queue2").
+        getResourceUsage().getVirtualCores());
+
+    // The old reservation should still be there...
+    assertEquals(3, scheduler.getSchedulerApp(attId).getCurrentReservation()
+        .getVirtualCores());
+    // ... but it should disappear when we update the first node.
+    scheduler.handle(updateEvent);
+    assertEquals(0, scheduler.getSchedulerApp(attId).getCurrentReservation()
+        .getVirtualCores());
+  }
 
   @Test
   public void testEmptyQueueName() throws Exception {


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

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
index 7a7cd24,0000000..dabae2c
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
@@@ -1,1013 -1,0 +1,1014 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hdfs.server.datanode.erasurecode;
 +
 +import java.io.BufferedOutputStream;
 +import java.io.DataInputStream;
 +import java.io.DataOutputStream;
 +import java.io.IOException;
 +import java.io.InputStream;
 +import java.io.OutputStream;
 +import java.net.InetSocketAddress;
 +import java.net.Socket;
 +import java.nio.ByteBuffer;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.BitSet;
 +import java.util.Collection;
 +import java.util.EnumSet;
 +import java.util.HashMap;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.concurrent.Callable;
 +import java.util.concurrent.CompletionService;
 +import java.util.concurrent.ExecutorCompletionService;
 +import java.util.concurrent.Future;
 +import java.util.concurrent.LinkedBlockingQueue;
 +import java.util.concurrent.SynchronousQueue;
 +import java.util.concurrent.ThreadPoolExecutor;
 +import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.StorageType;
 +import org.apache.hadoop.hdfs.BlockReader;
 +import org.apache.hadoop.hdfs.DFSConfigKeys;
 +import org.apache.hadoop.hdfs.DFSPacket;
 +import org.apache.hadoop.hdfs.DFSUtil;
++import org.apache.hadoop.hdfs.DFSUtilClient;
 +import org.apache.hadoop.hdfs.RemoteBlockReader2;
 +import org.apache.hadoop.hdfs.net.Peer;
 +import org.apache.hadoop.hdfs.net.TcpPeerServer;
 +import org.apache.hadoop.hdfs.protocol.DatanodeID;
 +import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 +import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 +import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
 +import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 +import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
 +import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 +import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
 +import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 +import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 +import org.apache.hadoop.hdfs.server.datanode.DataNode;
 +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 +import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 +import org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult;
 +import org.apache.hadoop.io.IOUtils;
 +import org.apache.hadoop.io.erasurecode.CodecUtil;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
 +import org.apache.hadoop.net.NetUtils;
 +import org.apache.hadoop.security.token.Token;
 +import org.apache.hadoop.util.Daemon;
 +import org.apache.hadoop.util.DataChecksum;
 +
 +import com.google.common.base.Preconditions;
 +
 +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.convertIndex4Decode;
 +
 +/**
 + * ErasureCodingWorker handles the erasure coding recovery work commands. These
 + * commands would be issued from Namenode as part of Datanode's heart beat
 + * response. BPOfferService delegates the work to this class for handling EC
 + * commands.
 + */
 +public final class ErasureCodingWorker {
 +  private static final Log LOG = DataNode.LOG;
 +  
 +  private final DataNode datanode; 
 +  private final Configuration conf;
 +
 +  private ThreadPoolExecutor STRIPED_BLK_RECOVERY_THREAD_POOL;
 +  private ThreadPoolExecutor STRIPED_READ_THREAD_POOL;
 +  private final int STRIPED_READ_TIMEOUT_MILLIS;
 +  private final int STRIPED_READ_BUFFER_SIZE;
 +
 +  public ErasureCodingWorker(Configuration conf, DataNode datanode) {
 +    this.datanode = datanode;
 +    this.conf = conf;
 +
 +    STRIPED_READ_TIMEOUT_MILLIS = conf.getInt(
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_KEY,
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_DEFAULT);
 +    initializeStripedReadThreadPool(conf.getInt(
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THREADS_KEY, 
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THREADS_DEFAULT));
 +    STRIPED_READ_BUFFER_SIZE = conf.getInt(
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY,
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT);
 +
 +    initializeStripedBlkRecoveryThreadPool(conf.getInt(
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_KEY,
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_DEFAULT));
 +  }
 +  
 +  private RawErasureDecoder newDecoder(int numDataUnits, int numParityUnits) {
 +    return CodecUtil.createRSRawDecoder(conf, numDataUnits, numParityUnits);
 +  }
 +
 +  private void initializeStripedReadThreadPool(int num) {
 +    if (LOG.isDebugEnabled()) {
 +      LOG.debug("Using striped reads; pool threads=" + num);
 +    }
 +    STRIPED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num, 60,
 +        TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
 +        new Daemon.DaemonFactory() {
 +      private final AtomicInteger threadIndex = new AtomicInteger(0);
 +
 +      @Override
 +      public Thread newThread(Runnable r) {
 +        Thread t = super.newThread(r);
 +        t.setName("stripedRead-" + threadIndex.getAndIncrement());
 +        return t;
 +      }
 +    }, new ThreadPoolExecutor.CallerRunsPolicy() {
 +      @Override
 +      public void rejectedExecution(Runnable runnable, ThreadPoolExecutor e) {
 +        LOG.info("Execution for striped reading rejected, "
 +            + "Executing in current thread");
 +        // will run in the current thread
 +        super.rejectedExecution(runnable, e);
 +      }
 +    });
 +    STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
 +  }
 +
 +  private void initializeStripedBlkRecoveryThreadPool(int num) {
 +    if (LOG.isDebugEnabled()) {
 +      LOG.debug("Using striped block recovery; pool threads=" + num);
 +    }
 +    STRIPED_BLK_RECOVERY_THREAD_POOL = new ThreadPoolExecutor(2, num, 60,
 +        TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(),
 +        new Daemon.DaemonFactory() {
 +          private final AtomicInteger threadIdx = new AtomicInteger(0);
 +
 +          @Override
 +          public Thread newThread(Runnable r) {
 +            Thread t = super.newThread(r);
 +            t.setName("stripedBlockRecovery-" + threadIdx.getAndIncrement());
 +            return t;
 +          }
 +        });
 +    STRIPED_BLK_RECOVERY_THREAD_POOL.allowCoreThreadTimeOut(true);
 +  }
 +
 +  /**
 +   * Handles the Erasure Coding recovery work commands.
 +   * 
 +   * @param ecTasks
 +   *          BlockECRecoveryInfo
 +   */
 +  public void processErasureCodingTasks(Collection<BlockECRecoveryInfo> ecTasks) {
 +    for (BlockECRecoveryInfo recoveryInfo : ecTasks) {
 +      try {
 +        STRIPED_BLK_RECOVERY_THREAD_POOL
 +            .submit(new ReconstructAndTransferBlock(recoveryInfo));
 +      } catch (Throwable e) {
 +        LOG.warn("Failed to recover striped block "
 +            + recoveryInfo.getExtendedBlock().getLocalBlock(), e);
 +      }
 +    }
 +  }
 +
 +  /**
 +   * ReconstructAndTransferBlock recover one or more missed striped block in the
 +   * striped block group, the minimum number of live striped blocks should be
 +   * no less than data block number.
 +   * 
 +   * | <- Striped Block Group -> |
 +   *  blk_0      blk_1       blk_2(*)   blk_3   ...   <- A striped block group
 +   *    |          |           |          |  
 +   *    v          v           v          v 
 +   * +------+   +------+   +------+   +------+
 +   * |cell_0|   |cell_1|   |cell_2|   |cell_3|  ...    
 +   * +------+   +------+   +------+   +------+     
 +   * |cell_4|   |cell_5|   |cell_6|   |cell_7|  ...
 +   * +------+   +------+   +------+   +------+
 +   * |cell_8|   |cell_9|   |cell10|   |cell11|  ...
 +   * +------+   +------+   +------+   +------+
 +   *  ...         ...       ...         ...
 +   *  
 +   * 
 +   * We use following steps to recover striped block group, in each round, we
 +   * recover <code>bufferSize</code> data until finish, the 
 +   * <code>bufferSize</code> is configurable and may be less or larger than 
 +   * cell size:
 +   * step1: read <code>bufferSize</code> data from minimum number of sources 
 +   *        required by recovery.
 +   * step2: decode data for targets.
 +   * step3: transfer data to targets.
 +   * 
 +   * In step1, try to read <code>bufferSize</code> data from minimum number
 +   * of sources , if there is corrupt or stale sources, read from new source
 +   * will be scheduled. The best sources are remembered for next round and 
 +   * may be updated in each round.
 +   * 
 +   * In step2, typically if source blocks we read are all data blocks, we 
 +   * need to call encode, and if there is one parity block, we need to call
 +   * decode. Notice we only read once and recover all missed striped block 
 +   * if they are more than one.
 +   * 
 +   * In step3, send the recovered data to targets by constructing packet 
 +   * and send them directly. Same as continuous block replication, we 
 +   * don't check the packet ack. Since the datanode doing the recovery work
 +   * are one of the source datanodes, so the recovered data are sent 
 +   * remotely.
 +   * 
 +   * There are some points we can do further improvements in next phase:
 +   * 1. we can read the block file directly on the local datanode, 
 +   *    currently we use remote block reader. (Notice short-circuit is not
 +   *    a good choice, see inline comments).
 +   * 2. We need to check the packet ack for EC recovery? Since EC recovery
 +   *    is more expensive than continuous block replication, it needs to 
 +   *    read from several other datanodes, should we make sure the 
 +   *    recovered result received by targets? 
 +   */
 +  private class ReconstructAndTransferBlock implements Runnable {
 +    private final int dataBlkNum;
 +    private final int parityBlkNum;
 +    private final int cellSize;
 +    
 +    private RawErasureDecoder decoder;
 +
 +    // Striped read buffer size
 +    private int bufferSize;
 +
 +    private final ExtendedBlock blockGroup;
 +    private final int minRequiredSources;
 +    // position in striped internal block
 +    private long positionInBlock;
 +
 +    // sources
 +    private final short[] liveIndices;
 +    private final DatanodeInfo[] sources;
 +
 +    private final List<StripedReader> stripedReaders;
 +
 +    // The buffers and indices for striped blocks whose length is 0
 +    private ByteBuffer[] zeroStripeBuffers;
 +    private short[] zeroStripeIndices;
 +
 +    // targets
 +    private final DatanodeInfo[] targets;
 +    private final StorageType[] targetStorageTypes;
 +
 +    private final short[] targetIndices;
 +    private final ByteBuffer[] targetBuffers;
 +
 +    private final Socket[] targetSockets;
 +    private final DataOutputStream[] targetOutputStreams;
 +    private final DataInputStream[] targetInputStreams;
 +
 +    private final long[] blockOffset4Targets;
 +    private final long[] seqNo4Targets;
 +
 +    private final static int WRITE_PACKET_SIZE = 64 * 1024;
 +    private DataChecksum checksum;
 +    private int maxChunksPerPacket;
 +    private byte[] packetBuf;
 +    private byte[] checksumBuf;
 +    private int bytesPerChecksum;
 +    private int checksumSize;
 +
 +    private final CachingStrategy cachingStrategy;
 +
 +    private final Map<Future<Void>, Integer> futures = new HashMap<>();
 +    private final CompletionService<Void> readService =
 +        new ExecutorCompletionService<>(STRIPED_READ_THREAD_POOL);
 +
 +    ReconstructAndTransferBlock(BlockECRecoveryInfo recoveryInfo) {
 +      ErasureCodingPolicy ecPolicy = recoveryInfo.getErasureCodingPolicy();
 +      dataBlkNum = ecPolicy.getNumDataUnits();
 +      parityBlkNum = ecPolicy.getNumParityUnits();
 +      cellSize = ecPolicy.getCellSize();
 +
 +      blockGroup = recoveryInfo.getExtendedBlock();
 +      final int cellsNum = (int)((blockGroup.getNumBytes() - 1) / cellSize + 1);
 +      minRequiredSources = Math.min(cellsNum, dataBlkNum);
 +
 +      liveIndices = recoveryInfo.getLiveBlockIndices();
 +      sources = recoveryInfo.getSourceDnInfos();
 +      stripedReaders = new ArrayList<>(sources.length);
 +
 +      Preconditions.checkArgument(liveIndices.length >= minRequiredSources,
 +          "No enough live striped blocks.");
 +      Preconditions.checkArgument(liveIndices.length == sources.length,
 +          "liveBlockIndices and source dns should match");
 +
 +      if (minRequiredSources < dataBlkNum) {
 +        zeroStripeBuffers = 
 +            new ByteBuffer[dataBlkNum - minRequiredSources];
 +        zeroStripeIndices = new short[dataBlkNum - minRequiredSources];
 +      }
 +
 +      targets = recoveryInfo.getTargetDnInfos();
 +      targetStorageTypes = recoveryInfo.getTargetStorageTypes();
 +      targetIndices = new short[targets.length];
 +      targetBuffers = new ByteBuffer[targets.length];
 +
 +      Preconditions.checkArgument(targetIndices.length <= parityBlkNum,
 +          "Too much missed striped blocks.");
 +
 +      targetSockets = new Socket[targets.length];
 +      targetOutputStreams = new DataOutputStream[targets.length];
 +      targetInputStreams = new DataInputStream[targets.length];
 +
 +      blockOffset4Targets = new long[targets.length];
 +      seqNo4Targets = new long[targets.length];
 +
 +      for (int i = 0; i < targets.length; i++) {
 +        blockOffset4Targets[i] = 0;
 +        seqNo4Targets[i] = 0;
 +      }
 +
 +      getTargetIndices();
 +      cachingStrategy = CachingStrategy.newDefaultStrategy();
 +    }
 +
 +    private ByteBuffer allocateBuffer(int length) {
 +      return ByteBuffer.allocate(length);
 +    }
 +
 +    private ExtendedBlock getBlock(ExtendedBlock blockGroup, int i) {
 +      return StripedBlockUtil.constructInternalBlock(blockGroup, cellSize,
 +          dataBlkNum, i);
 +    }
 +
 +    private long getBlockLen(ExtendedBlock blockGroup, int i) { 
 +      return StripedBlockUtil.getInternalBlockLength(blockGroup.getNumBytes(),
 +          cellSize, dataBlkNum, i);
 +    }
 +
 +    /**
 +     * StripedReader is used to read from one source DN, it contains a block
 +     * reader, buffer and striped block index.
 +     * Only allocate StripedReader once for one source, and the StripedReader
 +     * has the same array order with sources. Typically we only need to allocate
 +     * minimum number (minRequiredSources) of StripedReader, and allocate
 +     * new for new source DN if some existing DN invalid or slow.
 +     * If some source DN is corrupt, set the corresponding blockReader to 
 +     * null and will never read from it again.
 +     *  
 +     * @param i the array index of sources
 +     * @param offsetInBlock offset for the internal block
 +     * @return StripedReader
 +     */
 +    private StripedReader addStripedReader(int i, long offsetInBlock) {
 +      StripedReader reader = new StripedReader(liveIndices[i]);
 +      stripedReaders.add(reader);
 +
 +      BlockReader blockReader = newBlockReader(
 +          getBlock(blockGroup, liveIndices[i]), offsetInBlock, sources[i]);
 +      if (blockReader != null) {
 +        initChecksumAndBufferSizeIfNeeded(blockReader);
 +        reader.blockReader = blockReader;
 +      }
 +      reader.buffer = allocateBuffer(bufferSize);
 +      return reader;
 +    }
 +
 +    @Override
 +    public void run() {
 +      datanode.incrementXmitsInProgress();
 +      try {
 +        // Store the array indices of source DNs we have read successfully.
 +        // In each iteration of read, the success list may be updated if
 +        // some source DN is corrupted or slow. And use the updated success
 +        // list of DNs for next iteration read.
 +        int[] success = new int[minRequiredSources];
 +
 +        int nsuccess = 0;
 +        for (int i = 0; 
 +            i < sources.length && nsuccess < minRequiredSources; i++) {
 +          StripedReader reader = addStripedReader(i, 0);
 +          if (reader.blockReader != null) {
 +            success[nsuccess++] = i;
 +          }
 +        }
 +
 +        if (nsuccess < minRequiredSources) {
 +          String error = "Can't find minimum sources required by "
 +              + "recovery, block id: " + blockGroup.getBlockId();
 +          throw new IOException(error);
 +        }
 +
 +        if (zeroStripeBuffers != null) {
 +          for (int i = 0; i < zeroStripeBuffers.length; i++) {
 +            zeroStripeBuffers[i] = allocateBuffer(bufferSize);
 +          }
 +        }
 +
 +        for (int i = 0; i < targets.length; i++) {
 +          targetBuffers[i] = allocateBuffer(bufferSize);
 +        }
 +
 +        checksumSize = checksum.getChecksumSize();
 +        int chunkSize = bytesPerChecksum + checksumSize;
 +        maxChunksPerPacket = Math.max(
 +            (WRITE_PACKET_SIZE - PacketHeader.PKT_MAX_HEADER_LEN)/chunkSize, 1);
 +        int maxPacketSize = chunkSize * maxChunksPerPacket 
 +            + PacketHeader.PKT_MAX_HEADER_LEN;
 +
 +        packetBuf = new byte[maxPacketSize];
 +        checksumBuf = new byte[checksumSize * (bufferSize / bytesPerChecksum)];
 +
 +        // targetsStatus store whether some target is success, it will record
 +        // any failed target once, if some target failed (invalid DN or transfer
 +        // failed), will not transfer data to it any more.
 +        boolean[] targetsStatus = new boolean[targets.length];
 +        if (initTargetStreams(targetsStatus) == 0) {
 +          String error = "All targets are failed.";
 +          throw new IOException(error);
 +        }
 +
 +        long firstStripedBlockLength = getBlockLen(blockGroup, 0);
 +        while (positionInBlock < firstStripedBlockLength) {
 +          int toRead = Math.min(
 +              bufferSize, (int)(firstStripedBlockLength - positionInBlock));
 +          // step1: read from minimum source DNs required for reconstruction.
 +          //   The returned success list is the source DNs we do real read from
 +          success = readMinimumStripedData4Recovery(success);
 +
 +          // step2: decode to reconstruct targets
 +          long remaining = firstStripedBlockLength - positionInBlock;
 +          int toRecoverLen = remaining < bufferSize ? 
 +              (int)remaining : bufferSize;
 +          recoverTargets(success, targetsStatus, toRecoverLen);
 +
 +          // step3: transfer data
 +          if (transferData2Targets(targetsStatus) == 0) {
 +            String error = "Transfer failed for all targets.";
 +            throw new IOException(error);
 +          }
 +
 +          clearBuffers();
 +          positionInBlock += toRead;
 +        }
 +
 +        endTargetBlocks(targetsStatus);
 +
 +        // Currently we don't check the acks for packets, this is similar as
 +        // block replication.
 +      } catch (Throwable e) {
 +        LOG.warn("Failed to recover striped block: " + blockGroup, e);
 +      } finally {
 +        datanode.decrementXmitsInProgress();
 +        // close block readers
 +        for (StripedReader stripedReader : stripedReaders) {
 +          closeBlockReader(stripedReader.blockReader);
 +        }
 +        for (int i = 0; i < targets.length; i++) {
 +          IOUtils.closeStream(targetOutputStreams[i]);
 +          IOUtils.closeStream(targetInputStreams[i]);
 +          IOUtils.closeStream(targetSockets[i]);
 +        }
 +      }
 +    }
 +
 +    // init checksum from block reader
 +    private void initChecksumAndBufferSizeIfNeeded(BlockReader blockReader) {
 +      if (checksum == null) {
 +        checksum = blockReader.getDataChecksum();
 +        bytesPerChecksum = checksum.getBytesPerChecksum();
 +        // The bufferSize is flat to divide bytesPerChecksum
 +        int readBufferSize = STRIPED_READ_BUFFER_SIZE;
 +        bufferSize = readBufferSize < bytesPerChecksum ? bytesPerChecksum :
 +          readBufferSize - readBufferSize % bytesPerChecksum;
 +      } else {
 +        assert blockReader.getDataChecksum().equals(checksum);
 +      }
 +    }
 +
 +    private void getTargetIndices() {
 +      BitSet bitset = new BitSet(dataBlkNum + parityBlkNum);
 +      for (int i = 0; i < sources.length; i++) {
 +        bitset.set(liveIndices[i]);
 +      }
 +      int m = 0;
 +      int k = 0;
 +      for (int i = 0; i < dataBlkNum + parityBlkNum; i++) {
 +        if (!bitset.get(i)) {
 +          if (getBlockLen(blockGroup, i) > 0) {
 +            if (m < targets.length) {
 +              targetIndices[m++] = (short)i;
 +            }
 +          } else {
 +            zeroStripeIndices[k++] = (short)i;
 +          }
 +        }
 +      }
 +    }
 +
 +    private long getReadLength(int index) {
 +      long blockLen = getBlockLen(blockGroup, index);
 +      long remaining = blockLen - positionInBlock;
 +      return remaining > bufferSize ? bufferSize : remaining;
 +    }
 +
 +    /**
 +     * Read from minimum source DNs required for reconstruction in the iteration.
 +     * First try the success list which we think they are the best DNs
 +     * If source DN is corrupt or slow, try to read some other source DN, 
 +     * and will update the success list. 
 +     * 
 +     * Remember the updated success list and return it for following 
 +     * operations and next iteration read.
 +     * 
 +     * @param success the initial success list of source DNs we think best
 +     * @return updated success list of source DNs we do real read
 +     * @throws IOException
 +     */
 +    private int[] readMinimumStripedData4Recovery(final int[] success)
 +        throws IOException {
 +      int nsuccess = 0;
 +      int[] newSuccess = new int[minRequiredSources];
 +      BitSet used = new BitSet(sources.length);
 +      /*
 +       * Read from minimum source DNs required, the success list contains
 +       * source DNs which we think best.
 +       */
 +      for (int i = 0; i < minRequiredSources; i++) {
 +        StripedReader reader = stripedReaders.get(success[i]);
 +        if (getReadLength(liveIndices[success[i]]) > 0) {
 +          Callable<Void> readCallable = readFromBlock(
 +              reader.blockReader, reader.buffer);
 +          Future<Void> f = readService.submit(readCallable);
 +          futures.put(f, success[i]);
 +        } else {
 +          // If the read length is 0, we don't need to do real read
 +          reader.buffer.position(0);
 +          newSuccess[nsuccess++] = success[i];
 +        }
 +        used.set(success[i]);
 +      }
 +
 +      while (!futures.isEmpty()) {
 +        try {
 +          StripingChunkReadResult result =
 +              StripedBlockUtil.getNextCompletedStripedRead(
 +                  readService, futures, STRIPED_READ_TIMEOUT_MILLIS);
 +          int resultIndex = -1;
 +          if (result.state == StripingChunkReadResult.SUCCESSFUL) {
 +            resultIndex = result.index;
 +          } else if (result.state == StripingChunkReadResult.FAILED) {
 +            // If read failed for some source DN, we should not use it anymore 
 +            // and schedule read from another source DN.
 +            StripedReader failedReader = stripedReaders.get(result.index);
 +            closeBlockReader(failedReader.blockReader);
 +            failedReader.blockReader = null;
 +            resultIndex = scheduleNewRead(used);
 +          } else if (result.state == StripingChunkReadResult.TIMEOUT) {
 +            // If timeout, we also schedule a new read.
 +            resultIndex = scheduleNewRead(used);
 +          }
 +          if (resultIndex >= 0) {
 +            newSuccess[nsuccess++] = resultIndex;
 +            if (nsuccess >= minRequiredSources) {
 +              // cancel remaining reads if we read successfully from minimum
 +              // number of source DNs required by reconstruction.
 +              cancelReads(futures.keySet());
 +              futures.clear();
 +              break;
 +            }
 +          }
 +        } catch (InterruptedException e) {
 +          LOG.info("Read data interrupted.", e);
 +          break;
 +        }
 +      }
 +
 +      if (nsuccess < minRequiredSources) {
 +        String error = "Can't read data from minimum number of sources "
 +            + "required by reconstruction, block id: " + blockGroup.getBlockId();
 +        throw new IOException(error);
 +      }
 +
 +      return newSuccess;
 +    }
 +    
 +    private void paddingBufferToLen(ByteBuffer buffer, int len) {
 +      int toPadding = len - buffer.position();
 +      for (int i = 0; i < toPadding; i++) {
 +        buffer.put((byte) 0);
 +      }
 +    }
 +    
 +    // Initialize decoder
 +    private void initDecoderIfNecessary() {
 +      if (decoder == null) {
 +        decoder = newDecoder(dataBlkNum, parityBlkNum);
 +      }
 +    }
 +
 +    private int[] getErasedIndices(boolean[] targetsStatus) {
 +      int[] result = new int[targets.length];
 +      int m = 0;
 +      for (int i = 0; i < targets.length; i++) {
 +        if (targetsStatus[i]) {
 +          result[m++] = convertIndex4Decode(targetIndices[i], 
 +              dataBlkNum, parityBlkNum);
 +        }
 +      }
 +      return Arrays.copyOf(result, m);
 +    }
 +
 +    private void recoverTargets(int[] success, boolean[] targetsStatus,
 +        int toRecoverLen) {
 +      initDecoderIfNecessary();
 +      ByteBuffer[] inputs = new ByteBuffer[dataBlkNum + parityBlkNum];
 +      for (int i = 0; i < success.length; i++) {
 +        StripedReader reader = stripedReaders.get(success[i]);
 +        ByteBuffer buffer = reader.buffer;
 +        paddingBufferToLen(buffer, toRecoverLen);
 +        inputs[convertIndex4Decode(reader.index, dataBlkNum, parityBlkNum)] = 
 +            (ByteBuffer)buffer.flip();
 +      }
 +      if (success.length < dataBlkNum) {
 +        for (int i = 0; i < zeroStripeBuffers.length; i++) {
 +          ByteBuffer buffer = zeroStripeBuffers[i];
 +          paddingBufferToLen(buffer, toRecoverLen);
 +          int index = convertIndex4Decode(zeroStripeIndices[i], dataBlkNum,
 +              parityBlkNum);
 +          inputs[index] = (ByteBuffer)buffer.flip();
 +        }
 +      }
 +      int[] erasedIndices = getErasedIndices(targetsStatus);
 +      ByteBuffer[] outputs = new ByteBuffer[erasedIndices.length];
 +      int m = 0;
 +      for (int i = 0; i < targetBuffers.length; i++) {
 +        if (targetsStatus[i]) {
 +          outputs[m++] = targetBuffers[i];
 +          outputs[i].limit(toRecoverLen);
 +        }
 +      }
 +      decoder.decode(inputs, erasedIndices, outputs);
 +
 +      for (int i = 0; i < targets.length; i++) {
 +        if (targetsStatus[i]) {
 +          long blockLen = getBlockLen(blockGroup, targetIndices[i]);
 +          long remaining = blockLen - positionInBlock;
 +          if (remaining < 0) {
 +            targetBuffers[i].limit(0);
 +          } else if (remaining < toRecoverLen) {
 +            targetBuffers[i].limit((int)remaining);
 +          }
 +        }
 +      }
 +    }
 +
 +    /**
 +     * Schedule a read from some new source DN if some DN is corrupted
 +     * or slow, this is called from the read iteration.
 +     * Initially we may only have <code>minRequiredSources</code> number of 
 +     * StripedReader.
 +     * If the position is at the end of target block, don't need to do 
 +     * real read, and return the array index of source DN, otherwise -1.
 +     * 
 +     * @param used the used source DNs in this iteration.
 +     * @return the array index of source DN if don't need to do real read.
 +     */
 +    private int scheduleNewRead(BitSet used) {
 +      StripedReader reader = null;
 +      // step1: initially we may only have <code>minRequiredSources</code>
 +      // number of StripedReader, and there may be some source DNs we never 
 +      // read before, so will try to create StripedReader for one new source DN
 +      // and try to read from it. If found, go to step 3.
 +      int m = stripedReaders.size();
 +      while (reader == null && m < sources.length) {
 +        reader = addStripedReader(m, positionInBlock);
 +        if (getReadLength(liveIndices[m]) > 0) {
 +          if (reader.blockReader == null) {
 +            reader = null;
 +            m++;
 +          }
 +        } else {
 +          used.set(m);
 +          return m;
 +        }
 +      }
 +
 +      // step2: if there is no new source DN we can use, try to find a source 
 +      // DN we ever read from but because some reason, e.g., slow, it
 +      // is not in the success DN list at the begin of this iteration, so 
 +      // we have not tried it in this iteration. Now we have a chance to 
 +      // revisit it again.
 +      for (int i = 0; reader == null && i < stripedReaders.size(); i++) {
 +        if (!used.get(i)) {
 +          StripedReader r = stripedReaders.get(i);
 +          if (getReadLength(liveIndices[i]) > 0) {
 +            closeBlockReader(r.blockReader);
 +            r.blockReader = newBlockReader(
 +                getBlock(blockGroup, liveIndices[i]), positionInBlock,
 +                sources[i]);
 +            if (r.blockReader != null) {
 +              m = i;
 +              reader = r;
 +            }
 +          } else {
 +            used.set(i);
 +            r.buffer.position(0);
 +            return i;
 +          }
 +        }
 +      }
 +
 +      // step3: schedule if find a correct source DN and need to do real read.
 +      if (reader != null) {
 +        Callable<Void> readCallable = readFromBlock(
 +            reader.blockReader, reader.buffer);
 +        Future<Void> f = readService.submit(readCallable);
 +        futures.put(f, m);
 +        used.set(m);
 +      }
 +
 +      return -1;
 +    }
 +
 +    // cancel all reads.
 +    private void cancelReads(Collection<Future<Void>> futures) {
 +      for (Future<Void> future : futures) {
 +        future.cancel(true);
 +      }
 +    }
 +
 +    private Callable<Void> readFromBlock(final BlockReader reader,
 +        final ByteBuffer buf) {
 +      return new Callable<Void>() {
 +
 +        @Override
 +        public Void call() throws Exception {
 +          try {
 +            actualReadFromBlock(reader, buf);
 +            return null;
 +          } catch (IOException e) {
 +            LOG.info(e.getMessage());
 +            throw e;
 +          }
 +        }
 +
 +      };
 +    }
 +
 +    /**
 +     * Read bytes from block
 +     */
 +    private void actualReadFromBlock(BlockReader reader, ByteBuffer buf)
 +        throws IOException {
 +      int len = buf.remaining();
 +      int n = 0;
 +      while (n < len) {
 +        int nread = reader.read(buf);
 +        if (nread <= 0) {
 +          break;
 +        }
 +        n += nread;
 +      }
 +    }
 +
 +    // close block reader
 +    private void closeBlockReader(BlockReader blockReader) {
 +      try {
 +        if (blockReader != null) {
 +          blockReader.close();
 +        }
 +      } catch (IOException e) {
 +        // ignore
 +      }
 +    }
 +
 +    private InetSocketAddress getSocketAddress4Transfer(DatanodeInfo dnInfo) {
 +      return NetUtils.createSocketAddr(dnInfo.getXferAddr(
 +          datanode.getDnConf().getConnectToDnViaHostname()));
 +    }
 +
 +    private BlockReader newBlockReader(final ExtendedBlock block, 
 +        long offsetInBlock, DatanodeInfo dnInfo) {
 +      if (offsetInBlock >= block.getNumBytes()) {
 +        return null;
 +      }
 +      try {
 +        InetSocketAddress dnAddr = getSocketAddress4Transfer(dnInfo);
 +        Token<BlockTokenIdentifier> blockToken = datanode.getBlockAccessToken(
 +            block, EnumSet.of(BlockTokenIdentifier.AccessMode.READ));
 +        /*
 +         * This can be further improved if the replica is local, then we can
 +         * read directly from DN and need to check the replica is FINALIZED
 +         * state, notice we should not use short-circuit local read which
 +         * requires config for domain-socket in UNIX or legacy config in Windows.
 +         */
 +        return RemoteBlockReader2.newBlockReader(
 +            "dummy", block, blockToken, offsetInBlock, 
 +            block.getNumBytes() - offsetInBlock, true,
 +            "", newConnectedPeer(block, dnAddr, blockToken, dnInfo), dnInfo,
 +            null, cachingStrategy);
 +      } catch (IOException e) {
 +        return null;
 +      }
 +    }
 +
 +    private Peer newConnectedPeer(ExtendedBlock b, InetSocketAddress addr,
 +        Token<BlockTokenIdentifier> blockToken, DatanodeID datanodeId)
 +        throws IOException {
 +      Peer peer = null;
 +      boolean success = false;
 +      Socket sock = null;
 +      final int socketTimeout = datanode.getDnConf().getSocketTimeout(); 
 +      try {
 +        sock = NetUtils.getDefaultSocketFactory(conf).createSocket();
 +        NetUtils.connect(sock, addr, socketTimeout);
-         peer = TcpPeerServer.peerFromSocketAndKey(datanode.getSaslClient(), 
++        peer = DFSUtilClient.peerFromSocketAndKey(datanode.getSaslClient(),
 +            sock, datanode.getDataEncryptionKeyFactoryForBlock(b),
 +            blockToken, datanodeId);
 +        peer.setReadTimeout(socketTimeout);
 +        success = true;
 +        return peer;
 +      } finally {
 +        if (!success) {
 +          IOUtils.cleanup(LOG, peer);
 +          IOUtils.closeSocket(sock);
 +        }
 +      }
 +    }
 +
 +    /**
 +     * Send data to targets
 +     */
 +    private int transferData2Targets(boolean[] targetsStatus) {
 +      int nsuccess = 0;
 +      for (int i = 0; i < targets.length; i++) {
 +        if (targetsStatus[i]) {
 +          boolean success = false;
 +          try {
 +            ByteBuffer buffer = targetBuffers[i];
 +            
 +            if (buffer.remaining() == 0) {
 +              continue;
 +            }
 +
 +            checksum.calculateChunkedSums(
 +                buffer.array(), 0, buffer.remaining(), checksumBuf, 0);
 +
 +            int ckOff = 0;
 +            while (buffer.remaining() > 0) {
 +              DFSPacket packet = new DFSPacket(packetBuf, maxChunksPerPacket,
 +                  blockOffset4Targets[i], seqNo4Targets[i]++, checksumSize, false);
 +              int maxBytesToPacket = maxChunksPerPacket * bytesPerChecksum;
 +              int toWrite = buffer.remaining() > maxBytesToPacket ?
 +                  maxBytesToPacket : buffer.remaining();
 +              int ckLen = ((toWrite - 1) / bytesPerChecksum + 1) * checksumSize;
 +              packet.writeChecksum(checksumBuf, ckOff, ckLen);
 +              ckOff += ckLen;
 +              packet.writeData(buffer, toWrite);
 +
 +              // Send packet
 +              packet.writeTo(targetOutputStreams[i]);
 +
 +              blockOffset4Targets[i] += toWrite;
 +              nsuccess++;
 +              success = true;
 +            }
 +          } catch (IOException e) {
 +            LOG.warn(e.getMessage());
 +          }
 +          targetsStatus[i] = success;
 +        }
 +      }
 +      return nsuccess;
 +    }
 +
 +    /**
 +     * clear all buffers
 +     */
 +    private void clearBuffers() {
 +      for (StripedReader stripedReader : stripedReaders) {
 +        if (stripedReader.buffer != null) {
 +          stripedReader.buffer.clear();
 +        }
 +      }
 +
 +      if (zeroStripeBuffers != null) {
 +        for (int i = 0; i < zeroStripeBuffers.length; i++) {
 +          zeroStripeBuffers[i].clear();
 +        }
 +      }
 +
 +      for (int i = 0; i < targetBuffers.length; i++) {
 +        if (targetBuffers[i] != null) {
 +          cleanBuffer(targetBuffers[i]);
 +        }
 +      }
 +    }
 +    
 +    private ByteBuffer cleanBuffer(ByteBuffer buffer) {
 +      Arrays.fill(buffer.array(), (byte) 0);
 +      return (ByteBuffer)buffer.clear();
 +    }
 +
 +    // send an empty packet to mark the end of the block
 +    private void endTargetBlocks(boolean[] targetsStatus) {
 +      for (int i = 0; i < targets.length; i++) {
 +        if (targetsStatus[i]) {
 +          try {
 +            DFSPacket packet = new DFSPacket(packetBuf, 0, 
 +                blockOffset4Targets[i], seqNo4Targets[i]++, checksumSize, true);
 +            packet.writeTo(targetOutputStreams[i]);
 +            targetOutputStreams[i].flush();
 +          } catch (IOException e) {
 +            LOG.warn(e.getMessage());
 +          }
 +        }
 +      }
 +    }
 +
 +    /**
 +     * Initialize  output/input streams for transferring data to target
 +     * and send create block request. 
 +     */
 +    private int initTargetStreams(boolean[] targetsStatus) {
 +      int nsuccess = 0;
 +      for (int i = 0; i < targets.length; i++) {
 +        Socket socket = null;
 +        DataOutputStream out = null;
 +        DataInputStream in = null;
 +        boolean success = false;
 +        try {
 +          InetSocketAddress targetAddr = 
 +              getSocketAddress4Transfer(targets[i]);
 +          socket = datanode.newSocket();
 +          NetUtils.connect(socket, targetAddr, 
 +              datanode.getDnConf().getSocketTimeout());
 +          socket.setSoTimeout(datanode.getDnConf().getSocketTimeout());
 +
 +          ExtendedBlock block = getBlock(blockGroup, targetIndices[i]);
 +          Token<BlockTokenIdentifier> blockToken = 
 +              datanode.getBlockAccessToken(block,
 +                  EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE));
 +
 +          long writeTimeout = datanode.getDnConf().getSocketWriteTimeout();
 +          OutputStream unbufOut = NetUtils.getOutputStream(socket, writeTimeout);
 +          InputStream unbufIn = NetUtils.getInputStream(socket);
 +          DataEncryptionKeyFactory keyFactory =
 +            datanode.getDataEncryptionKeyFactoryForBlock(block);
 +          IOStreamPair saslStreams = datanode.getSaslClient().socketSend(
 +              socket, unbufOut, unbufIn, keyFactory, blockToken, targets[i]);
 +
 +          unbufOut = saslStreams.out;
 +          unbufIn = saslStreams.in;
 +
 +          out = new DataOutputStream(new BufferedOutputStream(unbufOut,
 +              DFSUtil.getSmallBufferSize(conf)));
 +          in = new DataInputStream(unbufIn);
 +
 +          DatanodeInfo source = new DatanodeInfo(datanode.getDatanodeId());
 +          new Sender(out).writeBlock(block, targetStorageTypes[i], 
 +              blockToken, "", new DatanodeInfo[]{targets[i]}, 
 +              new StorageType[]{targetStorageTypes[i]}, source, 
 +              BlockConstructionStage.PIPELINE_SETUP_CREATE, 0, 0, 0, 0, 
 +              checksum, cachingStrategy, false, false, null);
 +
 +          targetSockets[i] = socket;
 +          targetOutputStreams[i] = out;
 +          targetInputStreams[i] = in;
 +          nsuccess++;
 +          success = true;
 +        } catch (Throwable e) {
 +          LOG.warn(e.getMessage());
 +        } finally {
 +          if (!success) {
 +            IOUtils.closeStream(out);
 +            IOUtils.closeStream(in);
 +            IOUtils.closeStream(socket);
 +          }
 +        }
 +        targetsStatus[i] = success;
 +      }
 +      return nsuccess;
 +    }
 +  }
 +
 +  private static class StripedReader {
 +    private final short index; // internal block index
 +    private BlockReader blockReader;
 +    private ByteBuffer buffer;
 +
 +    private StripedReader(short index) {
 +      this.index = index;
 +    }
 +  }
 +}

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
index a115138,0ae739c..34b28e4
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
@@@ -39,10 -39,8 +39,9 @@@ import org.apache.hadoop.fs.permission.
  import org.apache.hadoop.fs.StorageType;
  import org.apache.hadoop.fs.XAttr;
  import org.apache.hadoop.hdfs.protocol.Block;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
  import org.apache.hadoop.hdfs.protocol.HdfsConstants;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
- import org.apache.hadoop.hdfs.protocolPB.PBHelper;
  import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
@@@ -334,21 -331,13 +333,21 @@@ public final class FSImageFormatPBINod
        INodeSection.INodeFile f = n.getFile();
        List<BlockProto> bp = f.getBlocksList();
        short replication = (short) f.getReplication();
 +      boolean isStriped = f.getIsStriped();
        LoaderContext state = parent.getLoaderContext();
 +      ErasureCodingPolicy ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
  
        BlockInfo[] blocks = new BlockInfo[bp.size()];
 -      for (int i = 0, e = bp.size(); i < e; ++i) {
 -        blocks[i] =
 -            new BlockInfoContiguous(PBHelperClient.convert(bp.get(i)), replication);
 +      for (int i = 0; i < bp.size(); ++i) {
 +        BlockProto b = bp.get(i);
 +        if (isStriped) {
-           blocks[i] = new BlockInfoStriped(PBHelper.convert(b), ecPolicy);
++          blocks[i] = new BlockInfoStriped(PBHelperClient.convert(b), ecPolicy);
 +        } else {
-           blocks[i] = new BlockInfoContiguous(PBHelper.convert(b),
++          blocks[i] = new BlockInfoContiguous(PBHelperClient.convert(b),
 +              replication);
 +        }
        }
 +
        final PermissionStatus permissions = loadPermission(f.getPermission(),
            parent.getLoaderContext().getStringTable());
  
@@@ -654,11 -632,10 +653,11 @@@
      private void save(OutputStream out, INodeFile n) throws IOException {
        INodeSection.INodeFile.Builder b = buildINodeFile(n,
            parent.getSaverContext());
 +      BlockInfo[] blocks = n.getBlocks();
  
 -      if (n.getBlocks() != null) {
 +      if (blocks != null) {
          for (Block block : n.getBlocks()) {
-           b.addBlocks(PBHelper.convert(block));
+           b.addBlocks(PBHelperClient.convert(block));
          }
        }
  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index ed52ca4,75b6be9..b6b151c
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@@ -4714,26 -4654,8 +4713,8 @@@ public class FSNamesystem implements Na
        && safeMode.isOn();
    }
  
-   /**
-    * Check if replication queues are to be populated
-    * @return true when node is HAState.Active and not in the very first safemode
-    */
-   @Override
-   public boolean isPopulatingReplQueues() {
-     if (!shouldPopulateReplQueues()) {
-       return false;
-     }
-     return initializedReplQueues;
-   }
- 
-   private boolean shouldPopulateReplQueues() {
-     if(haContext == null || haContext.getState() == null)
-       return false;
-     return haContext.getState().shouldPopulateReplQueues();
-   }
- 
    @Override
 -  public void incrementSafeBlockCount(int replication) {
 +  public void incrementSafeBlockCount(int storageNum, BlockInfo storedBlock) {
      // safeMode is volatile, and may be set to null at any time
      SafeModeInfo safeMode = this.safeMode;
      if (safeMode == null)
@@@ -6233,11 -6150,11 +6222,16 @@@
      return cacheManager;
    }
  
 +  /** @return the ErasureCodingPolicyManager. */
 +  public ErasureCodingPolicyManager getErasureCodingPolicyManager() {
 +    return ecPolicyManager;
 +  }
 +
+   @Override
+   public HAContext getHAContext() {
+     return haContext;
+   }
+ 
    @Override  // NameNodeMXBean
    public String getCorruptFiles() {
      List<String> list = new ArrayList<String>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 6f7b702,8565522..c765edc
--- 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
@@@ -664,37 -608,12 +664,24 @@@ public class INodeFile extends INodeWit
      return counts;
    }
  
 +  /**
 +   * Compute quota of striped file. Note that currently EC files do not support
 +   * append/hflush/hsync, thus the file length recorded in snapshots should be
 +   * the same with the current file length.
 +   */
 +  public final QuotaCounts computeQuotaUsageWithStriped(
 +      BlockStoragePolicy bsp, QuotaCounts counts) {
 +    counts.addNameSpace(1);
 +    counts.add(storagespaceConsumed(bsp));
 +    return counts;
 +  }
 +
    @Override
    public final ContentSummaryComputationContext computeContentSummary(
-       final ContentSummaryComputationContext summary) {
+       int snapshotId, final ContentSummaryComputationContext summary) {
      final ContentCounts counts = summary.getCounts();
-     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
-     final long fileLen;
-     if (sf == null) {
-       fileLen = computeFileSize();
-       counts.addContent(Content.FILE, 1);
-     } else {
-       final FileDiffList diffs = sf.getDiffs();
-       final int n = diffs.asList().size();
-       counts.addContent(Content.FILE, n);
-       if (n > 0 && sf.isCurrentFileDeleted()) {
-         fileLen =  diffs.getLast().getFileSize();
-       } else {
-         fileLen = computeFileSize();
-       }
-     }
+     counts.addContent(Content.FILE, 1);
+     final long fileLen = computeFileSize(snapshotId);
      counts.addContent(Content.LENGTH, fileLen);
      counts.addContent(Content.DISKSPACE, storagespaceConsumed(null)
          .getStorageSpace());

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
index e1702d9,5bc4033..b1012c2
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@@ -64,4 -52,5 +65,6 @@@ public interface Namesystem extends RwL
    boolean isInSnapshot(BlockInfo blockUC);
  
    CacheManager getCacheManager();
++
+   HAContext getHAContext();
 -}
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
index 252844c,06a8219..98deed2
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
@@@ -39,15 -39,11 +39,12 @@@ public interface SafeMode 
     */
    public boolean isInStartupSafeMode();
  
-   /** Check whether replication queues are being populated. */
-   public boolean isPopulatingReplQueues();
-     
    /**
     * Increment number of blocks that reached minimal replication.
 -   * @param replication current replication 
 +   * @param replication current replication
 +   * @param storedBlock current stored Block
     */
 -  public void incrementSafeBlockCount(int replication);
 +  public void incrementSafeBlockCount(int replication, BlockInfo storedBlock);
  
    /** Decrement number of blocks that reached minimal replication. */
    public void decrementSafeBlockCount(BlockInfo b);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
index 450d981,cf21411..ae23783
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
@@@ -243,15 -242,13 +243,15 @@@ public class FSImageFormatPBSnapshot 
          FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,
              pbf.getFileSize());
          List<BlockProto> bpl = pbf.getBlocksList();
 +        // in file diff there can only be contiguous blocks
          BlockInfo[] blocks = new BlockInfo[bpl.size()];
          for(int j = 0, e = bpl.size(); j < e; ++j) {
-           Block blk = PBHelper.convert(bpl.get(j));
+           Block blk = PBHelperClient.convert(bpl.get(j));
            BlockInfo storedBlock = bm.getStoredBlock(blk);
            if(storedBlock == null) {
 -            storedBlock = bm.addBlockCollection(
 -                new BlockInfoContiguous(blk, copy.getFileReplication()), file);
 +            storedBlock = (BlockInfoContiguous) fsn.getBlockManager()
 +                .addBlockCollectionWithCheck(new BlockInfoContiguous(blk,
 +                    copy.getFileReplication()), file);
            }
            blocks[j] = storedBlock;
          }

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java
index 6c06a8d,0000000..0499a2b
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java
@@@ -1,38 -1,0 +1,39 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + * <p/>
 + * http://www.apache.org/licenses/LICENSE-2.0
 + * <p/>
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.cli;
 +
 +import org.apache.hadoop.cli.util.CLICommandErasureCodingCli;
 +import org.apache.hadoop.cli.util.CLICommandTypes;
 +import org.apache.hadoop.cli.util.CLITestCmd;
 +import org.apache.hadoop.cli.util.CommandExecutor;
 +import org.apache.hadoop.cli.util.ErasureCodingCliCmdExecutor;
++import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.hdfs.tools.erasurecode.ECCli;
 +
 +public class CLITestCmdErasureCoding extends CLITestCmd {
 +  public CLITestCmdErasureCoding(String str, CLICommandTypes type) {
 +    super(str, type);
 +  }
 +
 +  @Override
-   public CommandExecutor getExecutor(String tag) throws IllegalArgumentException {
++  public CommandExecutor getExecutor(String tag, Configuration conf) throws IllegalArgumentException {
 +    if (getType() instanceof CLICommandErasureCodingCli)
 +      return new ErasureCodingCliCmdExecutor(tag, new ECCli());
-     return super.getExecutor(tag);
++    return super.getExecutor(tag, conf);
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java
index dfefb66,0000000..29ec98e
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java
@@@ -1,115 -1,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.cli;
 +
 +import org.apache.hadoop.cli.util.CLICommand;
 +import org.apache.hadoop.cli.util.CLICommandErasureCodingCli;
 +import org.apache.hadoop.cli.util.CommandExecutor.Result;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.hdfs.DFSConfigKeys;
 +import org.apache.hadoop.hdfs.MiniDFSCluster;
 +import org.junit.After;
 +import org.junit.Before;
 +import org.junit.Test;
 +import org.xml.sax.SAXException;
 +
 +public class TestErasureCodingCLI extends CLITestHelper {
 +  private final int NUM_OF_DATANODES = 3;
 +  private MiniDFSCluster dfsCluster = null;
 +  private FileSystem fs = null;
 +  private String namenode = null;
 +
 +  @Before
 +  @Override
 +  public void setUp() throws Exception {
 +    super.setUp();
 +
 +    dfsCluster = new MiniDFSCluster.Builder(conf)
 +        .numDataNodes(NUM_OF_DATANODES).build();
 +    dfsCluster.waitClusterUp();
 +    namenode = conf.get(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "file:///");
 +
 +    username = System.getProperty("user.name");
 +
 +    fs = dfsCluster.getFileSystem();
 +  }
 +
 +  @Override
 +  protected String getTestFile() {
 +    return "testErasureCodingConf.xml";
 +  }
 +
 +  @After
 +  @Override
 +  public void tearDown() throws Exception {
 +    if (fs != null) {
 +      fs.close();
 +    }
 +    if (dfsCluster != null) {
 +      dfsCluster.shutdown();
 +    }
 +    Thread.sleep(2000);
 +    super.tearDown();
 +  }
 +
 +  @Override
 +  protected String expandCommand(final String cmd) {
 +    String expCmd = cmd;
 +    expCmd = expCmd.replaceAll("NAMENODE", namenode);
 +    expCmd = expCmd.replaceAll("#LF#", System.getProperty("line.separator"));
 +    expCmd = super.expandCommand(expCmd);
 +    return expCmd;
 +  }
 +
 +  @Override
 +  protected TestConfigFileParser getConfigParser() {
 +    return new TestErasureCodingAdmin();
 +  }
 +
 +  private class TestErasureCodingAdmin extends
 +      CLITestHelper.TestConfigFileParser {
 +    @Override
 +    public void endElement(String uri, String localName, String qName)
 +        throws SAXException {
 +      if (qName.equals("ec-admin-command")) {
 +        if (testCommands != null) {
 +          testCommands.add(new CLITestCmdErasureCoding(charString,
 +              new CLICommandErasureCodingCli()));
 +        } else if (cleanupCommands != null) {
 +          cleanupCommands.add(new CLITestCmdErasureCoding(charString,
 +              new CLICommandErasureCodingCli()));
 +        }
 +      } else {
 +        super.endElement(uri, localName, qName);
 +      }
 +    }
 +  }
 +
 +  @Override
 +  protected Result execute(CLICommand cmd) throws Exception {
-     return cmd.getExecutor(namenode).executeCommand(cmd.getCmd());
++    return cmd.getExecutor(namenode, conf).executeCommand(cmd.getCmd());
 +  }
 +
 +  @Test
 +  @Override
 +  public void testAll() {
 +    super.testAll();
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 3551055,a7e80ca..12d4811
--- 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
@@@ -66,14 -66,9 +66,15 @@@ import java.util.Set
  import java.util.UUID;
  import java.util.concurrent.TimeoutException;
  import java.util.concurrent.atomic.AtomicBoolean;
 +import com.google.common.base.Charsets;
 +import com.google.common.base.Joiner;
 +import com.google.common.base.Preconditions;
 +import com.google.common.base.Supplier;
 +import com.google.common.collect.Lists;
 +import com.google.common.collect.Maps;
  
  import org.apache.commons.io.FileUtils;
+ import org.apache.commons.lang.UnhandledException;
  import org.apache.commons.logging.Log;
  import org.apache.commons.logging.LogFactory;
  import org.apache.hadoop.conf.Configuration;
@@@ -141,10 -133,8 +142,11 @@@ import org.apache.hadoop.hdfs.server.na
  import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
  import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
  import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 +import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 +import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
 +import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
  import org.apache.hadoop.hdfs.tools.DFSAdmin;
+ import org.apache.hadoop.hdfs.tools.JMXGet;
  import org.apache.hadoop.io.IOUtils;
  import org.apache.hadoop.io.nativeio.NativeIO;
  import org.apache.hadoop.net.NetUtils;
@@@ -1870,150 -1858,21 +1872,168 @@@ public class DFSTestUtil 
      }
    }
  
 +  public static StorageReceivedDeletedBlocks[] makeReportForReceivedBlock(
 +      Block block, BlockStatus blockStatus, DatanodeStorage storage) {
 +    ReceivedDeletedBlockInfo[] receivedBlocks = new ReceivedDeletedBlockInfo[1];
 +    receivedBlocks[0] = new ReceivedDeletedBlockInfo(block, blockStatus, null);
 +    StorageReceivedDeletedBlocks[] reports = new StorageReceivedDeletedBlocks[1];
 +    reports[0] = new StorageReceivedDeletedBlocks(storage, receivedBlocks);
 +    return reports;
 +  }
 +
 +  /**
 +   * Creates the metadata of a file in striped layout. This method only
 +   * manipulates the NameNode state without injecting data to DataNode.
 +   * You should disable periodical heartbeat before use this.
 +   *  @param file Path of the file to create
 +   * @param dir Parent path of the file
 +   * @param numBlocks Number of striped block groups to add to the file
 +   * @param numStripesPerBlk Number of striped cells in each block
 +   * @param toMkdir
 +   */
 +  public static void createStripedFile(MiniDFSCluster cluster, Path file, Path dir,
 +      int numBlocks, int numStripesPerBlk, boolean toMkdir) throws Exception {
 +    DistributedFileSystem dfs = cluster.getFileSystem();
 +    // If outer test already set EC policy, dir should be left as null
 +    if (toMkdir) {
 +      assert dir != null;
 +      dfs.mkdirs(dir);
 +      try {
 +        dfs.getClient().setErasureCodingPolicy(dir.toString(), null);
 +      } catch (IOException e) {
 +        if (!e.getMessage().contains("non-empty directory")) {
 +          throw e;
 +        }
 +      }
 +    }
 +
 +    FSDataOutputStream out = null;
 +    try {
 +      out = dfs.create(file, (short) 1); // create an empty file
 +
 +      FSNamesystem ns = cluster.getNamesystem();
 +      FSDirectory fsdir = ns.getFSDirectory();
 +      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
 +
 +      ExtendedBlock previous = null;
 +      for (int i = 0; i < numBlocks; i++) {
 +        Block newBlock = addStripedBlockToFile(cluster.getDataNodes(), dfs, ns,
 +            file.toString(), fileNode, dfs.getClient().getClientName(),
 +            previous, numStripesPerBlk);
 +        previous = new ExtendedBlock(ns.getBlockPoolId(), newBlock);
 +      }
 +
 +      dfs.getClient().namenode.complete(file.toString(),
 +          dfs.getClient().getClientName(), previous, fileNode.getId());
 +    } finally {
 +      IOUtils.cleanup(null, out);
 +    }
 +  }
 +
 +  /**
 +   * Adds a striped block group to a file. This method only manipulates NameNode
 +   * states of the file and the block without injecting data to DataNode.
 +   * It does mimic block reports.
 +   * You should disable periodical heartbeat before use this.
 +   * @param dataNodes List DataNodes to host the striped block group
 +   * @param previous Previous block in the file
 +   * @param numStripes Number of stripes in each block group
 +   * @return The added block group
 +   */
 +  public static Block addStripedBlockToFile(List<DataNode> dataNodes,
 +      DistributedFileSystem fs, FSNamesystem ns, String file, INodeFile fileNode,
 +      String clientName, ExtendedBlock previous, int numStripes)
 +      throws Exception {
 +    fs.getClient().namenode.addBlock(file, clientName, previous, null,
 +        fileNode.getId(), null);
 +
 +    final BlockInfo lastBlock = fileNode.getLastBlock();
 +    final int groupSize = fileNode.getPreferredBlockReplication();
 +    assert dataNodes.size() >= groupSize;
 +    // 1. RECEIVING_BLOCK IBR
 +    for (int i = 0; i < groupSize; i++) {
 +      DataNode dn = dataNodes.get(i);
 +      final Block block = new Block(lastBlock.getBlockId() + i, 0,
 +          lastBlock.getGenerationStamp());
 +      DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
 +      StorageReceivedDeletedBlocks[] reports = DFSTestUtil
 +          .makeReportForReceivedBlock(block,
 +              ReceivedDeletedBlockInfo.BlockStatus.RECEIVING_BLOCK, storage);
 +      for (StorageReceivedDeletedBlocks report : reports) {
 +        ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
 +      }
 +    }
 +
 +    // 2. RECEIVED_BLOCK IBR
 +    for (int i = 0; i < groupSize; i++) {
 +      DataNode dn = dataNodes.get(i);
 +      final Block block = new Block(lastBlock.getBlockId() + i,
 +          numStripes * BLOCK_STRIPED_CELL_SIZE, lastBlock.getGenerationStamp());
 +      DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
 +      StorageReceivedDeletedBlocks[] reports = DFSTestUtil
 +          .makeReportForReceivedBlock(block,
 +              ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
 +      for (StorageReceivedDeletedBlocks report : reports) {
 +        ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
 +      }
 +    }
 +
 +    lastBlock.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS);
 +    return lastBlock;
 +  }
 +
 +  /**
 +   * Because currently DFSStripedOutputStream does not support hflush/hsync,
 +   * tests can use this method to flush all the buffered data to DataNodes.
 +   */
 +  public static ExtendedBlock flushInternal(DFSStripedOutputStream out)
 +      throws IOException {
 +    out.flushInternal();
 +    return out.getBlock();
 +  }
 +
 +  /**
 +   * Verify that blocks in striped block group are on different nodes, and every
 +   * internal blocks exists.
 +   */
 +  public static void verifyLocatedStripedBlocks(LocatedBlocks lbs,
 +       int groupSize) {
 +    for (LocatedBlock lb : lbs.getLocatedBlocks()) {
 +      assert lb instanceof LocatedStripedBlock;
 +      HashSet<DatanodeInfo> locs = new HashSet<>();
 +      for (DatanodeInfo datanodeInfo : lb.getLocations()) {
 +        locs.add(datanodeInfo);
 +      }
 +      assertEquals(groupSize, lb.getLocations().length);
 +      assertEquals(groupSize, locs.size());
 +
 +      // verify that every internal blocks exists
 +      int[] blockIndices = ((LocatedStripedBlock) lb).getBlockIndices();
 +      assertEquals(groupSize, blockIndices.length);
 +      HashSet<Integer> found = new HashSet<>();
 +      for (int index : blockIndices) {
 +        assert index >=0;
 +        found.add(index);
 +      }
 +      assertEquals(groupSize, found.size());
 +    }
 +  }
++
+   public static void waitForMetric(final JMXGet jmx, final String metricName, final int expectedValue)
+       throws TimeoutException, InterruptedException {
+     GenericTestUtils.waitFor(new Supplier<Boolean>() {
+       @Override
+       public Boolean get() {
+         try {
+           final int currentValue = Integer.parseInt(jmx.getValue(metricName));
+           LOG.info("Waiting for " + metricName +
+                        " to reach value " + expectedValue +
+                        ", current value = " + currentValue);
+           return currentValue == expectedValue;
+         } catch (Exception e) {
+           throw new UnhandledException("Test failed due to unexpected exception", e);
+         }
+       }
+     }, 1000, Integer.MAX_VALUE);
+   }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
index 50f98a3,0000000..c28bff8
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
@@@ -1,160 -1,0 +1,163 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hdfs;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FSDataOutputStream;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.junit.Assert;
++import org.junit.Ignore;
 +import org.junit.Test;
 +
 +import java.io.IOException;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize;
 +import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks;
 +import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs;
 +import static org.apache.hadoop.hdfs.StripedFileTestUtil.parityBlocks;
 +
 +public class TestWriteStripedFileWithFailure {
 +  public static final Log LOG = LogFactory
 +      .getLog(TestWriteStripedFileWithFailure.class);
 +  private static MiniDFSCluster cluster;
 +  private static FileSystem fs;
 +  private static Configuration conf = new HdfsConfiguration();
 +  private final int smallFileLength = blockSize * dataBlocks - 123;
 +  private final int largeFileLength = blockSize * dataBlocks + 123;
 +  private final int[] fileLengths = {smallFileLength, largeFileLength};
 +
 +  public void setup() throws IOException {
 +    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
 +    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
 +    cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
 +    fs = cluster.getFileSystem();
 +  }
 +
 +  public void tearDown() throws IOException {
 +    if (cluster != null) {
 +      cluster.shutdown();
 +    }
 +  }
 +
 +  // Test writing file with some Datanodes failure
++  // TODO: enable this test after HDFS-8704 and HDFS-9040
++  @Ignore
 +  @Test(timeout = 300000)
 +  public void testWriteStripedFileWithDNFailure() throws IOException {
 +    for (int fileLength : fileLengths) {
 +      for (int dataDelNum = 1; dataDelNum < 4; dataDelNum++) {
 +        for (int parityDelNum = 0; (dataDelNum+parityDelNum) < 4; parityDelNum++) {
 +          try {
 +            // setup a new cluster with no dead datanode
 +            setup();
 +            writeFileWithDNFailure(fileLength, dataDelNum, parityDelNum);
 +          } catch (IOException ioe) {
 +            String fileType = fileLength < (blockSize * dataBlocks) ?
 +                "smallFile" : "largeFile";
 +            LOG.error("Failed to write file with DN failure:"
 +                + " fileType = "+ fileType
 +                + ", dataDelNum = " + dataDelNum
 +                + ", parityDelNum = " + parityDelNum);
 +            throw ioe;
 +          } finally {
 +            // tear down the cluster
 +            tearDown();
 +          }
 +        }
 +      }
 +    }
 +  }
 +
 +  /**
 +   * Test writing a file with shutting down some DNs(data DNs or parity DNs or both).
 +   * @param fileLength file length
 +   * @param dataDNFailureNum the shutdown number of data DNs
 +   * @param parityDNFailureNum the shutdown number of parity DNs
 +   * @throws IOException
 +   */
 +  private void writeFileWithDNFailure(int fileLength,
 +      int dataDNFailureNum, int parityDNFailureNum) throws IOException {
 +    String fileType = fileLength < (blockSize * dataBlocks) ?
 +        "smallFile" : "largeFile";
 +    String src = "/dnFailure_" + dataDNFailureNum + "_" + parityDNFailureNum
 +        + "_" + fileType;
 +    LOG.info("writeFileWithDNFailure: file = " + src
 +        + ", fileType = " + fileType
 +        + ", dataDNFailureNum = " + dataDNFailureNum
 +        + ", parityDNFailureNum = " + parityDNFailureNum);
 +
 +    Path srcPath = new Path(src);
 +    final AtomicInteger pos = new AtomicInteger();
 +    final FSDataOutputStream out = fs.create(srcPath);
 +    final DFSStripedOutputStream stripedOut
 +        = (DFSStripedOutputStream)out.getWrappedStream();
 +
 +    int[] dataDNFailureIndices = StripedFileTestUtil.randomArray(0, dataBlocks,
 +        dataDNFailureNum);
 +    Assert.assertNotNull(dataDNFailureIndices);
 +    int[] parityDNFailureIndices = StripedFileTestUtil.randomArray(dataBlocks,
 +        dataBlocks + parityBlocks, parityDNFailureNum);
 +    Assert.assertNotNull(parityDNFailureIndices);
 +
 +    int[] failedDataNodes = new int[dataDNFailureNum + parityDNFailureNum];
 +    System.arraycopy(dataDNFailureIndices, 0, failedDataNodes,
 +        0, dataDNFailureIndices.length);
 +    System.arraycopy(parityDNFailureIndices, 0, failedDataNodes,
 +        dataDNFailureIndices.length, parityDNFailureIndices.length);
 +
 +    final int killPos = fileLength/2;
 +    for (; pos.get() < fileLength; ) {
 +      final int i = pos.getAndIncrement();
 +      if (i == killPos) {
 +        for(int failedDn : failedDataNodes) {
 +          StripedFileTestUtil.killDatanode(cluster, stripedOut, failedDn, pos);
 +        }
 +      }
 +      write(out, i);
 +    }
 +    out.close();
 +
 +    // make sure the expected number of Datanode have been killed
 +    int dnFailureNum = dataDNFailureNum + parityDNFailureNum;
 +    Assert.assertEquals(cluster.getDataNodes().size(), numDNs - dnFailureNum);
 +
 +    byte[] smallBuf = new byte[1024];
 +    byte[] largeBuf = new byte[fileLength + 100];
 +    final byte[] expected = StripedFileTestUtil.generateBytes(fileLength);
 +    StripedFileTestUtil.verifyLength(fs, srcPath, fileLength);
 +    StripedFileTestUtil.verifySeek(fs, srcPath, fileLength);
 +    StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected,
 +        smallBuf);
 +    StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected, largeBuf);
 +
 +    // delete the file
 +    fs.delete(srcPath, true);
 +  }
 +
 +  void write(FSDataOutputStream out, int i) throws IOException {
 +    try {
 +      out.write(StripedFileTestUtil.getByte(i));
 +    } catch (IOException e) {
 +      throw new IOException("Failed at i=" + i, e);
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
index 0a27614,851e5b9..00a4575
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
@@@ -516,16 -489,16 +516,16 @@@ public class TestPBHelper 
    @Test
    public void testConvertLocatedBlock() {
      LocatedBlock lb = createLocatedBlock();
-     LocatedBlockProto lbProto = PBHelper.convertLocatedBlock(lb);
-     LocatedBlock lb2 = PBHelper.convertLocatedBlockProto(lbProto);
 -    LocatedBlockProto lbProto = PBHelperClient.convert(lb);
 -    LocatedBlock lb2 = PBHelperClient.convert(lbProto);
++    LocatedBlockProto lbProto = PBHelperClient.convertLocatedBlock(lb);
++    LocatedBlock lb2 = PBHelperClient.convertLocatedBlockProto(lbProto);
      compare(lb,lb2);
    }
  
    @Test
    public void testConvertLocatedBlockNoStorageMedia() {
      LocatedBlock lb = createLocatedBlockNoStorageMedia();
-     LocatedBlockProto lbProto = PBHelper.convertLocatedBlock(lb);
-     LocatedBlock lb2 = PBHelper.convertLocatedBlockProto(lbProto);
 -    LocatedBlockProto lbProto = PBHelperClient.convert(lb);
 -    LocatedBlock lb2 = PBHelperClient.convert(lbProto);
++    LocatedBlockProto lbProto = PBHelperClient.convertLocatedBlock(lb);
++    LocatedBlock lb2 = PBHelperClient.convertLocatedBlockProto(lbProto);
      compare(lb,lb2);
    }
  
@@@ -535,8 -508,8 +535,8 @@@
      for (int i=0;i<3;i++) {
        lbl.add(createLocatedBlock());
      }
-     List<LocatedBlockProto> lbpl = PBHelper.convertLocatedBlocks2(lbl);
-     List<LocatedBlock> lbl2 = PBHelper.convertLocatedBlocks(lbpl);
 -    List<LocatedBlockProto> lbpl = PBHelperClient.convertLocatedBlock2(lbl);
 -    List<LocatedBlock> lbl2 = PBHelperClient.convertLocatedBlock(lbpl);
++    List<LocatedBlockProto> lbpl = PBHelperClient.convertLocatedBlocks2(lbl);
++    List<LocatedBlock> lbl2 = PBHelperClient.convertLocatedBlocks(lbpl);
      assertEquals(lbl.size(), lbl2.size());
      for (int i=0;i<lbl.size();i++) {
        compare(lbl.get(i), lbl2.get(2));
@@@ -549,8 -522,8 +549,8 @@@
      for (int i=0;i<3;i++) {
        lbl[i] = createLocatedBlock();
      }
-     LocatedBlockProto [] lbpl = PBHelper.convertLocatedBlocks(lbl);
-     LocatedBlock [] lbl2 = PBHelper.convertLocatedBlocks(lbpl);
 -    LocatedBlockProto [] lbpl = PBHelperClient.convertLocatedBlock(lbl);
 -    LocatedBlock [] lbl2 = PBHelperClient.convertLocatedBlock(lbpl);
++    LocatedBlockProto [] lbpl = PBHelperClient.convertLocatedBlocks(lbl);
++    LocatedBlock [] lbl2 = PBHelperClient.convertLocatedBlocks(lbpl);
      assertEquals(lbl.length, lbl2.length);
      for (int i=0;i<lbl.length;i++) {
        compare(lbl[i], lbl2[i]);
@@@ -664,99 -637,6 +664,99 @@@
          .setType(AclEntryType.OTHER).build();
      AclStatus s = new AclStatus.Builder().owner("foo").group("bar").addEntry(e)
          .build();
-     Assert.assertEquals(s, PBHelper.convert(PBHelper.convert(s)));
+     Assert.assertEquals(s, PBHelperClient.convert(PBHelperClient.convert(s)));
    }
 +  
 +  @Test
 +  public void testBlockECRecoveryCommand() {
 +    DatanodeInfo[] dnInfos0 = new DatanodeInfo[] {
 +        DFSTestUtil.getLocalDatanodeInfo(), DFSTestUtil.getLocalDatanodeInfo() };
 +    DatanodeStorageInfo targetDnInfos_0 = BlockManagerTestUtil
 +        .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(),
 +            new DatanodeStorage("s00"));
 +    DatanodeStorageInfo targetDnInfos_1 = BlockManagerTestUtil
 +        .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(),
 +            new DatanodeStorage("s01"));
 +    DatanodeStorageInfo[] targetDnInfos0 = new DatanodeStorageInfo[] {
 +        targetDnInfos_0, targetDnInfos_1 };
 +    short[] liveBlkIndices0 = new short[2];
 +    BlockECRecoveryInfo blkECRecoveryInfo0 = new BlockECRecoveryInfo(
 +        new ExtendedBlock("bp1", 1234), dnInfos0, targetDnInfos0,
 +        liveBlkIndices0, ErasureCodingPolicyManager.getSystemDefaultPolicy());
 +    DatanodeInfo[] dnInfos1 = new DatanodeInfo[] {
 +        DFSTestUtil.getLocalDatanodeInfo(), DFSTestUtil.getLocalDatanodeInfo() };
 +    DatanodeStorageInfo targetDnInfos_2 = BlockManagerTestUtil
 +        .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(),
 +            new DatanodeStorage("s02"));
 +    DatanodeStorageInfo targetDnInfos_3 = BlockManagerTestUtil
 +        .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(),
 +            new DatanodeStorage("s03"));
 +    DatanodeStorageInfo[] targetDnInfos1 = new DatanodeStorageInfo[] {
 +        targetDnInfos_2, targetDnInfos_3 };
 +    short[] liveBlkIndices1 = new short[2];
 +    BlockECRecoveryInfo blkECRecoveryInfo1 = new BlockECRecoveryInfo(
 +        new ExtendedBlock("bp2", 3256), dnInfos1, targetDnInfos1,
 +        liveBlkIndices1, ErasureCodingPolicyManager.getSystemDefaultPolicy());
 +    List<BlockECRecoveryInfo> blkRecoveryInfosList = new ArrayList<BlockECRecoveryInfo>();
 +    blkRecoveryInfosList.add(blkECRecoveryInfo0);
 +    blkRecoveryInfosList.add(blkECRecoveryInfo1);
 +    BlockECRecoveryCommand blkECRecoveryCmd = new BlockECRecoveryCommand(
 +        DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY, blkRecoveryInfosList);
 +    BlockECRecoveryCommandProto blkECRecoveryCmdProto = PBHelper
 +        .convert(blkECRecoveryCmd);
 +    blkECRecoveryCmd = PBHelper.convert(blkECRecoveryCmdProto);
 +    Iterator<BlockECRecoveryInfo> iterator = blkECRecoveryCmd.getECTasks()
 +        .iterator();
 +    assertBlockECRecoveryInfoEquals(blkECRecoveryInfo0, iterator.next());
 +    assertBlockECRecoveryInfoEquals(blkECRecoveryInfo1, iterator.next());
 +  }
 +
 +  private void assertBlockECRecoveryInfoEquals(
 +      BlockECRecoveryInfo blkECRecoveryInfo1,
 +      BlockECRecoveryInfo blkECRecoveryInfo2) {
 +    assertEquals(blkECRecoveryInfo1.getExtendedBlock(),
 +        blkECRecoveryInfo2.getExtendedBlock());
 +
 +    DatanodeInfo[] sourceDnInfos1 = blkECRecoveryInfo1.getSourceDnInfos();
 +    DatanodeInfo[] sourceDnInfos2 = blkECRecoveryInfo2.getSourceDnInfos();
 +    assertDnInfosEqual(sourceDnInfos1, sourceDnInfos2);
 +
 +    DatanodeInfo[] targetDnInfos1 = blkECRecoveryInfo1.getTargetDnInfos();
 +    DatanodeInfo[] targetDnInfos2 = blkECRecoveryInfo2.getTargetDnInfos();
 +    assertDnInfosEqual(targetDnInfos1, targetDnInfos2);
 +
 +    String[] targetStorageIDs1 = blkECRecoveryInfo1.getTargetStorageIDs();
 +    String[] targetStorageIDs2 = blkECRecoveryInfo2.getTargetStorageIDs();
 +    assertEquals(targetStorageIDs1.length, targetStorageIDs2.length);
 +    for (int i = 0; i < targetStorageIDs1.length; i++) {
 +      assertEquals(targetStorageIDs1[i], targetStorageIDs2[i]);
 +    }
 +
 +    short[] liveBlockIndices1 = blkECRecoveryInfo1.getLiveBlockIndices();
 +    short[] liveBlockIndices2 = blkECRecoveryInfo2.getLiveBlockIndices();
 +    for (int i = 0; i < liveBlockIndices1.length; i++) {
 +      assertEquals(liveBlockIndices1[i], liveBlockIndices2[i]);
 +    }
 +    
 +    ErasureCodingPolicy ecPolicy1 = blkECRecoveryInfo1.getErasureCodingPolicy();
 +    ErasureCodingPolicy ecPolicy2 = blkECRecoveryInfo2.getErasureCodingPolicy();
 +    // Compare ECPolicies same as default ECPolicy as we used system default
 +    // ECPolicy used in this test
 +    compareECPolicies(ErasureCodingPolicyManager.getSystemDefaultPolicy(), ecPolicy1);
 +    compareECPolicies(ErasureCodingPolicyManager.getSystemDefaultPolicy(), ecPolicy2);
 +  }
 +
 +  private void compareECPolicies(ErasureCodingPolicy ecPolicy1, ErasureCodingPolicy ecPolicy2) {
 +    assertEquals(ecPolicy1.getName(), ecPolicy2.getName());
 +    assertEquals(ecPolicy1.getNumDataUnits(), ecPolicy2.getNumDataUnits());
 +    assertEquals(ecPolicy1.getNumParityUnits(), ecPolicy2.getNumParityUnits());
 +  }
 +
 +  private void assertDnInfosEqual(DatanodeInfo[] dnInfos1,
 +      DatanodeInfo[] dnInfos2) {
 +    assertEquals(dnInfos1.length, dnInfos2.length);
 +    for (int i = 0; i < dnInfos1.length; i++) {
 +      compare(dnInfos1[i], dnInfos2[i]);
 +    }
 +  }
  }


[44/50] [abbrv] hadoop git commit: HADOOP-12428. Fix inconsistency between log-level guards and statements. Contributed by Jagadesh Kiran N and Jackie Chang.

Posted by zh...@apache.org.
HADOOP-12428. Fix inconsistency between log-level guards and statements. Contributed by Jagadesh Kiran N and Jackie Chang.


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

Branch: refs/heads/HDFS-7285
Commit: dfd807afab0fae3839c9cc5d552aa0304444f956
Parents: 06022b8
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Tue Sep 22 12:54:29 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Tue Sep 22 12:54:29 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                  | 3 +++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                      | 3 +++
 .../java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java | 4 ++--
 .../java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java | 2 +-
 .../apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java  | 2 +-
 .../org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java | 2 +-
 hadoop-mapreduce-project/CHANGES.txt                             | 3 +++
 .../apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java  | 2 +-
 .../apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java  | 2 +-
 .../org/apache/hadoop/tools/mapred/UniformSizeInputFormat.java   | 3 ++-
 .../java/org/apache/hadoop/mapred/gridmix/SerialJobFactory.java  | 2 +-
 hadoop-yarn-project/CHANGES.txt                                  | 3 +++
 .../org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java   | 2 +-
 .../org/apache/hadoop/yarn/security/ContainerTokenSelector.java  | 2 +-
 .../java/org/apache/hadoop/yarn/security/NMTokenSelector.java    | 2 +-
 .../server/resourcemanager/scheduler/capacity/LeafQueue.java     | 4 ++--
 16 files changed, 27 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfd807af/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 0c8550d..6bc3379 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -788,6 +788,9 @@ Release 2.8.0 - UNRELEASED
     JarFile with other users when loading resource from URL in Configuration
     class. (zxu)
 
+    HADOOP-12428. Fix inconsistency between log-level guards and statements.
+    (Jagadesh Kiran N and Jackie Chang via ozawa)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfd807af/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 944986c..44774a7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -941,6 +941,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9111. Move hdfs-client protobuf convert methods from PBHelper to
     PBHelperClient. (Mingliang Liu via wheat9)
 
+    HADOOP-12428. Fix inconsistency between log-level guards and statements.
+    (Jagadesh Kiran N and Jackie Chang via ozawa)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfd807af/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index 4f6dc96..8c4e38a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -752,7 +752,7 @@ class DataXceiver extends Receiver implements Runnable {
             mirrorInStatus = connectAck.getStatus();
             firstBadLink = connectAck.getFirstBadLink();
             if (LOG.isDebugEnabled() || mirrorInStatus != SUCCESS) {
-              LOG.info("Datanode " + targets.length +
+              LOG.debug("Datanode " + targets.length +
                        " got response for connect ack " +
                        " from downstream datanode with firstbadlink as " +
                        firstBadLink);
@@ -791,7 +791,7 @@ class DataXceiver extends Receiver implements Runnable {
       // send connect-ack to source for clients and not transfer-RBW/Finalized
       if (isClient && !isTransfer) {
         if (LOG.isDebugEnabled() || mirrorInStatus != SUCCESS) {
-          LOG.info("Datanode " + targets.length +
+          LOG.debug("Datanode " + targets.length +
                    " forwarding connect ack to upstream firstbadlink is " +
                    firstBadLink);
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfd807af/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
index c6ae0d5..450ddee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
@@ -200,7 +200,7 @@ public class BackupImage extends FSImage {
     assert backupInputStream.length() == 0 : "backup input stream is not empty";
     try {
       if (LOG.isTraceEnabled()) {
-        LOG.debug("data:" + StringUtils.byteToHexString(data));
+        LOG.trace("data:" + StringUtils.byteToHexString(data));
       }
 
       FSEditLogLoader logLoader =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfd807af/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
index a84dcf1..deb76b3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
@@ -375,7 +375,7 @@ public class BootstrapStandby implements Tool, Configurable {
           "or call saveNamespace on the active node.\n" +
           "Error: " + e.getLocalizedMessage();
       if (LOG.isDebugEnabled()) {
-        LOG.fatal(msg, e);
+        LOG.debug(msg, e);
       } else {
         LOG.fatal(msg);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfd807af/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
index cfca77c..59f90a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
@@ -260,7 +260,7 @@ public class EditLogTailer {
         throw elie;
       } finally {
         if (editsLoaded > 0 || LOG.isDebugEnabled()) {
-          LOG.info(String.format("Loaded %d edits starting from txid %d ",
+          LOG.debug(String.format("Loaded %d edits starting from txid %d ",
               editsLoaded, lastTxnId));
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfd807af/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 42ea011..bcdac1f 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -387,6 +387,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6483. Replace deprecated method NameNode.getUri() with
     DFSUtilClient.getNNUri() in TestMRCredentials. (Mingliang Liu via wheat9)
 
+    HADOOP-12428. Fix inconsistency between log-level guards and statements.
+    (Jagadesh Kiran N and Jackie Chang via ozawa)
+
   OPTIMIZATIONS
 
     MAPREDUCE-6376. Add avro binary support for jhist files (Ray Chiang via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfd807af/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
index 78b0dc4..74a2753 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
@@ -1122,7 +1122,7 @@ public class RMContainerAllocator extends RMContainerRequestor
       assignedRequests.add(allocated, assigned.attemptID);
 
       if (LOG.isDebugEnabled()) {
-        LOG.info("Assigned container (" + allocated + ") "
+        LOG.debug("Assigned container (" + allocated + ") "
             + " to task " + assigned.attemptID + " on node "
             + allocated.getNodeId().toString());
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfd807af/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
index 155711f..d612126 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
@@ -506,7 +506,7 @@ public abstract class RMContainerRequestor extends RMCommunicator {
     addResourceRequestToAsk(remoteRequest);
 
     if (LOG.isDebugEnabled()) {
-      LOG.info("AFTER decResourceRequest:" + " applicationId="
+      LOG.debug("AFTER decResourceRequest:" + " applicationId="
           + applicationId.getId() + " priority=" + priority.getPriority()
           + " resourceName=" + resourceName + " numContainers="
           + remoteRequest.getNumContainers() + " #asks=" + ask.size());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfd807af/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/UniformSizeInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/UniformSizeInputFormat.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/UniformSizeInputFormat.java
index 8dc7a65..8f31234 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/UniformSizeInputFormat.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/UniformSizeInputFormat.java
@@ -112,7 +112,8 @@ public class UniformSizeInputFormat
         FileSplit split = new FileSplit(listingFilePath, lastSplitStart,
             lastPosition - lastSplitStart, null);
         if (LOG.isDebugEnabled()) {
-          LOG.info ("Creating split : " + split + ", bytes in split: " + currentSplitSize);
+          LOG.debug("Creating split : " + split + ", bytes in split: "
+              + currentSplitSize);
         }
         splits.add(split);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfd807af/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/SerialJobFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/SerialJobFactory.java b/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/SerialJobFactory.java
index af554ff..471d140 100644
--- a/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/SerialJobFactory.java
+++ b/hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/SerialJobFactory.java
@@ -130,7 +130,7 @@ public class SerialJobFactory extends JobFactory<JobStats> {
                   return;
                 }
                 if (LOG.isDebugEnabled()) {
-                  LOG.info(" job " + job.getName() + " completed ");
+                  LOG.debug(" job " + job.getName() + " completed ");
                 }
                 break;
               }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfd807af/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 0e20d9a..329e743 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -457,6 +457,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3920. FairScheduler container reservation on a node should be
     configurable to limit it to large containers (adhoot via asuresh)
 
+    HADOOP-12428. Fix inconsistency between log-level guards and statements.
+    (Jagadesh Kiran N and Jackie Chang via ozawa)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfd807af/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
index b1324c1..4cf9aa0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
@@ -748,7 +748,7 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
     }
 
     if (LOG.isDebugEnabled()) {
-      LOG.info("AFTER decResourceRequest:" + " applicationId="
+      LOG.debug("AFTER decResourceRequest:" + " applicationId="
           + " priority=" + priority.getPriority()
           + " resourceName=" + resourceName + " numContainers="
           + resourceRequestInfo.remoteRequest.getNumContainers() 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfd807af/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenSelector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenSelector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenSelector.java
index 5a3b494..a1c0a91 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenSelector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenSelector.java
@@ -46,7 +46,7 @@ public class ContainerTokenSelector implements
     }
     for (Token<? extends TokenIdentifier> token : tokens) {
       if (LOG.isDebugEnabled()) {
-        LOG.info("Looking for service: " + service + ". Current token is "
+        LOG.debug("Looking for service: " + service + ". Current token is "
             + token);
       }
       if (ContainerTokenIdentifier.KIND.equals(token.getKind()) && 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfd807af/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/NMTokenSelector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/NMTokenSelector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/NMTokenSelector.java
index c57e4a2..43899b5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/NMTokenSelector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/NMTokenSelector.java
@@ -42,7 +42,7 @@ public class NMTokenSelector implements
     }
     for (Token<? extends TokenIdentifier> token : tokens) {
       if (LOG.isDebugEnabled()) {
-        LOG.info("Looking for service: " + service + ". Current token is "
+        LOG.debug("Looking for service: " + service + ". Current token is "
             + token);
       }
       if (NMTokenIdentifier.KIND.equals(token.getKind()) && 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfd807af/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 b43f658..869b49a 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
@@ -1163,8 +1163,8 @@ public class LeafQueue extends AbstractCSQueue {
     metrics.setAvailableResourcesToUser(userName, application.getHeadroom());
     
     if (LOG.isDebugEnabled()) {
-      LOG.info(getQueueName() + 
-          " user=" + userName + 
+      LOG.debug(getQueueName() +
+          " user=" + userName +
           " used=" + queueUsage.getUsed() + " numContainers=" + numContainers +
           " headroom = " + application.getHeadroom() +
           " user-resources=" + user.getUsed()


[13/50] [abbrv] hadoop git commit: YARN-4034. Render cluster Max Priority in scheduler metrics in RM web UI. Contributed by Rohith Sharma K S

Posted by zh...@apache.org.
YARN-4034. Render cluster Max Priority in scheduler metrics in RM web UI. Contributed by Rohith Sharma K S


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

Branch: refs/heads/HDFS-7285
Commit: 6c6e734f0baaa7b0f8d6b85963e1ce87bac28b17
Parents: 0832b38
Author: Jian He <ji...@apache.org>
Authored: Thu Sep 17 14:55:50 2015 +0800
Committer: Jian He <ji...@apache.org>
Committed: Thu Sep 17 14:55:50 2015 +0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                               | 3 +++
 .../resourcemanager/scheduler/AbstractYarnScheduler.java      | 1 +
 .../yarn/server/resourcemanager/scheduler/YarnScheduler.java  | 7 +++++++
 .../server/resourcemanager/webapp/CapacitySchedulerPage.java  | 4 +++-
 .../server/resourcemanager/webapp/MetricsOverviewTable.java   | 3 +++
 .../webapp/dao/CapacitySchedulerLeafQueueInfo.java            | 6 ++++++
 .../yarn/server/resourcemanager/webapp/dao/SchedulerInfo.java | 6 ++++++
 .../yarn/server/resourcemanager/webapp/TestNodesPage.java     | 2 +-
 .../webapp/TestRMWebServicesCapacitySched.java                | 2 +-
 9 files changed, 31 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c6e734f/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index e5cb40e..18318fb 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -194,6 +194,9 @@ Release 2.8.0 - UNRELEASED
     command line. (Inigo Goiri, Kenji Kikushima and Junping Du 
     via junping_du)
 
+    YARN-4034. Render cluster Max Priority in scheduler metrics in RM web
+    UI. (Rohith Sharma K S via jianhe)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c6e734f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index 4c34c47..27d70cc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -713,6 +713,7 @@ public abstract class AbstractYarnScheduler
     // specific scheduler.
   }
 
+  @Override
   public Priority getMaxClusterLevelAppPriority() {
     return maxClusterLevelAppPriority;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c6e734f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
index e3c79f7..699d476 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
@@ -343,4 +343,11 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
    */
   List<ResourceRequest> getPendingResourceRequestsForAttempt(
       ApplicationAttemptId attemptId);
+
+  /**
+   * Get cluster max priority.
+   * 
+   * @return maximum priority of cluster
+   */
+  Priority getMaxClusterLevelAppPriority();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c6e734f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
index 9e27627..766eb93 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
@@ -158,7 +158,9 @@ class CapacitySchedulerPage extends RmView {
       _("Default Node Label Expression:",
               lqinfo.getDefaultNodeLabelExpression() == null
                   ? NodeLabel.DEFAULT_NODE_LABEL_PARTITION
-                  : lqinfo.getDefaultNodeLabelExpression());
+                  : lqinfo.getDefaultNodeLabelExpression()).
+      _("Default Application Priority:",
+              Integer.toString(lqinfo.getDefaultApplicationPriority()));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c6e734f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
index a5a9a7f..f59d5ed 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
@@ -167,6 +167,8 @@ public class MetricsOverviewTable extends HtmlBlock {
         th().$class("ui-state-default")._("Scheduling Resource Type")._().
         th().$class("ui-state-default")._("Minimum Allocation")._().
         th().$class("ui-state-default")._("Maximum Allocation")._().
+        th().$class("ui-state-default")
+            ._("Maximum Cluster Application Priority")._().
       _().
     _().
     tbody().$class("ui-widget-content").
@@ -175,6 +177,7 @@ public class MetricsOverviewTable extends HtmlBlock {
         td(String.valueOf(schedulerInfo.getSchedulerResourceTypes())).
         td(schedulerInfo.getMinAllocation().toString()).
         td(schedulerInfo.getMaxAllocation().toString()).
+        td(String.valueOf(schedulerInfo.getMaxClusterLevelAppPriority())).
       _().
     _()._();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c6e734f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java
index f31a256..5763177 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java
@@ -41,6 +41,7 @@ public class CapacitySchedulerLeafQueueInfo extends CapacitySchedulerQueueInfo {
   protected ResourceInfo userAMResourceLimit;
   protected boolean preemptionDisabled;
   protected String defaultNodeLabelExpression;
+  protected int defaultPriority;
 
   @XmlTransient
   protected String orderingPolicyInfo;
@@ -64,6 +65,7 @@ public class CapacitySchedulerLeafQueueInfo extends CapacitySchedulerQueueInfo {
     preemptionDisabled = q.getPreemptionDisabled();
     orderingPolicyInfo = q.getOrderingPolicy().getInfo();
     defaultNodeLabelExpression = q.getDefaultNodeLabelExpression();
+    defaultPriority = q.getDefaultApplicationPriority().getPriority();
   }
 
   public int getNumActiveApplications() {
@@ -122,4 +124,8 @@ public class CapacitySchedulerLeafQueueInfo extends CapacitySchedulerQueueInfo {
   public String getDefaultNodeLabelExpression() {
     return defaultNodeLabelExpression;
   }
+
+  public int getDefaultApplicationPriority() {
+    return defaultPriority;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c6e734f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedulerInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedulerInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedulerInfo.java
index 185b449..cf93edd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedulerInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedulerInfo.java
@@ -38,6 +38,7 @@ public class SchedulerInfo {
   protected ResourceInfo minAllocResource;
   protected ResourceInfo maxAllocResource;
   protected EnumSet<SchedulerResourceTypes> schedulingResourceTypes;
+  protected int maximumClusterPriority;
 
   public SchedulerInfo() {
   } // JAXB needs this
@@ -55,6 +56,8 @@ public class SchedulerInfo {
     this.minAllocResource = new ResourceInfo(rs.getMinimumResourceCapability());
     this.maxAllocResource = new ResourceInfo(rs.getMaximumResourceCapability());
     this.schedulingResourceTypes = rs.getSchedulingResourceTypes();
+    this.maximumClusterPriority =
+        rs.getMaxClusterLevelAppPriority().getPriority();
   }
 
   public String getSchedulerType() {
@@ -73,4 +76,7 @@ public class SchedulerInfo {
     return this.schedulingResourceTypes.toString();
   }
 
+  public int getMaxClusterLevelAppPriority() {
+    return this.maximumClusterPriority;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c6e734f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java
index 7c6d9a0..b70fdc1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java
@@ -48,7 +48,7 @@ public class TestNodesPage {
 
   // Number of Actual Table Headers for NodesPage.NodesBlock might change in
   // future. In that case this value should be adjusted to the new value.
-  final int numberOfThInMetricsTable = 21;
+  final int numberOfThInMetricsTable = 22;
   final int numberOfActualTableHeaders = 13;
 
   private Injector injector;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c6e734f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
index dad720e..dd0fcf9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
@@ -352,7 +352,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase {
     int numExpectedElements = 16;
     boolean isParentQueue = true;
     if (!info.has("queues")) {
-      numExpectedElements = 28;
+      numExpectedElements = 29;
       isParentQueue = false;
     }
     assertEquals("incorrect number of elements", numExpectedElements, info.length());


[02/50] [abbrv] hadoop git commit: HDFS-9082. Change the log level in WebHdfsFileSystem.initialize() from INFO to DEBUG. Contributed by Santhosh Nayak.

Posted by zh...@apache.org.
HDFS-9082. Change the log level in WebHdfsFileSystem.initialize() from INFO to DEBUG. Contributed by Santhosh Nayak.


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

Branch: refs/heads/HDFS-7285
Commit: 559c09dc0eba28666c4b16435512cc2d35e31683
Parents: 34ef1a0
Author: cnauroth <cn...@apache.org>
Authored: Tue Sep 15 15:13:43 2015 -0700
Committer: cnauroth <cn...@apache.org>
Committed: Tue Sep 15 15:13:43 2015 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java  | 4 ++--
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                      | 3 +++
 2 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/559c09dc/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index a75e78f..cfda3fe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -154,11 +154,11 @@ public class WebHdfsFileSystem extends FileSystem
         HdfsClientConfigKeys.DFS_WEBHDFS_OAUTH_ENABLED_DEFAULT);
 
     if(isOAuth) {
-      LOG.info("Enabling OAuth2 in WebHDFS");
+      LOG.debug("Enabling OAuth2 in WebHDFS");
       connectionFactory = URLConnectionFactory
           .newOAuth2URLConnectionFactory(conf);
     } else {
-      LOG.info("Not enabling OAuth2 in WebHDFS");
+      LOG.debug("Not enabling OAuth2 in WebHDFS");
       connectionFactory = URLConnectionFactory
           .newDefaultURLConnectionFactory(conf);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/559c09dc/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 6da3cff..4b310dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -920,6 +920,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8953. DataNode Metrics logging (Kanaka Kumar Avvaru via vinayakumarb)
 
+    HDFS-9082. Change the log level in WebHdfsFileSystem.initialize() from INFO
+    to DEBUG. (Santhosh Nayak via cnauroth)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than


[19/50] [abbrv] hadoop git commit: YARN-4135. Improve the assertion message in MockRM while failing after waiting for the state.(Nijel S F via rohithsharmaks)

Posted by zh...@apache.org.
YARN-4135. Improve the assertion message in MockRM while failing after waiting for the state.(Nijel S F via rohithsharmaks)


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

Branch: refs/heads/HDFS-7285
Commit: 723c31d45bc0f64b1d1a67350b108059d2a220a3
Parents: 6b97fa6
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Fri Sep 18 08:44:10 2015 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Fri Sep 18 08:44:10 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                | 3 +++
 .../org/apache/hadoop/yarn/server/resourcemanager/MockRM.java  | 6 ++++--
 2 files changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/723c31d4/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 18318fb..90b0e82 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -848,6 +848,9 @@ Release 2.8.0 - UNRELEASED
     YARN-4078. Add getPendingResourceRequestForAttempt in YarnScheduler interface.
     (Naganarasimha G R via jianhe)
 
+    YARN-4135. Improve the assertion message in MockRM while failing after waiting for the state.
+    (Nijel S F via rohithsharmaks)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/723c31d4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index e464401..249f093 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -156,7 +156,8 @@ public class MockRM extends ResourceManager {
     LOG.info("App State is : " + app.getState());
     if (waitedMsecs >= timeoutMsecs) {
       Assert.fail("App state is not correct (timedout): expected: " +
-          finalState + " actual: " + app.getState());
+          finalState + " actual: " + app.getState() +
+          " for the application " + appId);
     }
   }
   
@@ -185,7 +186,8 @@ public class MockRM extends ResourceManager {
     LOG.info("Attempt State is : " + attempt.getAppAttemptState());
     if (waitedMsecs >= timeoutMsecs) {
       Assert.fail("Attempt state is not correct (timedout): expected: "
-          + finalState + " actual: " + attempt.getAppAttemptState());
+          + finalState + " actual: " + attempt.getAppAttemptState()+
+          " for the application attempt " + attemptId);
     }
   }
 


[14/50] [abbrv] hadoop git commit: MAPREDUCE-6481. LineRecordReader may give incomplete record and wrong position/key information for uncompressed input sometimes. Contributed by Zhihai Xu

Posted by zh...@apache.org.
MAPREDUCE-6481. LineRecordReader may give incomplete record and wrong position/key information for uncompressed input sometimes. 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/58d1a02b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/58d1a02b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/58d1a02b

Branch: refs/heads/HDFS-7285
Commit: 58d1a02b8d66b1d2a6ac2158be32bd35ad2e69bd
Parents: 6c6e734
Author: Jason Lowe <jl...@apache.org>
Authored: Thu Sep 17 14:30:18 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Thu Sep 17 14:30:18 2015 +0000

----------------------------------------------------------------------
 .../java/org/apache/hadoop/util/LineReader.java |  17 +-
 hadoop-mapreduce-project/CHANGES.txt            |   4 +
 .../lib/input/UncompressedSplitLineReader.java  |  31 +---
 .../hadoop/mapred/TestLineRecordReader.java     | 138 ++++++++++++++++
 .../lib/input/TestLineRecordReader.java         | 161 +++++++++++++++++++
 5 files changed, 316 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/58d1a02b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java
index 1d1b569..900215a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java
@@ -303,7 +303,10 @@ public class LineReader implements Closeable {
         startPosn = bufferPosn = 0;
         bufferLength = fillBuffer(in, buffer, ambiguousByteCount > 0);
         if (bufferLength <= 0) {
-          str.append(recordDelimiterBytes, 0, ambiguousByteCount);
+          if (ambiguousByteCount > 0) {
+            str.append(recordDelimiterBytes, 0, ambiguousByteCount);
+            bytesConsumed += ambiguousByteCount;
+          }
           break; // EOF
         }
       }
@@ -325,13 +328,13 @@ public class LineReader implements Closeable {
       if (appendLength > maxLineLength - txtLength) {
         appendLength = maxLineLength - txtLength;
       }
+      bytesConsumed += ambiguousByteCount;
+      if (appendLength >= 0 && ambiguousByteCount > 0) {
+        //appending the ambiguous characters (refer case 2.2)
+        str.append(recordDelimiterBytes, 0, ambiguousByteCount);
+        ambiguousByteCount = 0;
+      }
       if (appendLength > 0) {
-        if (ambiguousByteCount > 0) {
-          str.append(recordDelimiterBytes, 0, ambiguousByteCount);
-          //appending the ambiguous characters (refer case 2.2)
-          bytesConsumed += ambiguousByteCount;
-          ambiguousByteCount=0;
-        }
         str.append(buffer, startPosn, appendLength);
         txtLength += appendLength;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58d1a02b/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 669fee5..cde6d92 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -566,6 +566,10 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6452. NPE when intermediate encrypt enabled for LocalRunner.
     (Zhihai Xu)
 
+    MAPREDUCE-6481. LineRecordReader may give incomplete record and wrong
+    position/key information for uncompressed input sometimes. (Zhihai Xu via
+    jlowe)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58d1a02b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/UncompressedSplitLineReader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/UncompressedSplitLineReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/UncompressedSplitLineReader.java
index 52fb7b0..38491b0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/UncompressedSplitLineReader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/UncompressedSplitLineReader.java
@@ -40,8 +40,6 @@ public class UncompressedSplitLineReader extends SplitLineReader {
   private long totalBytesRead = 0;
   private boolean finished = false;
   private boolean usingCRLF;
-  private int unusedBytes = 0;
-  private int lastBytesRead = 0;
 
   public UncompressedSplitLineReader(FSDataInputStream in, Configuration conf,
       byte[] recordDelimiterBytes, long splitLength) throws IOException {
@@ -59,7 +57,6 @@ public class UncompressedSplitLineReader extends SplitLineReader {
                                 (int)(splitLength - totalBytesRead));
     }
     int bytesRead = in.read(buffer, 0, maxBytesToRead);
-    lastBytesRead = bytesRead;
 
     // If the split ended in the middle of a record delimiter then we need
     // to read one additional record, as the consumer of the next split will
@@ -83,39 +80,17 @@ public class UncompressedSplitLineReader extends SplitLineReader {
   @Override
   public int readLine(Text str, int maxLineLength, int maxBytesToConsume)
       throws IOException {
-    long bytesRead = 0;
+    int bytesRead = 0;
     if (!finished) {
       // only allow at most one more record to be read after the stream
       // reports the split ended
       if (totalBytesRead > splitLength) {
         finished = true;
       }
-      bytesRead = totalBytesRead;
-      int bytesConsumed = super.readLine(str, maxLineLength, maxBytesToConsume);
-      bytesRead = totalBytesRead - bytesRead;
 
-      // No records left.
-      if (bytesConsumed == 0 && bytesRead == 0) {
-        return 0;
-      }
-
-      int bufferSize = getBufferSize();
-
-      // Add the remaining buffer size not used for the last call
-      // of fillBuffer method.
-      if (lastBytesRead <= 0) {
-        bytesRead += bufferSize;
-      } else if (bytesRead > 0) {
-        bytesRead += bufferSize - lastBytesRead;
-      }
-
-      // Adjust the size of the buffer not used for this record.
-      // The size is carried over for the next calculation.
-      bytesRead += unusedBytes;
-      unusedBytes = bufferSize - getBufferPosn();
-      bytesRead -= unusedBytes;
+      bytesRead = super.readLine(str, maxLineLength, maxBytesToConsume);
     }
-    return (int) bytesRead;
+    return bytesRead;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58d1a02b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLineRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLineRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLineRecordReader.java
index a5c9933..d33a614 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLineRecordReader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLineRecordReader.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.mapred;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
@@ -357,4 +358,141 @@ public class TestLineRecordReader {
       }
     }
   }
+
+  @Test
+  public void testUncompressedInputCustomDelimiterPosValue()
+      throws Exception {
+    Configuration conf = new Configuration();
+    String inputData = "1234567890ab12ab345";
+    Path inputFile = createInputFile(conf, inputData);
+    conf.setInt("io.file.buffer.size", 10);
+    conf.setInt(org.apache.hadoop.mapreduce.lib.input.
+        LineRecordReader.MAX_LINE_LENGTH, Integer.MAX_VALUE);
+    String delimiter = "ab";
+    byte[] recordDelimiterBytes = delimiter.getBytes(Charsets.UTF_8);
+    FileSplit split = new FileSplit(inputFile, 0, 15, (String[])null);
+    LineRecordReader reader = new LineRecordReader(conf, split,
+        recordDelimiterBytes);
+    LongWritable key = new LongWritable();
+    Text value = new Text();
+    reader.next(key, value);
+    // Get first record:"1234567890"
+    assertEquals(10, value.getLength());
+    // Position should be 12 right after "1234567890ab"
+    assertEquals(12, reader.getPos());
+    reader.next(key, value);
+    // Get second record:"12"
+    assertEquals(2, value.getLength());
+    // Position should be 16 right after "1234567890ab12ab"
+    assertEquals(16, reader.getPos());
+    reader.next(key, value);
+    // Get third record:"345"
+    assertEquals(3, value.getLength());
+    // Position should be 19 right after "1234567890ab12ab345"
+    assertEquals(19, reader.getPos());
+    assertFalse(reader.next(key, value));
+    assertEquals(19, reader.getPos());
+
+    split = new FileSplit(inputFile, 15, 4, (String[])null);
+    reader = new LineRecordReader(conf, split, recordDelimiterBytes);
+    // No record is in the second split because the second split dropped
+    // the first record, which was already reported by the first split.
+    // The position should be 19 right after "1234567890ab12ab345"
+    assertEquals(19, reader.getPos());
+    assertFalse(reader.next(key, value));
+    assertEquals(19, reader.getPos());
+
+    inputData = "123456789aab";
+    inputFile = createInputFile(conf, inputData);
+    split = new FileSplit(inputFile, 0, 12, (String[])null);
+    reader = new LineRecordReader(conf, split, recordDelimiterBytes);
+    reader.next(key, value);
+    // Get first record:"123456789a"
+    assertEquals(10, value.getLength());
+    // Position should be 12 right after "123456789aab"
+    assertEquals(12, reader.getPos());
+    assertFalse(reader.next(key, value));
+    assertEquals(12, reader.getPos());
+
+    inputData = "123456789a";
+    inputFile = createInputFile(conf, inputData);
+    split = new FileSplit(inputFile, 0, 10, (String[])null);
+    reader = new LineRecordReader(conf, split, recordDelimiterBytes);
+    reader.next(key, value);
+    // Get first record:"123456789a"
+    assertEquals(10, value.getLength());
+    // Position should be 10 right after "123456789a"
+    assertEquals(10, reader.getPos());
+    assertFalse(reader.next(key, value));
+    assertEquals(10, reader.getPos());
+
+    inputData = "123456789ab";
+    inputFile = createInputFile(conf, inputData);
+    split = new FileSplit(inputFile, 0, 11, (String[])null);
+    reader = new LineRecordReader(conf, split, recordDelimiterBytes);
+    reader.next(key, value);
+    // Get first record:"123456789"
+    assertEquals(9, value.getLength());
+    // Position should be 11 right after "123456789ab"
+    assertEquals(11, reader.getPos());
+    assertFalse(reader.next(key, value));
+    assertEquals(11, reader.getPos());
+  }
+
+  @Test
+  public void testUncompressedInputDefaultDelimiterPosValue()
+      throws Exception {
+    Configuration conf = new Configuration();
+    String inputData = "1234567890\r\n12\r\n345";
+    Path inputFile = createInputFile(conf, inputData);
+    conf.setInt("io.file.buffer.size", 10);
+    conf.setInt(org.apache.hadoop.mapreduce.lib.input.
+        LineRecordReader.MAX_LINE_LENGTH, Integer.MAX_VALUE);
+    FileSplit split = new FileSplit(inputFile, 0, 15, (String[])null);
+    LineRecordReader reader = new LineRecordReader(conf, split,
+        null);
+    LongWritable key = new LongWritable();
+    Text value = new Text();
+    reader.next(key, value);
+    // Get first record:"1234567890"
+    assertEquals(10, value.getLength());
+    // Position should be 12 right after "1234567890\r\n"
+    assertEquals(12, reader.getPos());
+    reader.next(key, value);
+    // Get second record:"12"
+    assertEquals(2, value.getLength());
+    // Position should be 16 right after "1234567890\r\n12\r\n"
+    assertEquals(16, reader.getPos());
+    assertFalse(reader.next(key, value));
+
+    split = new FileSplit(inputFile, 15, 4, (String[])null);
+    reader = new LineRecordReader(conf, split, null);
+    // The second split dropped the first record "\n"
+    // The position should be 16 right after "1234567890\r\n12\r\n"
+    assertEquals(16, reader.getPos());
+    reader.next(key, value);
+    // Get third record:"345"
+    assertEquals(3, value.getLength());
+    // Position should be 19 right after "1234567890\r\n12\r\n345"
+    assertEquals(19, reader.getPos());
+    assertFalse(reader.next(key, value));
+    assertEquals(19, reader.getPos());
+
+    inputData = "123456789\r\r\n";
+    inputFile = createInputFile(conf, inputData);
+    split = new FileSplit(inputFile, 0, 12, (String[])null);
+    reader = new LineRecordReader(conf, split, null);
+    reader.next(key, value);
+    // Get first record:"123456789"
+    assertEquals(9, value.getLength());
+    // Position should be 10 right after "123456789\r"
+    assertEquals(10, reader.getPos());
+    reader.next(key, value);
+    // Get second record:""
+    assertEquals(0, value.getLength());
+    // Position should be 12 right after "123456789\r\r\n"
+    assertEquals(12, reader.getPos());
+    assertFalse(reader.next(key, value));
+    assertEquals(12, reader.getPos());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58d1a02b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestLineRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestLineRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestLineRecordReader.java
index 3c1f28f..dfe8b5d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestLineRecordReader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestLineRecordReader.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.mapreduce.lib.input;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
@@ -37,6 +38,8 @@ import org.apache.commons.io.Charsets;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.compress.BZip2Codec;
 import org.apache.hadoop.io.compress.CodecPool;
 import org.apache.hadoop.io.compress.Decompressor;
@@ -341,4 +344,162 @@ public class TestLineRecordReader {
       }
     }
   }
+
+  @Test
+  public void testUncompressedInputCustomDelimiterPosValue()
+      throws Exception {
+    Configuration conf = new Configuration();
+    String inputData = "1234567890ab12ab345";
+    Path inputFile = createInputFile(conf, inputData);
+    conf.setInt("io.file.buffer.size", 10);
+    conf.setInt(org.apache.hadoop.mapreduce.lib.input.
+        LineRecordReader.MAX_LINE_LENGTH, Integer.MAX_VALUE);
+    String delimiter = "ab";
+    byte[] recordDelimiterBytes = delimiter.getBytes(Charsets.UTF_8);
+    FileSplit split = new FileSplit(inputFile, 0, 15, (String[])null);
+    TaskAttemptContext context = new TaskAttemptContextImpl(conf,
+        new TaskAttemptID());
+    LineRecordReader reader = new LineRecordReader(recordDelimiterBytes);
+    reader.initialize(split, context);
+    LongWritable key;
+    Text value;
+    reader.nextKeyValue();
+    key = reader.getCurrentKey();
+    value = reader.getCurrentValue();
+    // Get first record:"1234567890"
+    assertEquals(10, value.getLength());
+    assertEquals(0, key.get());
+    reader.nextKeyValue();
+    // Get second record:"12"
+    assertEquals(2, value.getLength());
+    // Key should be 12 right after "1234567890ab"
+    assertEquals(12, key.get());
+    reader.nextKeyValue();
+    // Get third record:"345"
+    assertEquals(3, value.getLength());
+    // Key should be 16 right after "1234567890ab12ab"
+    assertEquals(16, key.get());
+    assertFalse(reader.nextKeyValue());
+    // Key should be 19 right after "1234567890ab12ab345"
+    assertEquals(19, key.get());
+
+    split = new FileSplit(inputFile, 15, 4, (String[])null);
+    reader = new LineRecordReader(recordDelimiterBytes);
+    reader.initialize(split, context);
+    // No record is in the second split because the second split dropped
+    // the first record, which was already reported by the first split.
+    assertFalse(reader.nextKeyValue());
+
+    inputData = "123456789aab";
+    inputFile = createInputFile(conf, inputData);
+    split = new FileSplit(inputFile, 0, 12, (String[])null);
+    reader = new LineRecordReader(recordDelimiterBytes);
+    reader.initialize(split, context);
+    reader.nextKeyValue();
+    key = reader.getCurrentKey();
+    value = reader.getCurrentValue();
+    // Get first record:"123456789a"
+    assertEquals(10, value.getLength());
+    assertEquals(0, key.get());
+    assertFalse(reader.nextKeyValue());
+    // Key should be 12 right after "123456789aab"
+    assertEquals(12, key.get());
+
+    inputData = "123456789a";
+    inputFile = createInputFile(conf, inputData);
+    split = new FileSplit(inputFile, 0, 10, (String[])null);
+    reader = new LineRecordReader(recordDelimiterBytes);
+    reader.initialize(split, context);
+    reader.nextKeyValue();
+    key = reader.getCurrentKey();
+    value = reader.getCurrentValue();
+    // Get first record:"123456789a"
+    assertEquals(10, value.getLength());
+    assertEquals(0, key.get());
+    assertFalse(reader.nextKeyValue());
+    // Key should be 10 right after "123456789a"
+    assertEquals(10, key.get());
+
+    inputData = "123456789ab";
+    inputFile = createInputFile(conf, inputData);
+    split = new FileSplit(inputFile, 0, 11, (String[])null);
+    reader = new LineRecordReader(recordDelimiterBytes);
+    reader.initialize(split, context);
+    reader.nextKeyValue();
+    key = reader.getCurrentKey();
+    value = reader.getCurrentValue();
+    // Get first record:"123456789"
+    assertEquals(9, value.getLength());
+    assertEquals(0, key.get());
+    assertFalse(reader.nextKeyValue());
+    // Key should be 11 right after "123456789ab"
+    assertEquals(11, key.get());
+  }
+
+  @Test
+  public void testUncompressedInputDefaultDelimiterPosValue()
+      throws Exception {
+    Configuration conf = new Configuration();
+    String inputData = "1234567890\r\n12\r\n345";
+    Path inputFile = createInputFile(conf, inputData);
+    conf.setInt("io.file.buffer.size", 10);
+    conf.setInt(org.apache.hadoop.mapreduce.lib.input.
+        LineRecordReader.MAX_LINE_LENGTH, Integer.MAX_VALUE);
+    FileSplit split = new FileSplit(inputFile, 0, 15, (String[])null);
+    TaskAttemptContext context = new TaskAttemptContextImpl(conf,
+        new TaskAttemptID());
+    LineRecordReader reader = new LineRecordReader(null);
+    reader.initialize(split, context);
+    LongWritable key;
+    Text value;
+    reader.nextKeyValue();
+    key = reader.getCurrentKey();
+    value = reader.getCurrentValue();
+    // Get first record:"1234567890"
+    assertEquals(10, value.getLength());
+    assertEquals(0, key.get());
+    reader.nextKeyValue();
+    // Get second record:"12"
+    assertEquals(2, value.getLength());
+    // Key should be 12 right after "1234567890\r\n"
+    assertEquals(12, key.get());
+    assertFalse(reader.nextKeyValue());
+    // Key should be 16 right after "1234567890\r\n12\r\n"
+    assertEquals(16, key.get());
+
+    split = new FileSplit(inputFile, 15, 4, (String[])null);
+    reader = new LineRecordReader(null);
+    reader.initialize(split, context);
+    // The second split dropped the first record "\n"
+    reader.nextKeyValue();
+    key = reader.getCurrentKey();
+    value = reader.getCurrentValue();
+    // Get third record:"345"
+    assertEquals(3, value.getLength());
+    // Key should be 16 right after "1234567890\r\n12\r\n"
+    assertEquals(16, key.get());
+    assertFalse(reader.nextKeyValue());
+    // Key should be 19 right after "1234567890\r\n12\r\n345"
+    assertEquals(19, key.get());
+
+    inputData = "123456789\r\r\n";
+    inputFile = createInputFile(conf, inputData);
+    split = new FileSplit(inputFile, 0, 12, (String[])null);
+    reader = new LineRecordReader(null);
+    reader.initialize(split, context);
+    reader.nextKeyValue();
+    key = reader.getCurrentKey();
+    value = reader.getCurrentValue();
+    // Get first record:"123456789"
+    assertEquals(9, value.getLength());
+    assertEquals(0, key.get());
+    reader.nextKeyValue();
+    // Get second record:""
+    assertEquals(0, value.getLength());
+    // Key should be 10 right after "123456789\r"
+    assertEquals(10, key.get());
+    assertFalse(reader.nextKeyValue());
+    // Key should be 12 right after "123456789\r\r\n"
+    assertEquals(12, key.get());
+  }
 }


[33/50] [abbrv] hadoop git commit: HDFS-9101. Remove deprecated NameNode.getUri() static helper method. Contributed by Mingliang Liu.

Posted by zh...@apache.org.
HDFS-9101. Remove deprecated NameNode.getUri() static helper method. Contributed by Mingliang Liu.


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

Branch: refs/heads/HDFS-7285
Commit: 66b46d0885f7049d0485e0d08b5e7af9762f0a59
Parents: 94dec5a
Author: Haohui Mai <wh...@apache.org>
Authored: Fri Sep 18 18:23:53 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Sep 18 18:23:53 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/66b46d08/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 b905d42..65cde45 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -935,6 +935,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-5802. NameNode does not check for inode type before traversing down a
     path. (Xiao Chen via Yongjun Zhang)
 
+    HDFS-9101. Remove deprecated NameNode.getUri() static helper method.
+    (Mingliang Liu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66b46d08/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index 0431fee..565555e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -471,13 +471,6 @@ public class NameNode implements NameNodeStatusMXBean {
     return DFSUtilClient.getNNAddress(addr);
   }
 
-  @Deprecated
-  /**
-   * @deprecated Use {@link DFSUtilClient#getNNUri(InetSocketAddress)} instead.
-   */
-  public static URI getUri(InetSocketAddress namenode) {
-    return DFSUtilClient.getNNUri(namenode);
-  }
   //
   // Common NameNode methods implementation for the active name-node role.
   //


[43/50] [abbrv] hadoop git commit: HDFS-9111. Move hdfs-client protobuf convert methods from PBHelper to PBHelperClient. Contributed by Mingliang Liu.

Posted by zh...@apache.org.
HDFS-9111. Move hdfs-client protobuf convert methods from PBHelper to PBHelperClient. Contributed by Mingliang Liu.


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

Branch: refs/heads/HDFS-7285
Commit: 06022b8fdc40e50eaac63758246353058e8cfa6d
Parents: 8e01b0d
Author: Haohui Mai <wh...@apache.org>
Authored: Mon Sep 21 18:53:13 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Mon Sep 21 18:53:13 2015 -0700

----------------------------------------------------------------------
 .../hdfs/protocol/HdfsLocatedFileStatus.java    |   83 +
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  | 1975 ++++++++++++++++-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |    3 +
 .../hdfs/protocol/HdfsLocatedFileStatus.java    |   83 -
 .../hdfs/protocol/datatransfer/Receiver.java    |   33 +-
 ...tDatanodeProtocolServerSideTranslatorPB.java |    6 +-
 .../protocolPB/ClientNamenodeProtocolPB.java    |    4 +-
 ...tNamenodeProtocolServerSideTranslatorPB.java |  128 +-
 .../ClientNamenodeProtocolTranslatorPB.java     |  122 +-
 .../DatanodeProtocolClientSideTranslatorPB.java |   10 +-
 .../DatanodeProtocolServerSideTranslatorPB.java |   12 +-
 ...rDatanodeProtocolServerSideTranslatorPB.java |    2 +-
 .../protocolPB/JournalProtocolTranslatorPB.java |    2 +-
 .../NamenodeProtocolServerSideTranslatorPB.java |    3 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 2038 +-----------------
 .../QJournalProtocolTranslatorPB.java           |    3 +-
 .../hdfs/server/namenode/CacheManager.java      |    4 +-
 .../server/namenode/EncryptionZoneManager.java  |    4 +-
 .../hdfs/server/namenode/FSDirXAttrOp.java      |    3 +-
 .../hdfs/server/namenode/FSDirectory.java       |    7 +-
 .../hdfs/server/namenode/FSEditLogOp.java       |   20 +-
 .../server/namenode/FSImageFormatPBINode.java   |    7 +-
 .../snapshot/FSImageFormatPBSnapshot.java       |    6 +-
 .../hadoop/hdfs/protocolPB/TestPBHelper.java    |   32 +-
 24 files changed, 2289 insertions(+), 2301 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
new file mode 100644
index 0000000..23e8f57
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import java.net.URI;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+
+/** 
+ * Interface that represents the over the wire information
+ * including block locations for a file.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class HdfsLocatedFileStatus extends HdfsFileStatus {
+  private final LocatedBlocks locations;
+
+  /**
+   * Constructor
+   * 
+   * @param length size
+   * @param isdir if this is directory
+   * @param block_replication the file's replication factor
+   * @param blocksize the file's block size
+   * @param modification_time most recent modification time
+   * @param access_time most recent access time
+   * @param permission permission
+   * @param owner owner
+   * @param group group
+   * @param symlink symbolic link
+   * @param path local path name in java UTF8 format 
+   * @param fileId the file id
+   * @param locations block locations
+   * @param feInfo file encryption info
+   */
+  public HdfsLocatedFileStatus(long length, boolean isdir,
+      int block_replication, long blocksize, long modification_time,
+      long access_time, FsPermission permission, String owner, String group,
+      byte[] symlink, byte[] path, long fileId, LocatedBlocks locations,
+      int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy) {
+    super(length, isdir, block_replication, blocksize, modification_time,
+        access_time, permission, owner, group, symlink, path, fileId,
+        childrenNum, feInfo, storagePolicy);
+    this.locations = locations;
+  }
+
+  public LocatedBlocks getBlockLocations() {
+    return locations;
+  }
+
+  public final LocatedFileStatus makeQualifiedLocated(URI defaultUri,
+      Path path) {
+    return new LocatedFileStatus(getLen(), isDir(), getReplication(),
+        getBlockSize(), getModificationTime(),
+        getAccessTime(),
+        getPermission(), getOwner(), getGroup(),
+        isSymlink() ? new Path(getSymlink()) : null,
+        (getFullPath(path)).makeQualified(
+            defaultUri, null), // fully-qualify path
+        DFSUtilClient.locatedBlocks2Locations(getBlockLocations()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index 98de2e9..ae0a3f6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -17,46 +17,173 @@
  */
 package org.apache.hadoop.hdfs.protocolPB;
 
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.List;
+
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+import com.google.common.primitives.Shorts;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.CodedInputStream;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.hadoop.crypto.CipherOption;
 import org.apache.hadoop.crypto.CipherSuite;
+import org.apache.hadoop.crypto.CryptoProtocolVersion;
+import org.apache.hadoop.fs.CacheFlag;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclEntryScope;
+import org.apache.hadoop.fs.permission.AclEntryType;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.inotify.Event;
+import org.apache.hadoop.hdfs.inotify.EventBatch;
+import org.apache.hadoop.hdfs.inotify.EventBatchList;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
+import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolStats;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
+import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryScopeProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryTypeProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.FsActionProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclStatusProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveStatsProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheFlagProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolStatsProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateFlagProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeReportTypeProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeStorageReportProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeActionProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmIdProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.EncryptionZoneProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ContentSummaryProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CorruptFileBlocksProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CryptoProtocolVersionProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DataEncryptionKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfosProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeLocalInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto.StorageState;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsPermissionProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsServerDefaultsProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto.FileType;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlocksProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RollingUpgradeStatusProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryListingProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryStatusProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypesProto;
+import org.apache.hadoop.hdfs.protocol.proto.InotifyProtos;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrProto.XAttrNamespaceProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrSetFlagProto;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
 import org.apache.hadoop.hdfs.util.ExactSizeInputStream;
+import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
 
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.List;
-
 /**
- * Utilities for converting protobuf classes to and from implementation classes
- * and other helper utilities to help in dealing with protobuf.
+ * Utilities for converting protobuf classes to and from hdfs-client side
+ * implementation classes and other helper utilities to help in dealing with
+ * protobuf.
  *
  * Note that when converting from an internal type to protobuf type, the
  * converter never return null for protobuf type. The check for internal type
  * being null must be done before calling the convert() method.
  */
 public class PBHelperClient {
+  private static final XAttr.NameSpace[] XATTR_NAMESPACE_VALUES =
+      XAttr.NameSpace.values();
+  private static final AclEntryType[] ACL_ENTRY_TYPE_VALUES =
+      AclEntryType.values();
+  private static final AclEntryScope[] ACL_ENTRY_SCOPE_VALUES =
+      AclEntryScope.values();
+  private static final FsAction[] FSACTION_VALUES =
+      FsAction.values();
+
   private PBHelperClient() {
     /** Hidden constructor */
   }
@@ -253,7 +380,7 @@ public class PBHelperClient {
     final List<StorageTypeProto> protos = new ArrayList<>(
       types.length);
     for (int i = startIdx; i < types.length; ++i) {
-      protos.add(PBHelperClient.convertStorageType(types[i]));
+      protos.add(convertStorageType(types[i]));
     }
     return protos;
   }
@@ -369,4 +496,1834 @@ public class PBHelperClient {
     }
     return null;
   }
+
+  public static LocatedBlock convert(LocatedBlockProto proto) {
+    if (proto == null) return null;
+    List<DatanodeInfoProto> locs = proto.getLocsList();
+    DatanodeInfo[] targets = new DatanodeInfo[locs.size()];
+    for (int i = 0; i < locs.size(); i++) {
+      targets[i] = convert(locs.get(i));
+    }
+
+    final StorageType[] storageTypes = convertStorageTypes(
+        proto.getStorageTypesList(), locs.size());
+
+    final int storageIDsCount = proto.getStorageIDsCount();
+    final String[] storageIDs;
+    if (storageIDsCount == 0) {
+      storageIDs = null;
+    } else {
+      Preconditions.checkState(storageIDsCount == locs.size());
+      storageIDs = proto.getStorageIDsList().toArray(new String[storageIDsCount]);
+    }
+
+    // Set values from the isCached list, re-using references from loc
+    List<DatanodeInfo> cachedLocs = new ArrayList<>(locs.size());
+    List<Boolean> isCachedList = proto.getIsCachedList();
+    for (int i=0; i<isCachedList.size(); i++) {
+      if (isCachedList.get(i)) {
+        cachedLocs.add(targets[i]);
+      }
+    }
+
+    LocatedBlock lb = new LocatedBlock(convert(proto.getB()), targets,
+        storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(),
+        cachedLocs.toArray(new DatanodeInfo[0]));
+    lb.setBlockToken(convert(proto.getBlockToken()));
+
+    return lb;
+  }
+
+  static public DatanodeInfo convert(DatanodeInfoProto di) {
+    if (di == null) return null;
+    return new DatanodeInfo(
+        convert(di.getId()),
+        di.hasLocation() ? di.getLocation() : null,
+        di.getCapacity(),  di.getDfsUsed(),  di.getRemaining(),
+        di.getBlockPoolUsed(), di.getCacheCapacity(), di.getCacheUsed(),
+        di.getLastUpdate(), di.getLastUpdateMonotonic(),
+        di.getXceiverCount(), convert(di.getAdminState()),
+        di.hasUpgradeDomain() ? di.getUpgradeDomain() : null);
+  }
+
+  public static StorageType[] convertStorageTypes(
+      List<StorageTypeProto> storageTypesList, int expectedSize) {
+    final StorageType[] storageTypes = new StorageType[expectedSize];
+    if (storageTypesList.size() != expectedSize) {
+     // missing storage types
+      Preconditions.checkState(storageTypesList.isEmpty());
+      Arrays.fill(storageTypes, StorageType.DEFAULT);
+    } else {
+      for (int i = 0; i < storageTypes.length; ++i) {
+        storageTypes[i] = convertStorageType(storageTypesList.get(i));
+      }
+    }
+    return storageTypes;
+  }
+
+  public static Token<BlockTokenIdentifier> convert(
+      TokenProto blockToken) {
+    return new Token<>(blockToken.getIdentifier()
+        .toByteArray(), blockToken.getPassword().toByteArray(), new Text(
+        blockToken.getKind()), new Text(blockToken.getService()));
+  }
+
+  // DatanodeId
+  public static DatanodeID convert(DatanodeIDProto dn) {
+    return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getDatanodeUuid(),
+        dn.getXferPort(), dn.getInfoPort(), dn.hasInfoSecurePort() ? dn
+        .getInfoSecurePort() : 0, dn.getIpcPort());
+  }
+
+  public static AdminStates convert(AdminState adminState) {
+    switch(adminState) {
+    case DECOMMISSION_INPROGRESS:
+      return AdminStates.DECOMMISSION_INPROGRESS;
+    case DECOMMISSIONED:
+      return AdminStates.DECOMMISSIONED;
+    case NORMAL:
+    default:
+      return AdminStates.NORMAL;
+    }
+  }
+
+  // LocatedBlocks
+  public static LocatedBlocks convert(LocatedBlocksProto lb) {
+    return new LocatedBlocks(
+        lb.getFileLength(), lb.getUnderConstruction(),
+        convertLocatedBlock(lb.getBlocksList()),
+        lb.hasLastBlock() ? convert(lb.getLastBlock()) : null,
+        lb.getIsLastBlockComplete(),
+        lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) :
+            null);
+  }
+
+  public static BlockStoragePolicy[] convertStoragePolicies(
+      List<BlockStoragePolicyProto> policyProtos) {
+    if (policyProtos == null || policyProtos.size() == 0) {
+      return new BlockStoragePolicy[0];
+    }
+    BlockStoragePolicy[] policies = new BlockStoragePolicy[policyProtos.size()];
+    int i = 0;
+    for (BlockStoragePolicyProto proto : policyProtos) {
+      policies[i++] = convert(proto);
+    }
+    return policies;
+  }
+
+  public static EventBatchList convert(GetEditsFromTxidResponseProto resp) throws
+    IOException {
+    final InotifyProtos.EventsListProto list = resp.getEventsList();
+    final long firstTxid = list.getFirstTxid();
+    final long lastTxid = list.getLastTxid();
+
+    List<EventBatch> batches = Lists.newArrayList();
+    if (list.getEventsList().size() > 0) {
+      throw new IOException("Can't handle old inotify server response.");
+    }
+    for (InotifyProtos.EventBatchProto bp : list.getBatchList()) {
+      long txid = bp.getTxid();
+      if ((txid != -1) && ((txid < firstTxid) || (txid > lastTxid))) {
+        throw new IOException("Error converting TxidResponseProto: got a " +
+            "transaction id " + txid + " that was outside the range of [" +
+            firstTxid + ", " + lastTxid + "].");
+      }
+      List<Event> events = Lists.newArrayList();
+      for (InotifyProtos.EventProto p : bp.getEventsList()) {
+        switch (p.getType()) {
+          case EVENT_CLOSE:
+            InotifyProtos.CloseEventProto close =
+                InotifyProtos.CloseEventProto.parseFrom(p.getContents());
+            events.add(new Event.CloseEvent(close.getPath(),
+                close.getFileSize(), close.getTimestamp()));
+            break;
+          case EVENT_CREATE:
+            InotifyProtos.CreateEventProto create =
+                InotifyProtos.CreateEventProto.parseFrom(p.getContents());
+            events.add(new Event.CreateEvent.Builder()
+                .iNodeType(createTypeConvert(create.getType()))
+                .path(create.getPath())
+                .ctime(create.getCtime())
+                .ownerName(create.getOwnerName())
+                .groupName(create.getGroupName())
+                .perms(convert(create.getPerms()))
+                .replication(create.getReplication())
+                .symlinkTarget(create.getSymlinkTarget().isEmpty() ? null :
+                    create.getSymlinkTarget())
+                .defaultBlockSize(create.getDefaultBlockSize())
+                .overwrite(create.getOverwrite()).build());
+            break;
+          case EVENT_METADATA:
+            InotifyProtos.MetadataUpdateEventProto meta =
+                InotifyProtos.MetadataUpdateEventProto.parseFrom(p.getContents());
+            events.add(new Event.MetadataUpdateEvent.Builder()
+                .path(meta.getPath())
+                .metadataType(metadataUpdateTypeConvert(meta.getType()))
+                .mtime(meta.getMtime())
+                .atime(meta.getAtime())
+                .replication(meta.getReplication())
+                .ownerName(
+                    meta.getOwnerName().isEmpty() ? null : meta.getOwnerName())
+                .groupName(
+                    meta.getGroupName().isEmpty() ? null : meta.getGroupName())
+                .perms(meta.hasPerms() ? convert(meta.getPerms()) : null)
+                .acls(meta.getAclsList().isEmpty() ? null : convertAclEntry(
+                    meta.getAclsList()))
+                .xAttrs(meta.getXAttrsList().isEmpty() ? null : convertXAttrs(
+                    meta.getXAttrsList()))
+                .xAttrsRemoved(meta.getXAttrsRemoved())
+                .build());
+            break;
+          case EVENT_RENAME:
+            InotifyProtos.RenameEventProto rename =
+                InotifyProtos.RenameEventProto.parseFrom(p.getContents());
+            events.add(new Event.RenameEvent.Builder()
+                  .srcPath(rename.getSrcPath())
+                  .dstPath(rename.getDestPath())
+                  .timestamp(rename.getTimestamp())
+                  .build());
+            break;
+          case EVENT_APPEND:
+            InotifyProtos.AppendEventProto append =
+                InotifyProtos.AppendEventProto.parseFrom(p.getContents());
+            events.add(new Event.AppendEvent.Builder().path(append.getPath())
+                .newBlock(append.hasNewBlock() && append.getNewBlock())
+                .build());
+            break;
+          case EVENT_UNLINK:
+            InotifyProtos.UnlinkEventProto unlink =
+                InotifyProtos.UnlinkEventProto.parseFrom(p.getContents());
+            events.add(new Event.UnlinkEvent.Builder()
+                  .path(unlink.getPath())
+                  .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());
+        }
+      }
+      batches.add(new EventBatch(txid, events.toArray(new Event[0])));
+    }
+    return new EventBatchList(batches, resp.getEventsList().getFirstTxid(),
+        resp.getEventsList().getLastTxid(), resp.getEventsList().getSyncTxid());
+  }
+
+  // Located Block Arrays and Lists
+  public static LocatedBlockProto[] convertLocatedBlock(LocatedBlock[] lb) {
+    if (lb == null) return null;
+    return convertLocatedBlock2(Arrays.asList(lb)).toArray(
+        new LocatedBlockProto[lb.length]);
+  }
+
+  public static List<LocatedBlockProto> convertLocatedBlock2(List<LocatedBlock> lb) {
+    if (lb == null) return null;
+    final int len = lb.size();
+    List<LocatedBlockProto> result = new ArrayList<>(len);
+    for (int i = 0; i < len; ++i) {
+      result.add(convert(lb.get(i)));
+    }
+    return result;
+  }
+
+  public static LocatedBlockProto convert(LocatedBlock b) {
+    if (b == null) return null;
+    Builder builder = LocatedBlockProto.newBuilder();
+    DatanodeInfo[] locs = b.getLocations();
+    List<DatanodeInfo> cachedLocs =
+        Lists.newLinkedList(Arrays.asList(b.getCachedLocations()));
+    for (int i = 0; i < locs.length; i++) {
+      DatanodeInfo loc = locs[i];
+      builder.addLocs(i, convert(loc));
+      boolean locIsCached = cachedLocs.contains(loc);
+      builder.addIsCached(locIsCached);
+      if (locIsCached) {
+        cachedLocs.remove(loc);
+      }
+    }
+    Preconditions.checkArgument(cachedLocs.size() == 0,
+        "Found additional cached replica locations that are not in the set of"
+        + " storage-backed locations!");
+
+    StorageType[] storageTypes = b.getStorageTypes();
+    if (storageTypes != null) {
+      for (int i = 0; i < storageTypes.length; ++i) {
+        builder.addStorageTypes(convertStorageType(storageTypes[i]));
+      }
+    }
+    final String[] storageIDs = b.getStorageIDs();
+    if (storageIDs != null) {
+      builder.addAllStorageIDs(Arrays.asList(storageIDs));
+    }
+
+    return builder.setB(convert(b.getBlock()))
+        .setBlockToken(convert(b.getBlockToken()))
+        .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build();
+  }
+
+  public static BlockStoragePolicy convert(BlockStoragePolicyProto proto) {
+    List<StorageTypeProto> cList = proto.getCreationPolicy()
+        .getStorageTypesList();
+    StorageType[] creationTypes = convertStorageTypes(cList, cList.size());
+    List<StorageTypeProto> cfList = proto.hasCreationFallbackPolicy() ? proto
+        .getCreationFallbackPolicy().getStorageTypesList() : null;
+    StorageType[] creationFallbackTypes = cfList == null ? StorageType
+        .EMPTY_ARRAY : convertStorageTypes(cfList, cfList.size());
+    List<StorageTypeProto> rfList = proto.hasReplicationFallbackPolicy() ?
+        proto.getReplicationFallbackPolicy().getStorageTypesList() : null;
+    StorageType[] replicationFallbackTypes = rfList == null ? StorageType
+        .EMPTY_ARRAY : convertStorageTypes(rfList, rfList.size());
+    return new BlockStoragePolicy((byte) proto.getPolicyId(), proto.getName(),
+        creationTypes, creationFallbackTypes, replicationFallbackTypes);
+  }
+
+  public static FsActionProto convert(FsAction v) {
+    return FsActionProto.valueOf(v != null ? v.ordinal() : 0);
+  }
+
+  public static XAttrProto convertXAttrProto(XAttr a) {
+    XAttrProto.Builder builder = XAttrProto.newBuilder();
+    builder.setNamespace(convert(a.getNameSpace()));
+    if (a.getName() != null) {
+      builder.setName(a.getName());
+    }
+    if (a.getValue() != null) {
+      builder.setValue(getByteString(a.getValue()));
+    }
+    return builder.build();
+  }
+
+  public static List<XAttr> convert(ListXAttrsResponseProto a) {
+    final List<XAttrProto> xAttrs = a.getXAttrsList();
+    return convertXAttrs(xAttrs);
+  }
+
+  public static List<XAttr> convert(GetXAttrsResponseProto a) {
+    List<XAttrProto> xAttrs = a.getXAttrsList();
+    return convertXAttrs(xAttrs);
+  }
+
+  public static List<XAttr> convertXAttrs(List<XAttrProto> xAttrSpec) {
+    ArrayList<XAttr> xAttrs = Lists.newArrayListWithCapacity(xAttrSpec.size());
+    for (XAttrProto a : xAttrSpec) {
+      XAttr.Builder builder = new XAttr.Builder();
+      builder.setNameSpace(convert(a.getNamespace()));
+      if (a.hasName()) {
+        builder.setName(a.getName());
+      }
+      if (a.hasValue()) {
+        builder.setValue(a.getValue().toByteArray());
+      }
+      xAttrs.add(builder.build());
+    }
+    return xAttrs;
+  }
+
+  static XAttrNamespaceProto convert(XAttr.NameSpace v) {
+    return XAttrNamespaceProto.valueOf(v.ordinal());
+  }
+
+  static XAttr.NameSpace convert(XAttrNamespaceProto v) {
+    return castEnum(v, XATTR_NAMESPACE_VALUES);
+  }
+
+  static <T extends Enum<T>, U extends Enum<U>> U castEnum(T from, U[] to) {
+    return to[from.ordinal()];
+  }
+
+  static InotifyProtos.MetadataUpdateType metadataUpdateTypeConvert(
+      Event.MetadataUpdateEvent.MetadataType type) {
+    switch (type) {
+    case TIMES:
+      return InotifyProtos.MetadataUpdateType.META_TYPE_TIMES;
+    case REPLICATION:
+      return InotifyProtos.MetadataUpdateType.META_TYPE_REPLICATION;
+    case OWNER:
+      return InotifyProtos.MetadataUpdateType.META_TYPE_OWNER;
+    case PERMS:
+      return InotifyProtos.MetadataUpdateType.META_TYPE_PERMS;
+    case ACLS:
+      return InotifyProtos.MetadataUpdateType.META_TYPE_ACLS;
+    case XATTRS:
+      return InotifyProtos.MetadataUpdateType.META_TYPE_XATTRS;
+    default:
+      return null;
+    }
+  }
+
+  private static Event.MetadataUpdateEvent.MetadataType metadataUpdateTypeConvert(
+      InotifyProtos.MetadataUpdateType type) {
+    switch (type) {
+    case META_TYPE_TIMES:
+      return Event.MetadataUpdateEvent.MetadataType.TIMES;
+    case META_TYPE_REPLICATION:
+      return Event.MetadataUpdateEvent.MetadataType.REPLICATION;
+    case META_TYPE_OWNER:
+      return Event.MetadataUpdateEvent.MetadataType.OWNER;
+    case META_TYPE_PERMS:
+      return Event.MetadataUpdateEvent.MetadataType.PERMS;
+    case META_TYPE_ACLS:
+      return Event.MetadataUpdateEvent.MetadataType.ACLS;
+    case META_TYPE_XATTRS:
+      return Event.MetadataUpdateEvent.MetadataType.XATTRS;
+    default:
+      return null;
+    }
+  }
+
+  static InotifyProtos.INodeType createTypeConvert(Event.CreateEvent.INodeType
+                                                       type) {
+    switch (type) {
+    case DIRECTORY:
+      return InotifyProtos.INodeType.I_TYPE_DIRECTORY;
+    case FILE:
+      return InotifyProtos.INodeType.I_TYPE_FILE;
+    case SYMLINK:
+      return InotifyProtos.INodeType.I_TYPE_SYMLINK;
+    default:
+      return null;
+    }
+  }
+
+  public static List<LocatedBlock> convertLocatedBlock(
+      List<LocatedBlockProto> lb) {
+    if (lb == null) return null;
+    final int len = lb.size();
+    List<LocatedBlock> result = new ArrayList<>(len);
+    for (int i = 0; i < len; ++i) {
+      result.add(convert(lb.get(i)));
+    }
+    return result;
+  }
+
+  public static List<AclEntry> convertAclEntry(List<AclEntryProto> aclSpec) {
+    ArrayList<AclEntry> r = Lists.newArrayListWithCapacity(aclSpec.size());
+    for (AclEntryProto e : aclSpec) {
+      AclEntry.Builder builder = new AclEntry.Builder();
+      builder.setType(convert(e.getType()));
+      builder.setScope(convert(e.getScope()));
+      builder.setPermission(convert(e.getPermissions()));
+      if (e.hasName()) {
+        builder.setName(e.getName());
+      }
+      r.add(builder.build());
+    }
+    return r;
+  }
+
+  static AclEntryScopeProto convert(AclEntryScope v) {
+    return AclEntryScopeProto.valueOf(v.ordinal());
+  }
+
+  private static AclEntryScope convert(AclEntryScopeProto v) {
+    return castEnum(v, ACL_ENTRY_SCOPE_VALUES);
+  }
+
+  static AclEntryTypeProto convert(AclEntryType e) {
+    return AclEntryTypeProto.valueOf(e.ordinal());
+  }
+
+  private static AclEntryType convert(AclEntryTypeProto v) {
+    return castEnum(v, ACL_ENTRY_TYPE_VALUES);
+  }
+
+  public static FsAction convert(FsActionProto v) {
+    return castEnum(v, FSACTION_VALUES);
+  }
+
+  public static FsPermission convert(FsPermissionProto p) {
+    return new FsPermissionExtension((short)p.getPerm());
+  }
+
+  private static Event.CreateEvent.INodeType createTypeConvert(
+      InotifyProtos.INodeType type) {
+    switch (type) {
+    case I_TYPE_DIRECTORY:
+      return Event.CreateEvent.INodeType.DIRECTORY;
+    case I_TYPE_FILE:
+      return Event.CreateEvent.INodeType.FILE;
+    case I_TYPE_SYMLINK:
+      return Event.CreateEvent.INodeType.SYMLINK;
+    default:
+      return null;
+    }
+  }
+
+  public static HdfsProtos.FileEncryptionInfoProto convert(
+      FileEncryptionInfo info) {
+    if (info == null) {
+      return null;
+    }
+    return HdfsProtos.FileEncryptionInfoProto.newBuilder()
+        .setSuite(convert(info.getCipherSuite()))
+        .setCryptoProtocolVersion(convert(info.getCryptoProtocolVersion()))
+        .setKey(getByteString(info.getEncryptedDataEncryptionKey()))
+        .setIv(getByteString(info.getIV()))
+        .setEzKeyVersionName(info.getEzKeyVersionName())
+        .setKeyName(info.getKeyName())
+        .build();
+  }
+
+  public static CryptoProtocolVersionProto convert(CryptoProtocolVersion
+      version) {
+    switch(version) {
+    case UNKNOWN:
+      return CryptoProtocolVersionProto.UNKNOWN_PROTOCOL_VERSION;
+    case ENCRYPTION_ZONES:
+      return CryptoProtocolVersionProto.ENCRYPTION_ZONES;
+    default:
+      return null;
+    }
+  }
+
+  public static FileEncryptionInfo convert(
+      HdfsProtos.FileEncryptionInfoProto proto) {
+    if (proto == null) {
+      return null;
+    }
+    CipherSuite suite = convert(proto.getSuite());
+    CryptoProtocolVersion version = convert(proto.getCryptoProtocolVersion());
+    byte[] key = proto.getKey().toByteArray();
+    byte[] iv = proto.getIv().toByteArray();
+    String ezKeyVersionName = proto.getEzKeyVersionName();
+    String keyName = proto.getKeyName();
+    return new FileEncryptionInfo(suite, version, key, iv, keyName,
+        ezKeyVersionName);
+  }
+
+  public static CryptoProtocolVersion convert(CryptoProtocolVersionProto
+      proto) {
+    switch(proto) {
+    case ENCRYPTION_ZONES:
+      return CryptoProtocolVersion.ENCRYPTION_ZONES;
+    default:
+      // Set to UNKNOWN and stash the unknown enum value
+      CryptoProtocolVersion version = CryptoProtocolVersion.UNKNOWN;
+      version.setUnknownValue(proto.getNumber());
+      return version;
+    }
+  }
+
+  public static List<XAttrProto> convertXAttrProto(
+      List<XAttr> xAttrSpec) {
+    if (xAttrSpec == null) {
+      return Lists.newArrayListWithCapacity(0);
+    }
+    ArrayList<XAttrProto> xAttrs = Lists.newArrayListWithCapacity(
+        xAttrSpec.size());
+    for (XAttr a : xAttrSpec) {
+      XAttrProto.Builder builder = XAttrProto.newBuilder();
+      builder.setNamespace(convert(a.getNameSpace()));
+      if (a.getName() != null) {
+        builder.setName(a.getName());
+      }
+      if (a.getValue() != null) {
+        builder.setValue(getByteString(a.getValue()));
+      }
+      xAttrs.add(builder.build());
+    }
+    return xAttrs;
+  }
+
+  /**
+   * The flag field in PB is a bitmask whose values are the same a the
+   * emum values of XAttrSetFlag
+   */
+  public static int convert(EnumSet<XAttrSetFlag> flag) {
+    int value = 0;
+    if (flag.contains(XAttrSetFlag.CREATE)) {
+      value |= XAttrSetFlagProto.XATTR_CREATE.getNumber();
+    }
+    if (flag.contains(XAttrSetFlag.REPLACE)) {
+      value |= XAttrSetFlagProto.XATTR_REPLACE.getNumber();
+    }
+    return value;
+  }
+
+  public static EncryptionZone convert(EncryptionZoneProto proto) {
+    return new EncryptionZone(proto.getId(), proto.getPath(),
+        convert(proto.getSuite()), convert(proto.getCryptoProtocolVersion()),
+        proto.getKeyName());
+  }
+
+  public static AclStatus convert(GetAclStatusResponseProto e) {
+    AclStatusProto r = e.getResult();
+    AclStatus.Builder builder = new AclStatus.Builder();
+    builder.owner(r.getOwner()).group(r.getGroup()).stickyBit(r.getSticky())
+        .addEntries(convertAclEntry(r.getEntriesList()));
+    if (r.hasPermission()) {
+      builder.setPermission(convert(r.getPermission()));
+    }
+    return builder.build();
+  }
+
+  public static List<AclEntryProto> convertAclEntryProto(
+      List<AclEntry> aclSpec) {
+    ArrayList<AclEntryProto> r = Lists.newArrayListWithCapacity(aclSpec.size());
+    for (AclEntry e : aclSpec) {
+      AclEntryProto.Builder builder = AclEntryProto.newBuilder();
+      builder.setType(convert(e.getType()));
+      builder.setScope(convert(e.getScope()));
+      builder.setPermissions(convert(e.getPermission()));
+      if (e.getName() != null) {
+        builder.setName(e.getName());
+      }
+      r.add(builder.build());
+    }
+    return r;
+  }
+
+  public static CachePoolEntry convert(CachePoolEntryProto proto) {
+    CachePoolInfo info = convert(proto.getInfo());
+    CachePoolStats stats = convert(proto.getStats());
+    return new CachePoolEntry(info, stats);
+  }
+
+  public static CachePoolInfo convert (CachePoolInfoProto proto) {
+    // Pool name is a required field, the rest are optional
+    String poolName = checkNotNull(proto.getPoolName());
+    CachePoolInfo info = new CachePoolInfo(poolName);
+    if (proto.hasOwnerName()) {
+        info.setOwnerName(proto.getOwnerName());
+    }
+    if (proto.hasGroupName()) {
+      info.setGroupName(proto.getGroupName());
+    }
+    if (proto.hasMode()) {
+      info.setMode(new FsPermission((short)proto.getMode()));
+    }
+    if (proto.hasLimit())  {
+      info.setLimit(proto.getLimit());
+    }
+    if (proto.hasMaxRelativeExpiry()) {
+      info.setMaxRelativeExpiryMs(proto.getMaxRelativeExpiry());
+    }
+    return info;
+  }
+
+  public static CachePoolStats convert (CachePoolStatsProto proto) {
+    CachePoolStats.Builder builder = new CachePoolStats.Builder();
+    builder.setBytesNeeded(proto.getBytesNeeded());
+    builder.setBytesCached(proto.getBytesCached());
+    builder.setBytesOverlimit(proto.getBytesOverlimit());
+    builder.setFilesNeeded(proto.getFilesNeeded());
+    builder.setFilesCached(proto.getFilesCached());
+    return builder.build();
+  }
+
+  public static CachePoolInfoProto convert(CachePoolInfo info) {
+    CachePoolInfoProto.Builder builder = CachePoolInfoProto.newBuilder();
+    builder.setPoolName(info.getPoolName());
+    if (info.getOwnerName() != null) {
+      builder.setOwnerName(info.getOwnerName());
+    }
+    if (info.getGroupName() != null) {
+      builder.setGroupName(info.getGroupName());
+    }
+    if (info.getMode() != null) {
+      builder.setMode(info.getMode().toShort());
+    }
+    if (info.getLimit() != null) {
+      builder.setLimit(info.getLimit());
+    }
+    if (info.getMaxRelativeExpiryMs() != null) {
+      builder.setMaxRelativeExpiry(info.getMaxRelativeExpiryMs());
+    }
+    return builder.build();
+  }
+
+  public static CacheDirectiveInfoProto convert
+      (CacheDirectiveInfo info) {
+    CacheDirectiveInfoProto.Builder builder =
+        CacheDirectiveInfoProto.newBuilder();
+    if (info.getId() != null) {
+      builder.setId(info.getId());
+    }
+    if (info.getPath() != null) {
+      builder.setPath(info.getPath().toUri().getPath());
+    }
+    if (info.getReplication() != null) {
+      builder.setReplication(info.getReplication());
+    }
+    if (info.getPool() != null) {
+      builder.setPool(info.getPool());
+    }
+    if (info.getExpiration() != null) {
+      builder.setExpiration(convert(info.getExpiration()));
+    }
+    return builder.build();
+  }
+
+  public static CacheDirectiveInfoExpirationProto convert(
+      CacheDirectiveInfo.Expiration expiration) {
+    return CacheDirectiveInfoExpirationProto.newBuilder()
+        .setIsRelative(expiration.isRelative())
+        .setMillis(expiration.getMillis())
+        .build();
+  }
+
+  public static CacheDirectiveEntry convert(CacheDirectiveEntryProto proto) {
+    CacheDirectiveInfo info = convert(proto.getInfo());
+    CacheDirectiveStats stats = convert(proto.getStats());
+    return new CacheDirectiveEntry(info, stats);
+  }
+
+  public static CacheDirectiveStats convert(CacheDirectiveStatsProto proto) {
+    CacheDirectiveStats.Builder builder = new CacheDirectiveStats.Builder();
+    builder.setBytesNeeded(proto.getBytesNeeded());
+    builder.setBytesCached(proto.getBytesCached());
+    builder.setFilesNeeded(proto.getFilesNeeded());
+    builder.setFilesCached(proto.getFilesCached());
+    builder.setHasExpired(proto.getHasExpired());
+    return builder.build();
+  }
+
+  public static CacheDirectiveInfo convert
+      (CacheDirectiveInfoProto proto) {
+    CacheDirectiveInfo.Builder builder =
+        new CacheDirectiveInfo.Builder();
+    if (proto.hasId()) {
+      builder.setId(proto.getId());
+    }
+    if (proto.hasPath()) {
+      builder.setPath(new Path(proto.getPath()));
+    }
+    if (proto.hasReplication()) {
+      builder.setReplication(Shorts.checkedCast(
+          proto.getReplication()));
+    }
+    if (proto.hasPool()) {
+      builder.setPool(proto.getPool());
+    }
+    if (proto.hasExpiration()) {
+      builder.setExpiration(convert(proto.getExpiration()));
+    }
+    return builder.build();
+  }
+
+  public static CacheDirectiveInfo.Expiration convert(
+      CacheDirectiveInfoExpirationProto proto) {
+    if (proto.getIsRelative()) {
+      return CacheDirectiveInfo.Expiration.newRelative(proto.getMillis());
+    }
+    return CacheDirectiveInfo.Expiration.newAbsolute(proto.getMillis());
+  }
+
+  public static int convertCacheFlags(EnumSet<CacheFlag> flags) {
+    int value = 0;
+    if (flags.contains(CacheFlag.FORCE)) {
+      value |= CacheFlagProto.FORCE.getNumber();
+    }
+    return value;
+  }
+
+  public static SnapshotDiffReport convert(SnapshotDiffReportProto reportProto) {
+    if (reportProto == null) {
+      return null;
+    }
+    String snapshotDir = reportProto.getSnapshotRoot();
+    String fromSnapshot = reportProto.getFromSnapshot();
+    String toSnapshot = reportProto.getToSnapshot();
+    List<SnapshotDiffReportEntryProto> list = reportProto
+        .getDiffReportEntriesList();
+    List<DiffReportEntry> entries = new ArrayList<>();
+    for (SnapshotDiffReportEntryProto entryProto : list) {
+      DiffReportEntry entry = convert(entryProto);
+      if (entry != null)
+        entries.add(entry);
+    }
+    return new SnapshotDiffReport(snapshotDir, fromSnapshot, toSnapshot,
+        entries);
+  }
+
+  public static DiffReportEntry convert(SnapshotDiffReportEntryProto entry) {
+    if (entry == null) {
+      return null;
+    }
+    DiffType type = DiffType.getTypeFromLabel(entry
+        .getModificationLabel());
+    return type == null ? null : new DiffReportEntry(type, entry.getFullpath()
+        .toByteArray(), entry.hasTargetPath() ? entry.getTargetPath()
+        .toByteArray() : null);
+  }
+
+  public static SnapshottableDirectoryStatus[] convert(
+      SnapshottableDirectoryListingProto sdlp) {
+    if (sdlp == null)
+      return null;
+    List<SnapshottableDirectoryStatusProto> list = sdlp
+        .getSnapshottableDirListingList();
+    if (list.isEmpty()) {
+      return new SnapshottableDirectoryStatus[0];
+    } else {
+      SnapshottableDirectoryStatus[] result =
+          new SnapshottableDirectoryStatus[list.size()];
+      for (int i = 0; i < list.size(); i++) {
+        result[i] = convert(list.get(i));
+      }
+      return result;
+    }
+  }
+
+  public static SnapshottableDirectoryStatus convert(
+      SnapshottableDirectoryStatusProto sdirStatusProto) {
+    if (sdirStatusProto == null) {
+      return null;
+    }
+    final HdfsFileStatusProto status = sdirStatusProto.getDirStatus();
+    return new SnapshottableDirectoryStatus(
+        status.getModificationTime(),
+        status.getAccessTime(),
+        convert(status.getPermission()),
+        status.getOwner(),
+        status.getGroup(),
+        status.getPath().toByteArray(),
+        status.getFileId(),
+        status.getChildrenNum(),
+        sdirStatusProto.getSnapshotNumber(),
+        sdirStatusProto.getSnapshotQuota(),
+        sdirStatusProto.getParentFullpath().toByteArray());
+  }
+
+  // DataEncryptionKey
+  public static DataEncryptionKey convert(DataEncryptionKeyProto bet) {
+    String encryptionAlgorithm = bet.getEncryptionAlgorithm();
+    return new DataEncryptionKey(bet.getKeyId(),
+        bet.getBlockPoolId(),
+        bet.getNonce().toByteArray(),
+        bet.getEncryptionKey().toByteArray(),
+        bet.getExpiryDate(),
+        encryptionAlgorithm.isEmpty() ? null : encryptionAlgorithm);
+  }
+
+  public static Token<DelegationTokenIdentifier> convertDelegationToken(
+      TokenProto blockToken) {
+    return new Token<>(blockToken.getIdentifier()
+        .toByteArray(), blockToken.getPassword().toByteArray(), new Text(
+        blockToken.getKind()), new Text(blockToken.getService()));
+  }
+
+  // Arrays of DatanodeId
+  public static DatanodeIDProto[] convert(DatanodeID[] did) {
+    if (did == null)
+      return null;
+    final int len = did.length;
+    DatanodeIDProto[] result = new DatanodeIDProto[len];
+    for (int i = 0; i < len; ++i) {
+      result[i] = convert(did[i]);
+    }
+    return result;
+  }
+
+  public static FsPermissionProto convert(FsPermission p) {
+    return FsPermissionProto.newBuilder().setPerm(p.toExtendedShort()).build();
+  }
+
+  public static HdfsFileStatus convert(HdfsFileStatusProto fs) {
+    if (fs == null)
+      return null;
+    return new HdfsLocatedFileStatus(
+        fs.getLength(), fs.getFileType().equals(FileType.IS_DIR),
+        fs.getBlockReplication(), fs.getBlocksize(),
+        fs.getModificationTime(), fs.getAccessTime(),
+        convert(fs.getPermission()), fs.getOwner(), fs.getGroup(),
+        fs.getFileType().equals(FileType.IS_SYMLINK) ?
+            fs.getSymlink().toByteArray() : null,
+        fs.getPath().toByteArray(),
+        fs.hasFileId()? fs.getFileId(): HdfsConstants.GRANDFATHER_INODE_ID,
+        fs.hasLocations() ? convert(fs.getLocations()) : null,
+        fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
+        fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,
+        fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy()
+            : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
+  }
+
+  public static CorruptFileBlocks convert(CorruptFileBlocksProto c) {
+    if (c == null)
+      return null;
+    List<String> fileList = c.getFilesList();
+    return new CorruptFileBlocks(fileList.toArray(new String[fileList.size()]),
+        c.getCookie());
+  }
+
+  public static ContentSummary convert(ContentSummaryProto cs) {
+    if (cs == null) return null;
+    ContentSummary.Builder builder = new ContentSummary.Builder();
+    builder.length(cs.getLength()).
+        fileCount(cs.getFileCount()).
+        directoryCount(cs.getDirectoryCount()).
+        quota(cs.getQuota()).
+        spaceConsumed(cs.getSpaceConsumed()).
+        spaceQuota(cs.getSpaceQuota());
+    if (cs.hasTypeQuotaInfos()) {
+      for (HdfsProtos.StorageTypeQuotaInfoProto info :
+          cs.getTypeQuotaInfos().getTypeQuotaInfoList()) {
+        StorageType type = convertStorageType(info.getType());
+        builder.typeConsumed(type, info.getConsumed());
+        builder.typeQuota(type, info.getQuota());
+      }
+    }
+    return builder.build();
+  }
+
+  public static RollingUpgradeActionProto convert(RollingUpgradeAction a) {
+    switch (a) {
+    case QUERY:
+      return RollingUpgradeActionProto.QUERY;
+    case PREPARE:
+      return RollingUpgradeActionProto.START;
+    case FINALIZE:
+      return RollingUpgradeActionProto.FINALIZE;
+    default:
+      throw new IllegalArgumentException("Unexpected value: " + a);
+    }
+  }
+
+  public static RollingUpgradeInfo convert(RollingUpgradeInfoProto proto) {
+    RollingUpgradeStatusProto status = proto.getStatus();
+    return new RollingUpgradeInfo(status.getBlockPoolId(),
+        proto.getCreatedRollbackImages(),
+        proto.getStartTime(), proto.getFinalizeTime());
+  }
+
+  public static DatanodeStorageReport[] convertDatanodeStorageReports(
+      List<DatanodeStorageReportProto> protos) {
+    final DatanodeStorageReport[] reports
+        = new DatanodeStorageReport[protos.size()];
+    for(int i = 0; i < reports.length; i++) {
+      reports[i] = convertDatanodeStorageReport(protos.get(i));
+    }
+    return reports;
+  }
+
+  public static DatanodeStorageReport convertDatanodeStorageReport(
+      DatanodeStorageReportProto proto) {
+    return new DatanodeStorageReport(
+        convert(proto.getDatanodeInfo()),
+        convertStorageReports(proto.getStorageReportsList()));
+  }
+
+  public static StorageReport[] convertStorageReports(
+      List<StorageReportProto> list) {
+    final StorageReport[] report = new StorageReport[list.size()];
+    for (int i = 0; i < report.length; i++) {
+      report[i] = convert(list.get(i));
+    }
+    return report;
+  }
+
+  public static StorageReport convert(StorageReportProto p) {
+    return new StorageReport(
+        p.hasStorage() ?
+            convert(p.getStorage()) :
+            new DatanodeStorage(p.getStorageUuid()),
+        p.getFailed(), p.getCapacity(), p.getDfsUsed(), p.getRemaining(),
+        p.getBlockPoolUsed());
+  }
+
+  public static DatanodeStorage convert(DatanodeStorageProto s) {
+    return new DatanodeStorage(s.getStorageUuid(),
+        convertState(s.getState()), convertStorageType(s.getStorageType()));
+  }
+
+  private static State convertState(StorageState state) {
+    switch(state) {
+    case READ_ONLY_SHARED:
+      return State.READ_ONLY_SHARED;
+    case NORMAL:
+    default:
+      return State.NORMAL;
+    }
+  }
+
+  public static SafeModeActionProto convert(
+      SafeModeAction a) {
+    switch (a) {
+    case SAFEMODE_LEAVE:
+      return SafeModeActionProto.SAFEMODE_LEAVE;
+    case SAFEMODE_ENTER:
+      return SafeModeActionProto.SAFEMODE_ENTER;
+    case SAFEMODE_GET:
+      return SafeModeActionProto.SAFEMODE_GET;
+    default:
+      throw new IllegalArgumentException("Unexpected SafeModeAction :" + a);
+    }
+  }
+
+  public static DatanodeInfo[] convert(List<DatanodeInfoProto> list) {
+    DatanodeInfo[] info = new DatanodeInfo[list.size()];
+    for (int i = 0; i < info.length; i++) {
+      info[i] = convert(list.get(i));
+    }
+    return info;
+  }
+
+  public static long[] convert(GetFsStatsResponseProto res) {
+    long[] result = new long[7];
+    result[ClientProtocol.GET_STATS_CAPACITY_IDX] = res.getCapacity();
+    result[ClientProtocol.GET_STATS_USED_IDX] = res.getUsed();
+    result[ClientProtocol.GET_STATS_REMAINING_IDX] = res.getRemaining();
+    result[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX] = res.getUnderReplicated();
+    result[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX] = res.getCorruptBlocks();
+    result[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX] = res.getMissingBlocks();
+    result[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX] =
+        res.getMissingReplOneBlocks();
+    return result;
+  }
+
+  public static DatanodeReportTypeProto
+    convert(DatanodeReportType t) {
+    switch (t) {
+    case ALL: return DatanodeReportTypeProto.ALL;
+    case LIVE: return DatanodeReportTypeProto.LIVE;
+    case DEAD: return DatanodeReportTypeProto.DEAD;
+    case DECOMMISSIONING: return DatanodeReportTypeProto.DECOMMISSIONING;
+    default:
+      throw new IllegalArgumentException("Unexpected data type report:" + t);
+    }
+  }
+
+  public static DirectoryListing convert(DirectoryListingProto dl) {
+    if (dl == null)
+      return null;
+    List<HdfsFileStatusProto> partList =  dl.getPartialListingList();
+    return new DirectoryListing(partList.isEmpty() ?
+        new HdfsLocatedFileStatus[0] :
+        convert(partList.toArray(new HdfsFileStatusProto[partList.size()])),
+        dl.getRemainingEntries());
+  }
+
+  public static HdfsFileStatus[] convert(HdfsFileStatusProto[] fs) {
+    if (fs == null) return null;
+    final int len = fs.length;
+    HdfsFileStatus[] result = new HdfsFileStatus[len];
+    for (int i = 0; i < len; ++i) {
+      result[i] = convert(fs[i]);
+    }
+    return result;
+  }
+
+  // The creatFlag field in PB is a bitmask whose values are the same a the
+  // emum values of CreateFlag
+  public static int convertCreateFlag(EnumSetWritable<CreateFlag> flag) {
+    int value = 0;
+    if (flag.contains(CreateFlag.APPEND)) {
+      value |= CreateFlagProto.APPEND.getNumber();
+    }
+    if (flag.contains(CreateFlag.CREATE)) {
+      value |= CreateFlagProto.CREATE.getNumber();
+    }
+    if (flag.contains(CreateFlag.OVERWRITE)) {
+      value |= CreateFlagProto.OVERWRITE.getNumber();
+    }
+    if (flag.contains(CreateFlag.LAZY_PERSIST)) {
+      value |= CreateFlagProto.LAZY_PERSIST.getNumber();
+    }
+    if (flag.contains(CreateFlag.NEW_BLOCK)) {
+      value |= CreateFlagProto.NEW_BLOCK.getNumber();
+    }
+    return value;
+  }
+
+  public static FsServerDefaults convert(FsServerDefaultsProto fs) {
+    if (fs == null) return null;
+    return new FsServerDefaults(
+        fs.getBlockSize(), fs.getBytesPerChecksum(),
+        fs.getWritePacketSize(), (short) fs.getReplication(),
+        fs.getFileBufferSize(),
+        fs.getEncryptDataTransfer(),
+        fs.getTrashInterval(),
+        convert(fs.getChecksumType()));
+  }
+
+  public static List<CryptoProtocolVersionProto> convert(
+      CryptoProtocolVersion[] versions) {
+    List<CryptoProtocolVersionProto> protos =
+        Lists.newArrayListWithCapacity(versions.length);
+    for (CryptoProtocolVersion v: versions) {
+      protos.add(convert(v));
+    }
+    return protos;
+  }
+
+  static List<StorageTypesProto> convert(StorageType[][] types) {
+    List<StorageTypesProto> list = Lists.newArrayList();
+    if (types != null) {
+      for (StorageType[] ts : types) {
+        StorageTypesProto.Builder builder = StorageTypesProto.newBuilder();
+        builder.addAllStorageTypes(convertStorageTypes(ts));
+        list.add(builder.build());
+      }
+    }
+    return list;
+  }
+
+  public static BlockStoragePolicyProto convert(BlockStoragePolicy policy) {
+    BlockStoragePolicyProto.Builder builder = BlockStoragePolicyProto
+        .newBuilder().setPolicyId(policy.getId()).setName(policy.getName());
+    // creation storage types
+    StorageTypesProto creationProto = convert(policy.getStorageTypes());
+    Preconditions.checkArgument(creationProto != null);
+    builder.setCreationPolicy(creationProto);
+    // creation fallback
+    StorageTypesProto creationFallbackProto = convert(
+        policy.getCreationFallbacks());
+    if (creationFallbackProto != null) {
+      builder.setCreationFallbackPolicy(creationFallbackProto);
+    }
+    // replication fallback
+    StorageTypesProto replicationFallbackProto = convert(
+        policy.getReplicationFallbacks());
+    if (replicationFallbackProto != null) {
+      builder.setReplicationFallbackPolicy(replicationFallbackProto);
+    }
+    return builder.build();
+  }
+
+  public static StorageTypesProto convert(StorageType[] types) {
+    if (types == null || types.length == 0) {
+      return null;
+    }
+    List<StorageTypeProto> list = convertStorageTypes(types);
+    return StorageTypesProto.newBuilder().addAllStorageTypes(list).build();
+  }
+
+  public static DatanodeID[] convert(DatanodeIDProto[] did) {
+    if (did == null) return null;
+    final int len = did.length;
+    DatanodeID[] result = new DatanodeID[len];
+    for (int i = 0; i < len; ++i) {
+      result[i] = convert(did[i]);
+    }
+    return result;
+  }
+
+  // Block
+  public static BlockProto convert(Block b) {
+    return BlockProto.newBuilder().setBlockId(b.getBlockId())
+        .setGenStamp(b.getGenerationStamp()).setNumBytes(b.getNumBytes())
+        .build();
+  }
+
+  public static Block convert(BlockProto b) {
+    return new Block(b.getBlockId(), b.getNumBytes(), b.getGenStamp());
+  }
+
+  static public DatanodeInfo[] convert(DatanodeInfoProto di[]) {
+    if (di == null) return null;
+    DatanodeInfo[] result = new DatanodeInfo[di.length];
+    for (int i = 0; i < di.length; i++) {
+      result[i] = convert(di[i]);
+    }
+    return result;
+  }
+
+  public static DatanodeStorageReportProto convertDatanodeStorageReport(
+      DatanodeStorageReport report) {
+    return DatanodeStorageReportProto.newBuilder()
+        .setDatanodeInfo(convert(report.getDatanodeInfo()))
+        .addAllStorageReports(convertStorageReports(report.getStorageReports()))
+        .build();
+  }
+
+  public static List<DatanodeStorageReportProto> convertDatanodeStorageReports(
+      DatanodeStorageReport[] reports) {
+    final List<DatanodeStorageReportProto> protos
+        = new ArrayList<>(reports.length);
+    for(int i = 0; i < reports.length; i++) {
+      protos.add(convertDatanodeStorageReport(reports[i]));
+    }
+    return protos;
+  }
+
+  public static LocatedBlock[] convertLocatedBlock(LocatedBlockProto[] lb) {
+    if (lb == null) return null;
+    return convertLocatedBlock(Arrays.asList(lb)).toArray(
+        new LocatedBlock[lb.length]);
+  }
+
+  public static LocatedBlocksProto convert(LocatedBlocks lb) {
+    if (lb == null) {
+      return null;
+    }
+    LocatedBlocksProto.Builder builder =
+        LocatedBlocksProto.newBuilder();
+    if (lb.getLastLocatedBlock() != null) {
+      builder.setLastBlock(convert(lb.getLastLocatedBlock()));
+    }
+    if (lb.getFileEncryptionInfo() != null) {
+      builder.setFileEncryptionInfo(convert(lb.getFileEncryptionInfo()));
+    }
+    return builder.setFileLength(lb.getFileLength())
+        .setUnderConstruction(lb.isUnderConstruction())
+        .addAllBlocks(convertLocatedBlock2(lb.getLocatedBlocks()))
+        .setIsLastBlockComplete(lb.isLastBlockComplete()).build();
+  }
+
+  public static DataEncryptionKeyProto convert(DataEncryptionKey bet) {
+    DataEncryptionKeyProto.Builder b = DataEncryptionKeyProto.newBuilder()
+        .setKeyId(bet.keyId)
+        .setBlockPoolId(bet.blockPoolId)
+        .setNonce(ByteString.copyFrom(bet.nonce))
+        .setEncryptionKey(ByteString.copyFrom(bet.encryptionKey))
+        .setExpiryDate(bet.expiryDate);
+    if (bet.encryptionAlgorithm != null) {
+      b.setEncryptionAlgorithm(bet.encryptionAlgorithm);
+    }
+    return b.build();
+  }
+
+  public static FsServerDefaultsProto convert(FsServerDefaults fs) {
+    if (fs == null) return null;
+    return FsServerDefaultsProto.newBuilder().
+      setBlockSize(fs.getBlockSize()).
+      setBytesPerChecksum(fs.getBytesPerChecksum()).
+      setWritePacketSize(fs.getWritePacketSize())
+      .setReplication(fs.getReplication())
+      .setFileBufferSize(fs.getFileBufferSize())
+      .setEncryptDataTransfer(fs.getEncryptDataTransfer())
+      .setTrashInterval(fs.getTrashInterval())
+      .setChecksumType(convert(fs.getChecksumType()))
+      .build();
+  }
+
+  public static EnumSetWritable<CreateFlag> convertCreateFlag(int flag) {
+    EnumSet<CreateFlag> result =
+       EnumSet.noneOf(CreateFlag.class);
+    if ((flag & CreateFlagProto.APPEND_VALUE) == CreateFlagProto.APPEND_VALUE) {
+      result.add(CreateFlag.APPEND);
+    }
+    if ((flag & CreateFlagProto.CREATE_VALUE) == CreateFlagProto.CREATE_VALUE) {
+      result.add(CreateFlag.CREATE);
+    }
+    if ((flag & CreateFlagProto.OVERWRITE_VALUE)
+        == CreateFlagProto.OVERWRITE_VALUE) {
+      result.add(CreateFlag.OVERWRITE);
+    }
+    if ((flag & CreateFlagProto.LAZY_PERSIST_VALUE)
+        == CreateFlagProto.LAZY_PERSIST_VALUE) {
+      result.add(CreateFlag.LAZY_PERSIST);
+    }
+    if ((flag & CreateFlagProto.NEW_BLOCK_VALUE)
+        == CreateFlagProto.NEW_BLOCK_VALUE) {
+      result.add(CreateFlag.NEW_BLOCK);
+    }
+    return new EnumSetWritable<CreateFlag>(result, CreateFlag.class);
+  }
+
+  public static EnumSet<CacheFlag> convertCacheFlags(int flags) {
+    EnumSet<CacheFlag> result = EnumSet.noneOf(CacheFlag.class);
+    if ((flags & CacheFlagProto.FORCE_VALUE) == CacheFlagProto.FORCE_VALUE) {
+      result.add(CacheFlag.FORCE);
+    }
+    return result;
+  }
+
+  public static HdfsFileStatusProto convert(HdfsFileStatus fs) {
+    if (fs == null)
+      return null;
+    FileType fType = FileType.IS_FILE;
+    if (fs.isDir()) {
+      fType = FileType.IS_DIR;
+    } else if (fs.isSymlink()) {
+      fType = FileType.IS_SYMLINK;
+    }
+
+    HdfsFileStatusProto.Builder builder =
+     HdfsFileStatusProto.newBuilder().
+      setLength(fs.getLen()).
+      setFileType(fType).
+      setBlockReplication(fs.getReplication()).
+      setBlocksize(fs.getBlockSize()).
+      setModificationTime(fs.getModificationTime()).
+      setAccessTime(fs.getAccessTime()).
+      setPermission(convert(fs.getPermission())).
+      setOwner(fs.getOwner()).
+      setGroup(fs.getGroup()).
+      setFileId(fs.getFileId()).
+      setChildrenNum(fs.getChildrenNum()).
+      setPath(ByteString.copyFrom(fs.getLocalNameInBytes())).
+      setStoragePolicy(fs.getStoragePolicy());
+    if (fs.isSymlink())  {
+      builder.setSymlink(ByteString.copyFrom(fs.getSymlinkInBytes()));
+    }
+    if (fs.getFileEncryptionInfo() != null) {
+      builder.setFileEncryptionInfo(convert(fs.getFileEncryptionInfo()));
+    }
+    if (fs instanceof HdfsLocatedFileStatus) {
+      final HdfsLocatedFileStatus lfs = (HdfsLocatedFileStatus) fs;
+      LocatedBlocks locations = lfs.getBlockLocations();
+      if (locations != null) {
+        builder.setLocations(convert(locations));
+      }
+    }
+    return builder.build();
+  }
+
+  public static SnapshottableDirectoryStatusProto convert(
+      SnapshottableDirectoryStatus status) {
+    if (status == null) {
+      return null;
+    }
+    int snapshotNumber = status.getSnapshotNumber();
+    int snapshotQuota = status.getSnapshotQuota();
+    byte[] parentFullPath = status.getParentFullPath();
+    ByteString parentFullPathBytes = ByteString.copyFrom(
+        parentFullPath == null ? DFSUtilClient.EMPTY_BYTES : parentFullPath);
+    HdfsFileStatusProto fs = convert(status.getDirStatus());
+    SnapshottableDirectoryStatusProto.Builder builder =
+        SnapshottableDirectoryStatusProto
+        .newBuilder().setSnapshotNumber(snapshotNumber)
+        .setSnapshotQuota(snapshotQuota).setParentFullpath(parentFullPathBytes)
+        .setDirStatus(fs);
+    return builder.build();
+  }
+
+  public static HdfsFileStatusProto[] convert(HdfsFileStatus[] fs) {
+    if (fs == null) return null;
+    final int len = fs.length;
+    HdfsFileStatusProto[] result = new HdfsFileStatusProto[len];
+    for (int i = 0; i < len; ++i) {
+      result[i] = convert(fs[i]);
+    }
+    return result;
+  }
+
+  public static DirectoryListingProto convert(DirectoryListing d) {
+    if (d == null)
+      return null;
+    return DirectoryListingProto.newBuilder().
+        addAllPartialListing(Arrays.asList(
+            convert(d.getPartialListing()))).
+        setRemainingEntries(d.getRemainingEntries()).
+        build();
+  }
+
+  public static GetFsStatsResponseProto convert(long[] fsStats) {
+    GetFsStatsResponseProto.Builder result = GetFsStatsResponseProto
+        .newBuilder();
+    if (fsStats.length >= ClientProtocol.GET_STATS_CAPACITY_IDX + 1)
+      result.setCapacity(fsStats[ClientProtocol.GET_STATS_CAPACITY_IDX]);
+    if (fsStats.length >= ClientProtocol.GET_STATS_USED_IDX + 1)
+      result.setUsed(fsStats[ClientProtocol.GET_STATS_USED_IDX]);
+    if (fsStats.length >= ClientProtocol.GET_STATS_REMAINING_IDX + 1)
+      result.setRemaining(fsStats[ClientProtocol.GET_STATS_REMAINING_IDX]);
+    if (fsStats.length >= ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX + 1)
+      result.setUnderReplicated(
+              fsStats[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX]);
+    if (fsStats.length >= ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX + 1)
+      result.setCorruptBlocks(
+          fsStats[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX]);
+    if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX + 1)
+      result.setMissingBlocks(
+          fsStats[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX]);
+    if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX + 1)
+      result.setMissingReplOneBlocks(
+          fsStats[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX]);
+    return result.build();
+  }
+
+  public static DatanodeReportType convert(DatanodeReportTypeProto t) {
+    switch (t) {
+    case ALL: return DatanodeReportType.ALL;
+    case LIVE: return DatanodeReportType.LIVE;
+    case DEAD: return DatanodeReportType.DEAD;
+    case DECOMMISSIONING: return DatanodeReportType.DECOMMISSIONING;
+    default:
+      throw new IllegalArgumentException("Unexpected data type report:" + t);
+    }
+  }
+
+  public static SafeModeAction convert(
+      SafeModeActionProto a) {
+    switch (a) {
+    case SAFEMODE_LEAVE:
+      return SafeModeAction.SAFEMODE_LEAVE;
+    case SAFEMODE_ENTER:
+      return SafeModeAction.SAFEMODE_ENTER;
+    case SAFEMODE_GET:
+      return SafeModeAction.SAFEMODE_GET;
+    default:
+      throw new IllegalArgumentException("Unexpected SafeModeAction :" + a);
+    }
+  }
+
+  public static RollingUpgradeAction convert(RollingUpgradeActionProto a) {
+    switch (a) {
+    case QUERY:
+      return RollingUpgradeAction.QUERY;
+    case START:
+      return RollingUpgradeAction.PREPARE;
+    case FINALIZE:
+      return RollingUpgradeAction.FINALIZE;
+    default:
+      throw new IllegalArgumentException("Unexpected value: " + a);
+    }
+  }
+
+  public static RollingUpgradeStatusProto convertRollingUpgradeStatus(
+      RollingUpgradeStatus status) {
+    return RollingUpgradeStatusProto.newBuilder()
+        .setBlockPoolId(status.getBlockPoolId())
+        .setFinalized(status.isFinalized())
+        .build();
+  }
+
+  public static RollingUpgradeStatus convert(RollingUpgradeStatusProto proto) {
+    return new RollingUpgradeStatus(proto.getBlockPoolId(),
+        proto.getFinalized());
+  }
+
+  public static RollingUpgradeInfoProto convert(RollingUpgradeInfo info) {
+    return RollingUpgradeInfoProto.newBuilder()
+        .setStatus(convertRollingUpgradeStatus(info))
+        .setCreatedRollbackImages(info.createdRollbackImages())
+        .setStartTime(info.getStartTime())
+        .setFinalizeTime(info.getFinalizeTime())
+        .build();
+  }
+
+  public static CorruptFileBlocksProto convert(CorruptFileBlocks c) {
+    if (c == null)
+      return null;
+    return CorruptFileBlocksProto.newBuilder().
+        addAllFiles(Arrays.asList(c.getFiles())).
+        setCookie(c.getCookie()).
+        build();
+  }
+
+  public static ContentSummaryProto convert(ContentSummary cs) {
+    if (cs == null) return null;
+    ContentSummaryProto.Builder builder = ContentSummaryProto.newBuilder();
+        builder.setLength(cs.getLength()).
+        setFileCount(cs.getFileCount()).
+        setDirectoryCount(cs.getDirectoryCount()).
+        setQuota(cs.getQuota()).
+        setSpaceConsumed(cs.getSpaceConsumed()).
+        setSpaceQuota(cs.getSpaceQuota());
+
+    if (cs.isTypeQuotaSet() || cs.isTypeConsumedAvailable()) {
+      HdfsProtos.StorageTypeQuotaInfosProto.Builder isb =
+          HdfsProtos.StorageTypeQuotaInfosProto.newBuilder();
+      for (StorageType t: StorageType.getTypesSupportingQuota()) {
+        HdfsProtos.StorageTypeQuotaInfoProto info =
+            HdfsProtos.StorageTypeQuotaInfoProto.newBuilder().
+                setType(convertStorageType(t)).
+                setConsumed(cs.getTypeConsumed(t)).
+                setQuota(cs.getTypeQuota(t)).
+                build();
+        isb.addTypeQuotaInfo(info);
+      }
+      builder.setTypeQuotaInfos(isb);
+    }
+    return builder.build();
+  }
+
+  public static DatanodeStorageProto convert(DatanodeStorage s) {
+    return DatanodeStorageProto.newBuilder()
+        .setState(convertState(s.getState()))
+        .setStorageType(convertStorageType(s.getStorageType()))
+        .setStorageUuid(s.getStorageID()).build();
+  }
+
+  private static StorageState convertState(State state) {
+    switch(state) {
+    case READ_ONLY_SHARED:
+      return StorageState.READ_ONLY_SHARED;
+    case NORMAL:
+    default:
+      return StorageState.NORMAL;
+    }
+  }
+
+  public static StorageReportProto convert(StorageReport r) {
+    StorageReportProto.Builder builder = StorageReportProto.newBuilder()
+        .setBlockPoolUsed(r.getBlockPoolUsed()).setCapacity(r.getCapacity())
+        .setDfsUsed(r.getDfsUsed()).setRemaining(r.getRemaining())
+        .setStorageUuid(r.getStorage().getStorageID())
+        .setStorage(convert(r.getStorage()));
+    return builder.build();
+  }
+
+  public static List<StorageReportProto> convertStorageReports(StorageReport[] storages) {
+    final List<StorageReportProto> protos = new ArrayList<StorageReportProto>(
+        storages.length);
+    for(int i = 0; i < storages.length; i++) {
+      protos.add(convert(storages[i]));
+    }
+    return protos;
+  }
+
+  public static SnapshottableDirectoryListingProto convert(
+      SnapshottableDirectoryStatus[] status) {
+    if (status == null)
+      return null;
+    SnapshottableDirectoryStatusProto[] protos =
+        new SnapshottableDirectoryStatusProto[status.length];
+    for (int i = 0; i < status.length; i++) {
+      protos[i] = convert(status[i]);
+    }
+    List<SnapshottableDirectoryStatusProto> protoList = Arrays.asList(protos);
+    return SnapshottableDirectoryListingProto.newBuilder()
+        .addAllSnapshottableDirListing(protoList).build();
+  }
+
+  public static SnapshotDiffReportEntryProto convert(DiffReportEntry entry) {
+    if (entry == null) {
+      return null;
+    }
+    ByteString sourcePath = ByteString
+        .copyFrom(entry.getSourcePath() == null ? DFSUtilClient.EMPTY_BYTES : entry
+            .getSourcePath());
+    String modification = entry.getType().getLabel();
+    SnapshotDiffReportEntryProto.Builder builder = SnapshotDiffReportEntryProto
+        .newBuilder().setFullpath(sourcePath)
+        .setModificationLabel(modification);
+    if (entry.getType() == DiffType.RENAME) {
+      ByteString targetPath = ByteString
+          .copyFrom(entry.getTargetPath() == null ? DFSUtilClient.EMPTY_BYTES : entry
+              .getTargetPath());
+      builder.setTargetPath(targetPath);
+    }
+    return builder.build();
+  }
+
+  public static SnapshotDiffReportProto convert(SnapshotDiffReport report) {
+    if (report == null) {
+      return null;
+    }
+    List<DiffReportEntry> entries = report.getDiffList();
+    List<SnapshotDiffReportEntryProto> entryProtos = new ArrayList<>();
+    for (DiffReportEntry entry : entries) {
+      SnapshotDiffReportEntryProto entryProto = convert(entry);
+      if (entryProto != null)
+        entryProtos.add(entryProto);
+    }
+
+    SnapshotDiffReportProto reportProto = SnapshotDiffReportProto.newBuilder()
+        .setSnapshotRoot(report.getSnapshotRoot())
+        .setFromSnapshot(report.getFromSnapshot())
+        .setToSnapshot(report.getLaterSnapshotName())
+        .addAllDiffReportEntries(entryProtos).build();
+    return reportProto;
+  }
+
+  public static CacheDirectiveStatsProto convert(CacheDirectiveStats stats) {
+    CacheDirectiveStatsProto.Builder builder =
+        CacheDirectiveStatsProto.newBuilder();
+    builder.setBytesNeeded(stats.getBytesNeeded());
+    builder.setBytesCached(stats.getBytesCached());
+    builder.setFilesNeeded(stats.getFilesNeeded());
+    builder.setFilesCached(stats.getFilesCached());
+    builder.setHasExpired(stats.hasExpired());
+    return builder.build();
+  }
+
+  public static CacheDirectiveEntryProto convert(CacheDirectiveEntry entry) {
+    CacheDirectiveEntryProto.Builder builder =
+        CacheDirectiveEntryProto.newBuilder();
+    builder.setInfo(convert(entry.getInfo()));
+    builder.setStats(convert(entry.getStats()));
+    return builder.build();
+  }
+
+  public static boolean[] convertBooleanList(
+    List<Boolean> targetPinningsList) {
+    final boolean[] targetPinnings = new boolean[targetPinningsList.size()];
+    for (int i = 0; i < targetPinningsList.size(); i++) {
+      targetPinnings[i] = targetPinningsList.get(i);
+    }
+    return targetPinnings;
+  }
+
+  public static CachePoolStatsProto convert(CachePoolStats stats) {
+    CachePoolStatsProto.Builder builder = CachePoolStatsProto.newBuilder();
+    builder.setBytesNeeded(stats.getBytesNeeded());
+    builder.setBytesCached(stats.getBytesCached());
+    builder.setBytesOverlimit(stats.getBytesOverlimit());
+    builder.setFilesNeeded(stats.getFilesNeeded());
+    builder.setFilesCached(stats.getFilesCached());
+    return builder.build();
+  }
+
+  public static CachePoolEntryProto convert(CachePoolEntry entry) {
+    CachePoolEntryProto.Builder builder = CachePoolEntryProto.newBuilder();
+    builder.setInfo(convert(entry.getInfo()));
+    builder.setStats(convert(entry.getStats()));
+    return builder.build();
+  }
+
+  public static DatanodeLocalInfoProto convert(DatanodeLocalInfo info) {
+    DatanodeLocalInfoProto.Builder builder = DatanodeLocalInfoProto.newBuilder();
+    builder.setSoftwareVersion(info.getSoftwareVersion());
+    builder.setConfigVersion(info.getConfigVersion());
+    builder.setUptime(info.getUptime());
+    return builder.build();
+  }
+
+  public static GetAclStatusResponseProto convert(AclStatus e) {
+    AclStatusProto.Builder builder = AclStatusProto.newBuilder();
+    builder.setOwner(e.getOwner())
+        .setGroup(e.getGroup()).setSticky(e.isStickyBit())
+        .addAllEntries(convertAclEntryProto(e.getEntries()));
+    if (e.getPermission() != null) {
+      builder.setPermission(convert(e.getPermission()));
+    }
+    AclStatusProto r = builder.build();
+    return GetAclStatusResponseProto.newBuilder().setResult(r).build();
+  }
+
+  public static EnumSet<XAttrSetFlag> convert(int flag) {
+    EnumSet<XAttrSetFlag> result =
+        EnumSet.noneOf(XAttrSetFlag.class);
+    if ((flag & XAttrSetFlagProto.XATTR_CREATE_VALUE) ==
+        XAttrSetFlagProto.XATTR_CREATE_VALUE) {
+      result.add(XAttrSetFlag.CREATE);
+    }
+    if ((flag & XAttrSetFlagProto.XATTR_REPLACE_VALUE) ==
+        XAttrSetFlagProto.XATTR_REPLACE_VALUE) {
+      result.add(XAttrSetFlag.REPLACE);
+    }
+    return result;
+  }
+
+  public static XAttr convertXAttr(XAttrProto a) {
+    XAttr.Builder builder = new XAttr.Builder();
+    builder.setNameSpace(convert(a.getNamespace()));
+    if (a.hasName()) {
+      builder.setName(a.getName());
+    }
+    if (a.hasValue()) {
+      builder.setValue(a.getValue().toByteArray());
+    }
+    return builder.build();
+  }
+
+  public static GetXAttrsResponseProto convertXAttrsResponse(
+      List<XAttr> xAttrs) {
+    GetXAttrsResponseProto.Builder builder = GetXAttrsResponseProto
+        .newBuilder();
+    if (xAttrs != null) {
+      builder.addAllXAttrs(convertXAttrProto(xAttrs));
+    }
+    return builder.build();
+  }
+
+  public static ListXAttrsResponseProto convertListXAttrsResponse(
+    List<XAttr> names) {
+    ListXAttrsResponseProto.Builder builder =
+      ListXAttrsResponseProto.newBuilder();
+    if (names != null) {
+      builder.addAllXAttrs(convertXAttrProto(names));
+    }
+    return builder.build();
+  }
+
+  public static EncryptionZoneProto convert(EncryptionZone zone) {
+    return EncryptionZoneProto.newBuilder()
+        .setId(zone.getId())
+        .setPath(zone.getPath())
+        .setSuite(convert(zone.getSuite()))
+        .setCryptoProtocolVersion(convert(zone.getVersion()))
+        .setKeyName(zone.getKeyName())
+        .build();
+  }
+
+  public static SlotId convert(ShortCircuitShmSlotProto slotId) {
+    return new SlotId(convert(slotId.getShmId()),
+        slotId.getSlotIdx());
+  }
+
+  public static GetEditsFromTxidResponseProto convertEditsResponse(EventBatchList el) {
+    InotifyProtos.EventsListProto.Builder builder =
+        InotifyProtos.EventsListProto.newBuilder();
+    for (EventBatch b : el.getBatches()) {
+      List<InotifyProtos.EventProto> events = Lists.newArrayList();
+      for (Event e : b.getEvents()) {
+        switch (e.getEventType()) {
+          case CLOSE:
+            Event.CloseEvent ce = (Event.CloseEvent) e;
+            events.add(InotifyProtos.EventProto.newBuilder()
+                .setType(InotifyProtos.EventType.EVENT_CLOSE)
+                .setContents(
+                    InotifyProtos.CloseEventProto.newBuilder()
+                        .setPath(ce.getPath())
+                        .setFileSize(ce.getFileSize())
+                        .setTimestamp(ce.getTimestamp()).build().toByteString()
+                ).build());
+            break;
+          case CREATE:
+            Event.CreateEvent ce2 = (Event.CreateEvent) e;
+            events.add(InotifyProtos.EventProto.newBuilder()
+                .setType(InotifyProtos.EventType.EVENT_CREATE)
+                .setContents(
+                    InotifyProtos.CreateEventProto.newBuilder()
+                        .setType(createTypeConvert(ce2.getiNodeType()))
+                        .setPath(ce2.getPath())
+                        .setCtime(ce2.getCtime())
+                        .setOwnerName(ce2.getOwnerName())
+                        .setGroupName(ce2.getGroupName())
+                        .setPerms(convert(ce2.getPerms()))
+                        .setReplication(ce2.getReplication())
+                        .setSymlinkTarget(ce2.getSymlinkTarget() == null ?
+                            "" : ce2.getSymlinkTarget())
+                        .setDefaultBlockSize(ce2.getDefaultBlockSize())
+                        .setOverwrite(ce2.getOverwrite()).build().toByteString()
+                ).build());
+            break;
+          case METADATA:
+            Event.MetadataUpdateEvent me = (Event.MetadataUpdateEvent) e;
+            InotifyProtos.MetadataUpdateEventProto.Builder metaB =
+                InotifyProtos.MetadataUpdateEventProto.newBuilder()
+                    .setPath(me.getPath())
+                    .setType(metadataUpdateTypeConvert(me.getMetadataType()))
+                    .setMtime(me.getMtime())
+                    .setAtime(me.getAtime())
+                    .setReplication(me.getReplication())
+                    .setOwnerName(me.getOwnerName() == null ? "" :
+                        me.getOwnerName())
+                    .setGroupName(me.getGroupName() == null ? "" :
+                        me.getGroupName())
+                    .addAllAcls(me.getAcls() == null ?
+                        Lists.<AclEntryProto>newArrayList() :
+                        convertAclEntryProto(me.getAcls()))
+                    .addAllXAttrs(me.getxAttrs() == null ?
+                        Lists.<XAttrProto>newArrayList() :
+                        convertXAttrProto(me.getxAttrs()))
+                    .setXAttrsRemoved(me.isxAttrsRemoved());
+            if (me.getPerms() != null) {
+              metaB.setPerms(convert(me.getPerms()));
+            }
+            events.add(InotifyProtos.EventProto.newBuilder()
+                .setType(InotifyProtos.EventType.EVENT_METADATA)
+                .setContents(metaB.build().toByteString())
+                .build());
+            break;
+          case RENAME:
+            Event.RenameEvent re = (Event.RenameEvent) e;
+            events.add(InotifyProtos.EventProto.newBuilder()
+                .setType(InotifyProtos.EventType.EVENT_RENAME)
+                .setContents(
+                    InotifyProtos.RenameEventProto.newBuilder()
+                        .setSrcPath(re.getSrcPath())
+                        .setDestPath(re.getDstPath())
+                        .setTimestamp(re.getTimestamp()).build().toByteString()
+                ).build());
+            break;
+          case APPEND:
+            Event.AppendEvent re2 = (Event.AppendEvent) e;
+            events.add(InotifyProtos.EventProto.newBuilder()
+                .setType(InotifyProtos.EventType.EVENT_APPEND)
+                .setContents(InotifyProtos.AppendEventProto.newBuilder()
+                    .setPath(re2.getPath())
+                    .setNewBlock(re2.toNewBlock()).build().toByteString())
+                .build());
+            break;
+          case UNLINK:
+            Event.UnlinkEvent ue = (Event.UnlinkEvent) e;
+            events.add(InotifyProtos.EventProto.newBuilder()
+                .setType(InotifyProtos.EventType.EVENT_UNLINK)
+                .setContents(
+                    InotifyProtos.UnlinkEventProto.newBuilder()
+                        .setPath(ue.getPath())
+                        .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);
+        }
+      }
+      builder.addBatch(InotifyProtos.EventBatchProto.newBuilder().
+          setTxid(b.getTxid()).
+          addAllEvents(events));
+    }
+    builder.setFirstTxid(el.getFirstTxid());
+    builder.setLastTxid(el.getLastTxid());
+    builder.setSyncTxid(el.getSyncTxid());
+    return GetEditsFromTxidResponseProto.newBuilder().setEventsList(
+        builder.build()).build();
+  }
+
+  public static CryptoProtocolVersion[] convertCryptoProtocolVersions(
+      List<CryptoProtocolVersionProto> protos) {
+    List<CryptoProtocolVersion> versions =
+        Lists.newArrayListWithCapacity(protos.size());
+    for (CryptoProtocolVersionProto p: protos) {
+      versions.add(convert(p));
+    }
+    return versions.toArray(new CryptoProtocolVersion[]{});
+  }
+
+  public static HdfsProtos.PerFileEncryptionInfoProto convertPerFileEncInfo(
+      FileEncryptionInfo info) {
+    if (info == null) {
+      return null;
+    }
+    return HdfsProtos.PerFileEncryptionInfoProto.newBuilder()
+        .setKey(getByteString(info.getEncryptedDataEncryptionKey()))
+        .setIv(getByteString(info.getIV()))
+        .setEzKeyVersionName(info.getEzKeyVersionName())
+        .build();
+  }
+
+  public static HdfsProtos.ZoneEncryptionInfoProto convert(
+      CipherSuite suite, CryptoProtocolVersion version, String keyName) {
+    if (suite == null || version == null || keyName == null) {
+      return null;
+    }
+    return HdfsProtos.ZoneEncryptionInfoProto.newBuilder()
+        .setSuite(convert(suite))
+        .setCryptoProtocolVersion(convert(version))
+        .setKeyName(keyName)
+        .build();
+  }
+
+  public static FileEncryptionInfo convert(
+      HdfsProtos.PerFileEncryptionInfoProto fileProto,
+      CipherSuite suite, CryptoProtocolVersion version, String keyName) {
+    if (fileProto == null || suite == null || version == null ||
+        keyName == null) {
+      return null;
+    }
+    byte[] key = fileProto.getKey().toByteArray();
+    byte[] iv = fileProto.getIv().toByteArray();
+    String ezKeyVersionName = fileProto.getEzKeyVersionName();
+    return new FileEncryptionInfo(suite, version, key, iv, keyName,
+        ezKeyVersionName);
+  }
+
+  public static DatanodeInfo[] convert(DatanodeInfosProto datanodeInfosProto) {
+    List<DatanodeInfoProto> proto = datanodeInfosProto.getDatanodesList();
+    DatanodeInfo[] infos = new DatanodeInfo[proto.size()];
+    for (int i = 0; i < infos.length; i++) {
+      infos[i] = convert(proto.get(i));
+    }
+    return infos;
+  }
+
+  static List<DatanodeInfosProto> convert(DatanodeInfo[][] targets) {
+    DatanodeInfosProto[] ret = new DatanodeInfosProto[targets.length];
+    for (int i = 0; i < targets.length; i++) {
+      ret[i] = DatanodeInfosProto.newBuilder()
+          .addAllDatanodes(convert(targets[i])).build();
+    }
+    return Arrays.asList(ret);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/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 65cde45..944986c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -938,6 +938,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9101. Remove deprecated NameNode.getUri() static helper method.
     (Mingliang Liu via wheat9)
 
+    HDFS-9111. Move hdfs-client protobuf convert methods from PBHelper to
+    PBHelperClient. (Mingliang Liu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than


[21/50] [abbrv] hadoop git commit: HDFS-6955. DN should reserve disk space for a full block when creating tmp files (Contributed by Kanaka Kumar Avvaru)

Posted by zh...@apache.org.
HDFS-6955. DN should reserve disk space for a full block when creating tmp files (Contributed by Kanaka Kumar Avvaru)


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

Branch: refs/heads/HDFS-7285
Commit: 92c1af1646b1d91a2ab7821e4f7d450e3b6e10bb
Parents: a7201d6
Author: Vinayakumar B <vi...@apache.org>
Authored: Fri Sep 18 16:37:10 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Fri Sep 18 16:37:10 2015 +0530

----------------------------------------------------------------------
 .../hdfs/server/datanode/BlockReceiver.java     |   5 +-
 .../server/datanode/fsdataset/FsVolumeSpi.java  |   8 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |  13 +-
 .../datanode/fsdataset/impl/FsVolumeImpl.java   |  72 ++-
 .../server/datanode/SimulatedFSDataset.java     |   2 +-
 .../server/datanode/TestDirectoryScanner.java   |   2 +-
 .../datanode/extdataset/ExternalVolumeImpl.java |   2 +-
 .../fsdataset/impl/TestRbwSpaceReservation.java | 452 ---------------
 .../fsdataset/impl/TestSpaceReservation.java    | 576 +++++++++++++++++++
 9 files changed, 637 insertions(+), 495 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/92c1af16/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 bc5396f..957b2c7 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
@@ -117,7 +117,7 @@ class BlockReceiver implements Closeable {
   /** the block to receive */
   private final ExtendedBlock block; 
   /** the replica to write */
-  private final ReplicaInPipelineInterface replicaInfo;
+  private ReplicaInPipelineInterface replicaInfo;
   /** pipeline stage */
   private final BlockConstructionStage stage;
   private final boolean isTransfer;
@@ -259,6 +259,9 @@ class BlockReceiver implements Closeable {
     } catch (ReplicaNotFoundException bne) {
       throw bne;
     } catch(IOException ioe) {
+      if (replicaInfo != null) {
+        replicaInfo.releaseAllBytesReserved();
+      }
       IOUtils.closeStream(this);
       cleanupBlock();
       

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92c1af16/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 ee01924..9e16121 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
@@ -62,13 +62,13 @@ public interface FsVolumeSpi {
   boolean isTransientStorage();
 
   /**
-   * Reserve disk space for an RBW block so a writer does not run out of
-   * space before the block is full.
+   * Reserve disk space for a block (RBW or Re-replicating)
+   * so a writer does not run out of space before the block is full.
    */
-  void reserveSpaceForRbw(long bytesToReserve);
+  void reserveSpaceForReplica(long bytesToReserve);
 
   /**
-   * Release disk space previously reserved for RBW block.
+   * Release disk space previously reserved for block opened for write.
    */
   void releaseReservedSpace(long bytesToRelease);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92c1af16/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 8722d35..32eb724 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -1157,7 +1157,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     
     // Replace finalized replica by a RBW replica in replicas map
     volumeMap.add(bpid, newReplicaInfo);
-    v.reserveSpaceForRbw(estimateBlockLen - replicaInfo.getNumBytes());
+    v.reserveSpaceForReplica(estimateBlockLen - replicaInfo.getNumBytes());
     return newReplicaInfo;
   }
 
@@ -1487,7 +1487,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           }
           ReplicaInPipeline newReplicaInfo =
               new ReplicaInPipeline(b.getBlockId(), b.getGenerationStamp(), v,
-                  f.getParentFile(), 0);
+                  f.getParentFile(), b.getLocalBlock().getNumBytes());
           volumeMap.add(b.getBlockPoolId(), newReplicaInfo);
           return new ReplicaHandler(newReplicaInfo, ref);
         } else {
@@ -1604,7 +1604,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     if (replicaInfo != null && replicaInfo.getState() == ReplicaState.TEMPORARY) {
       // remove from volumeMap
       volumeMap.remove(b.getBlockPoolId(), b.getLocalBlock());
-      
+
       // delete the on-disk temp file
       if (delBlockFromDisk(replicaInfo.getBlockFile(), 
           replicaInfo.getMetaFile(), b.getLocalBlock())) {
@@ -2555,14 +2555,15 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     final long usedSpace; // size of space used by HDFS
     final long freeSpace; // size of free space excluding reserved space
     final long reservedSpace; // size of space reserved for non-HDFS
-    final long reservedSpaceForRBW; // size of space reserved RBW
+    final long reservedSpaceForReplicas; // size of space reserved RBW or
+                                    // re-replication
 
     VolumeInfo(FsVolumeImpl v, long usedSpace, long freeSpace) {
       this.directory = v.toString();
       this.usedSpace = usedSpace;
       this.freeSpace = freeSpace;
       this.reservedSpace = v.getReserved();
-      this.reservedSpaceForRBW = v.getReservedForRbw();
+      this.reservedSpaceForReplicas = v.getReservedForReplicas();
     }
   }  
 
@@ -2596,7 +2597,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       innerInfo.put("usedSpace", v.usedSpace);
       innerInfo.put("freeSpace", v.freeSpace);
       innerInfo.put("reservedSpace", v.reservedSpace);
-      innerInfo.put("reservedSpaceForRBW", v.reservedSpaceForRBW);
+      innerInfo.put("reservedSpaceForReplicas", v.reservedSpaceForReplicas);
       info.put(v.directory, innerInfo);
     }
     return info;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92c1af16/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
index e90f5d2..8fd52c3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
@@ -22,8 +22,8 @@ import java.io.File;
 import java.io.FileOutputStream;
 import java.io.FilenameFilter;
 import java.io.IOException;
-import java.nio.channels.ClosedChannelException;
 import java.io.OutputStreamWriter;
+import java.nio.channels.ClosedChannelException;
 import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.nio.file.StandardCopyOption;
@@ -40,9 +40,6 @@ import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DF;
@@ -54,21 +51,24 @@ import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
-import org.apache.hadoop.util.CloseableReferenceCount;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.CloseableReferenceCount;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.util.Time;
 import org.codehaus.jackson.annotate.JsonProperty;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
 /**
  * The underlying volume used to store replica.
  * 
@@ -90,8 +90,9 @@ public class FsVolumeImpl implements FsVolumeSpi {
   private final long reserved;
   private CloseableReferenceCount reference = new CloseableReferenceCount();
 
-  // Disk space reserved for open blocks.
-  private AtomicLong reservedForRbw;
+  // Disk space reserved for blocks (RBW or Re-replicating) open for write.
+  private AtomicLong reservedForReplicas;
+  private long recentReserved = 0;
 
   // Capacity configured. This is useful when we want to
   // limit the visible capacity for tests. If negative, then we just
@@ -113,8 +114,8 @@ public class FsVolumeImpl implements FsVolumeSpi {
     this.reserved = conf.getLong(
         DFSConfigKeys.DFS_DATANODE_DU_RESERVED_KEY,
         DFSConfigKeys.DFS_DATANODE_DU_RESERVED_DEFAULT);
-    this.reservedForRbw = new AtomicLong(0L);
-    this.currentDir = currentDir; 
+    this.reservedForReplicas = new AtomicLong(0L);
+    this.currentDir = currentDir;
     File parent = currentDir.getParentFile();
     this.usage = new DF(parent, conf);
     this.storageType = storageType;
@@ -353,8 +354,9 @@ public class FsVolumeImpl implements FsVolumeSpi {
    */
   @Override
   public long getAvailable() throws IOException {
-    long remaining = getCapacity() - getDfsUsed() - reservedForRbw.get();
-    long available = usage.getAvailable() - reserved - reservedForRbw.get();
+    long remaining = getCapacity() - getDfsUsed() - reservedForReplicas.get();
+    long available = usage.getAvailable() - reserved
+        - reservedForReplicas.get();
     if (remaining > available) {
       remaining = available;
     }
@@ -362,10 +364,15 @@ public class FsVolumeImpl implements FsVolumeSpi {
   }
 
   @VisibleForTesting
-  public long getReservedForRbw() {
-    return reservedForRbw.get();
+  public long getReservedForReplicas() {
+    return reservedForReplicas.get();
   }
-    
+
+  @VisibleForTesting
+  long getRecentReserved() {
+    return recentReserved;
+  }
+
   long getReserved(){
     return reserved;
   }
@@ -412,13 +419,20 @@ public class FsVolumeImpl implements FsVolumeSpi {
    */
   File createTmpFile(String bpid, Block b) throws IOException {
     checkReference();
-    return getBlockPoolSlice(bpid).createTmpFile(b);
+    reserveSpaceForReplica(b.getNumBytes());
+    try {
+      return getBlockPoolSlice(bpid).createTmpFile(b);
+    } catch (IOException exception) {
+      releaseReservedSpace(b.getNumBytes());
+      throw exception;
+    }
   }
 
   @Override
-  public void reserveSpaceForRbw(long bytesToReserve) {
+  public void reserveSpaceForReplica(long bytesToReserve) {
     if (bytesToReserve != 0) {
-      reservedForRbw.addAndGet(bytesToReserve);
+      reservedForReplicas.addAndGet(bytesToReserve);
+      recentReserved = bytesToReserve;
     }
   }
 
@@ -428,14 +442,15 @@ public class FsVolumeImpl implements FsVolumeSpi {
 
       long oldReservation, newReservation;
       do {
-        oldReservation = reservedForRbw.get();
+        oldReservation = reservedForReplicas.get();
         newReservation = oldReservation - bytesToRelease;
         if (newReservation < 0) {
-          // Failsafe, this should never occur in practice, but if it does we don't
-          // want to start advertising more space than we have available.
+          // Failsafe, this should never occur in practice, but if it does we
+          // don't want to start advertising more space than we have available.
           newReservation = 0;
         }
-      } while (!reservedForRbw.compareAndSet(oldReservation, newReservation));
+      } while (!reservedForReplicas.compareAndSet(oldReservation,
+          newReservation));
     }
   }
 
@@ -779,7 +794,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
    */
   File createRbwFile(String bpid, Block b) throws IOException {
     checkReference();
-    reserveSpaceForRbw(b.getNumBytes());
+    reserveSpaceForReplica(b.getNumBytes());
     try {
       return getBlockPoolSlice(bpid).createRbwFile(b);
     } catch (IOException exception) {
@@ -790,16 +805,15 @@ public class FsVolumeImpl implements FsVolumeSpi {
 
   /**
    *
-   * @param bytesReservedForRbw Space that was reserved during
+   * @param bytesReserved Space that was reserved during
    *     block creation. Now that the block is being finalized we
    *     can free up this space.
    * @return
    * @throws IOException
    */
-  File addFinalizedBlock(String bpid, Block b,
-                         File f, long bytesReservedForRbw)
+  File addFinalizedBlock(String bpid, Block b, File f, long bytesReserved)
       throws IOException {
-    releaseReservedSpace(bytesReservedForRbw);
+    releaseReservedSpace(bytesReserved);
     return getBlockPoolSlice(bpid).addFinalizedBlock(b, f);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92c1af16/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index 5d1b31a..acbd8a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -492,7 +492,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     }
 
     @Override
-    public void reserveSpaceForRbw(long bytesToReserve) {
+    public void reserveSpaceForReplica(long bytesToReserve) {
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92c1af16/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
index 9b942b7..baf50d8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
@@ -612,7 +612,7 @@ public class TestDirectoryScanner {
     }
 
     @Override
-    public void reserveSpaceForRbw(long bytesToReserve) {
+    public void reserveSpaceForReplica(long bytesToReserve) {
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92c1af16/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java
index 3242ff7..985a259 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java
@@ -74,7 +74,7 @@ public class ExternalVolumeImpl implements FsVolumeSpi {
   }
 
   @Override
-  public void reserveSpaceForRbw(long bytesToReserve) {
+  public void reserveSpaceForReplica(long bytesToReserve) {
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92c1af16/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestRbwSpaceReservation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestRbwSpaceReservation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestRbwSpaceReservation.java
deleted file mode 100644
index a647d96..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestRbwSpaceReservation.java
+++ /dev/null
@@ -1,452 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
-
-import com.google.common.base.Supplier;
-import org.apache.commons.io.IOUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.commons.logging.impl.Log4JLogger;
-import org.apache.hadoop.conf.Configuration;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
-import static org.hamcrest.core.Is.is;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.*;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.util.Daemon;
-import org.apache.log4j.Level;
-import org.junit.After;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.mockito.Mockito;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.lang.management.ManagementFactory;
-import java.lang.reflect.Field;
-import java.util.Map;
-import java.util.Random;
-import java.util.concurrent.TimeoutException;
-
-import javax.management.MBeanServer;
-import javax.management.ObjectName;
-
-/**
- * Ensure that the DN reserves disk space equivalent to a full block for
- * replica being written (RBW).
- */
-public class TestRbwSpaceReservation {
-  static final Log LOG = LogFactory.getLog(TestRbwSpaceReservation.class);
-
-  private static final int DU_REFRESH_INTERVAL_MSEC = 500;
-  private static final int STORAGES_PER_DATANODE = 1;
-  private static final int BLOCK_SIZE = 1024 * 1024;
-  private static final int SMALL_BLOCK_SIZE = 1024;
-
-  protected MiniDFSCluster cluster;
-  private Configuration conf;
-  private DistributedFileSystem fs = null;
-  private DFSClient client = null;
-  FsVolumeReference singletonVolumeRef = null;
-  FsVolumeImpl singletonVolume = null;
-
-  private static Random rand = new Random();
-
-  private void initConfig(int blockSize) {
-    conf = new HdfsConfiguration();
-
-    // Refresh disk usage information frequently.
-    conf.setInt(FS_DU_INTERVAL_KEY, DU_REFRESH_INTERVAL_MSEC);
-    conf.setLong(DFS_BLOCK_SIZE_KEY, blockSize);
-
-    // Disable the scanner
-    conf.setInt(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1);
-  }
-
-  static {
-    ((Log4JLogger) FsDatasetImpl.LOG).getLogger().setLevel(Level.ALL);
-    ((Log4JLogger) DataNode.LOG).getLogger().setLevel(Level.ALL);
-  }
-
-  /**
-   *
-   * @param blockSize
-   * @param perVolumeCapacity limit the capacity of each volume to the given
-   *                          value. If negative, then don't limit.
-   * @throws IOException
-   */
-  private void startCluster(int blockSize, int numDatanodes, long perVolumeCapacity) throws IOException {
-    initConfig(blockSize);
-
-    cluster = new MiniDFSCluster
-        .Builder(conf)
-        .storagesPerDatanode(STORAGES_PER_DATANODE)
-        .numDataNodes(numDatanodes)
-        .build();
-    fs = cluster.getFileSystem();
-    client = fs.getClient();
-    cluster.waitActive();
-
-    if (perVolumeCapacity >= 0) {
-      try (FsDatasetSpi.FsVolumeReferences volumes =
-          cluster.getDataNodes().get(0).getFSDataset().getFsVolumeReferences()) {
-        singletonVolumeRef = volumes.get(0).obtainReference();
-      }
-      singletonVolume = ((FsVolumeImpl) singletonVolumeRef.getVolume());
-      singletonVolume.setCapacityForTesting(perVolumeCapacity);
-    }
-  }
-
-  @After
-  public void shutdownCluster() throws IOException {
-    if (singletonVolumeRef != null) {
-      singletonVolumeRef.close();
-      singletonVolumeRef = null;
-    }
-
-    if (client != null) {
-      client.close();
-      client = null;
-    }
-
-    if (fs != null) {
-      fs.close();
-      fs = null;
-    }
-
-    if (cluster != null) {
-      cluster.shutdown();
-      cluster = null;
-    }
-  }
-
-  private void createFileAndTestSpaceReservation(
-      final String fileNamePrefix, final int fileBlockSize)
-      throws IOException, InterruptedException {
-    // Enough for 1 block + meta files + some delta.
-    final long configuredCapacity = fileBlockSize * 2 - 1;
-    startCluster(BLOCK_SIZE, 1, configuredCapacity);
-    FSDataOutputStream out = null;
-    Path path = new Path("/" + fileNamePrefix + ".dat");
-
-    try {
-      out = fs.create(path, false, 4096, (short) 1, fileBlockSize);
-
-      byte[] buffer = new byte[rand.nextInt(fileBlockSize / 4)];
-      out.write(buffer);
-      out.hsync();
-      int bytesWritten = buffer.length;
-
-      // Check that space was reserved for a full block minus the bytesWritten.
-      assertThat(singletonVolume.getReservedForRbw(),
-                 is((long) fileBlockSize - bytesWritten));
-      out.close();
-      out = null;
-
-      // Check that the reserved space has been released since we closed the
-      // file.
-      assertThat(singletonVolume.getReservedForRbw(), is(0L));
-
-      // Reopen the file for appends and write 1 more byte.
-      out = fs.append(path);
-      out.write(buffer);
-      out.hsync();
-      bytesWritten += buffer.length;
-
-      // Check that space was again reserved for a full block minus the
-      // bytesWritten so far.
-      assertThat(singletonVolume.getReservedForRbw(),
-                 is((long) fileBlockSize - bytesWritten));
-
-      // Write once again and again verify the available space. This ensures
-      // that the reserved space is progressively adjusted to account for bytes
-      // written to disk.
-      out.write(buffer);
-      out.hsync();
-      bytesWritten += buffer.length;
-      assertThat(singletonVolume.getReservedForRbw(),
-                 is((long) fileBlockSize - bytesWritten));
-    } finally {
-      if (out != null) {
-        out.close();
-      }
-    }
-  }
-
-  @Test (timeout=300000)
-  public void testWithDefaultBlockSize()
-      throws IOException, InterruptedException {
-    createFileAndTestSpaceReservation(GenericTestUtils.getMethodName(), BLOCK_SIZE);
-  }
-
-  @Test (timeout=300000)
-  public void testWithNonDefaultBlockSize()
-      throws IOException, InterruptedException {
-    // Same test as previous one, but with a non-default block size.
-    createFileAndTestSpaceReservation(GenericTestUtils.getMethodName(), BLOCK_SIZE * 2);
-  }
-
-  @Rule
-  public ExpectedException thrown = ExpectedException.none();
-
-  @Test (timeout=300000)
-  public void testWithLimitedSpace() throws IOException {
-    // Cluster with just enough space for a full block + meta.
-    startCluster(BLOCK_SIZE, 1, 2 * BLOCK_SIZE - 1);
-    final String methodName = GenericTestUtils.getMethodName();
-    Path file1 = new Path("/" + methodName + ".01.dat");
-    Path file2 = new Path("/" + methodName + ".02.dat");
-
-    // Create two files.
-    FSDataOutputStream os1 = null, os2 = null;
-
-    try {
-      os1 = fs.create(file1);
-      os2 = fs.create(file2);
-
-      // Write one byte to the first file.
-      byte[] data = new byte[1];
-      os1.write(data);
-      os1.hsync();
-
-      // Try to write one byte to the second file.
-      // The block allocation must fail.
-      thrown.expect(RemoteException.class);
-      os2.write(data);
-      os2.hsync();
-    } finally {
-      if (os1 != null) {
-        os1.close();
-      }
-
-      // os2.close() will fail as no block was allocated.
-    }
-  }
-
-  /**
-   * Ensure that reserved space is released when the client goes away
-   * unexpectedly.
-   *
-   * The verification is done for each replica in the write pipeline.
-   *
-   * @throws IOException
-   */
-  @Test(timeout=300000)
-  public void testSpaceReleasedOnUnexpectedEof()
-      throws IOException, InterruptedException, TimeoutException {
-    final short replication = 3;
-    startCluster(BLOCK_SIZE, replication, -1);
-
-    final String methodName = GenericTestUtils.getMethodName();
-    final Path file = new Path("/" + methodName + ".01.dat");
-
-    // Write 1 byte to the file and kill the writer.
-    FSDataOutputStream os = fs.create(file, replication);
-    os.write(new byte[1]);
-    os.hsync();
-    DFSTestUtil.abortStream((DFSOutputStream) os.getWrappedStream());
-
-    // Ensure all space reserved for the replica was released on each
-    // DataNode.
-    for (DataNode dn : cluster.getDataNodes()) {
-      try (FsDatasetSpi.FsVolumeReferences volumes =
-          dn.getFSDataset().getFsVolumeReferences()) {
-        final FsVolumeImpl volume = (FsVolumeImpl) volumes.get(0);
-        GenericTestUtils.waitFor(new Supplier<Boolean>() {
-          @Override
-          public Boolean get() {
-            return (volume.getReservedForRbw() == 0);
-          }
-        }, 500, Integer.MAX_VALUE); // Wait until the test times out.
-      }
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test(timeout = 30000)
-  public void testRBWFileCreationError() throws Exception {
-
-    final short replication = 1;
-    startCluster(BLOCK_SIZE, replication, -1);
-
-    final FsVolumeImpl fsVolumeImpl = (FsVolumeImpl) cluster.getDataNodes()
-        .get(0).getFSDataset().getFsVolumeReferences().get(0);
-    final String methodName = GenericTestUtils.getMethodName();
-    final Path file = new Path("/" + methodName + ".01.dat");
-
-    // Mock BlockPoolSlice so that RBW file creation gives IOExcception
-    BlockPoolSlice blockPoolSlice = Mockito.mock(BlockPoolSlice.class);
-    Mockito.when(blockPoolSlice.createRbwFile((Block) Mockito.any()))
-        .thenThrow(new IOException("Synthetic IO Exception Throgh MOCK"));
-
-    Field field = FsVolumeImpl.class.getDeclaredField("bpSlices");
-    field.setAccessible(true);
-    Map<String, BlockPoolSlice> bpSlices = (Map<String, BlockPoolSlice>) field
-        .get(fsVolumeImpl);
-    bpSlices.put(fsVolumeImpl.getBlockPoolList()[0], blockPoolSlice);
-
-    try {
-      // Write 1 byte to the file
-      FSDataOutputStream os = fs.create(file, replication);
-      os.write(new byte[1]);
-      os.hsync();
-      os.close();
-      fail("Expecting IOException file creation failure");
-    } catch (IOException e) {
-      // Exception can be ignored (expected)
-    }
-
-    // Ensure RBW space reserved is released
-    assertTrue("Expected ZERO but got " + fsVolumeImpl.getReservedForRbw(),
-        fsVolumeImpl.getReservedForRbw() == 0);
-  }
-
-  @Test(timeout = 30000)
-  public void testRBWInJMXBean() throws Exception {
-
-    final short replication = 1;
-    startCluster(BLOCK_SIZE, replication, -1);
-
-    final String methodName = GenericTestUtils.getMethodName();
-    final Path file = new Path("/" + methodName + ".01.dat");
-
-    try (FSDataOutputStream os = fs.create(file, replication)) {
-      // Write 1 byte to the file
-      os.write(new byte[1]);
-      os.hsync();
-
-      final MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
-      final ObjectName mxbeanName = new ObjectName(
-          "Hadoop:service=DataNode,name=DataNodeInfo");
-      final String volumeInfo = (String) mbs.getAttribute(mxbeanName,
-          "VolumeInfo");
-
-      assertTrue(volumeInfo.contains("reservedSpaceForRBW"));
-    }
-  }
-
-  /**
-   * Stress test to ensure we are not leaking reserved space.
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  @Test (timeout=600000)
-  public void stressTest() throws IOException, InterruptedException {
-    final int numWriters = 5;
-    startCluster(SMALL_BLOCK_SIZE, 1, SMALL_BLOCK_SIZE * numWriters * 10);
-    Writer[] writers = new Writer[numWriters];
-
-    // Start a few writers and let them run for a while.
-    for (int i = 0; i < numWriters; ++i) {
-      writers[i] = new Writer(client, SMALL_BLOCK_SIZE);
-      writers[i].start();
-    }
-
-    Thread.sleep(60000);
-
-    // Stop the writers.
-    for (Writer w : writers) {
-      w.stopWriter();
-    }
-    int filesCreated = 0;
-    int numFailures = 0;
-    for (Writer w : writers) {
-      w.join();
-      filesCreated += w.getFilesCreated();
-      numFailures += w.getNumFailures();
-    }
-
-    LOG.info("Stress test created " + filesCreated +
-             " files and hit " + numFailures + " failures");
-
-    // Check no space was leaked.
-    assertThat(singletonVolume.getReservedForRbw(), is(0L));
-  }
-
-  private static class Writer extends Daemon {
-    private volatile boolean keepRunning;
-    private final DFSClient localClient;
-    private int filesCreated = 0;
-    private int numFailures = 0;
-    byte[] data;
-
-    Writer(DFSClient client, int blockSize) throws IOException {
-      localClient = client;
-      keepRunning = true;
-      filesCreated = 0;
-      numFailures = 0;
-
-      // At least some of the files should span a block boundary.
-      data = new byte[blockSize * 2];
-    }
-
-    @Override
-    public void run() {
-      /**
-       * Create a file, write up to 3 blocks of data and close the file.
-       * Do this in a loop until we are told to stop.
-       */
-      while (keepRunning) {
-        OutputStream os = null;
-        try {
-          String filename = "/file-" + rand.nextLong();
-          os = localClient.create(filename, false);
-          os.write(data, 0, rand.nextInt(data.length));
-          IOUtils.closeQuietly(os);
-          os = null;
-          localClient.delete(filename, false);
-          Thread.sleep(50);     // Sleep for a bit to avoid killing the system.
-          ++filesCreated;
-        } catch (IOException ioe) {
-          // Just ignore the exception and keep going.
-          ++numFailures;
-        } catch (InterruptedException ie) {
-          return;
-        } finally {
-          if (os != null) {
-            IOUtils.closeQuietly(os);
-          }
-        }
-      }
-    }
-
-    public void stopWriter() {
-      keepRunning = false;
-    }
-
-    public int getFilesCreated() {
-      return filesCreated;
-    }
-
-    public int getNumFailures() {
-      return numFailures;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92c1af16/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestSpaceReservation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestSpaceReservation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestSpaceReservation.java
new file mode 100644
index 0000000..c494288
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestSpaceReservation.java
@@ -0,0 +1,576 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+
+import com.google.common.base.Supplier;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.*;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Daemon;
+import org.apache.log4j.Level;
+import org.junit.After;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.lang.management.ManagementFactory;
+import java.lang.reflect.Field;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.TimeoutException;
+
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+
+/**
+ * Ensure that the DN reserves disk space equivalent to a full block for
+ * replica being written (RBW) & Replica being copied from another DN.
+ */
+public class TestSpaceReservation {
+  static final Log LOG = LogFactory.getLog(TestSpaceReservation.class);
+
+  private static final int DU_REFRESH_INTERVAL_MSEC = 500;
+  private static final int STORAGES_PER_DATANODE = 1;
+  private static final int BLOCK_SIZE = 1024 * 1024;
+  private static final int SMALL_BLOCK_SIZE = 1024;
+
+  protected MiniDFSCluster cluster;
+  private Configuration conf;
+  private DistributedFileSystem fs = null;
+  private DFSClient client = null;
+  FsVolumeReference singletonVolumeRef = null;
+  FsVolumeImpl singletonVolume = null;
+
+  private static Random rand = new Random();
+
+  private void initConfig(int blockSize) {
+    conf = new HdfsConfiguration();
+
+    // Refresh disk usage information frequently.
+    conf.setInt(FS_DU_INTERVAL_KEY, DU_REFRESH_INTERVAL_MSEC);
+    conf.setLong(DFS_BLOCK_SIZE_KEY, blockSize);
+
+    // Disable the scanner
+    conf.setInt(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1);
+  }
+
+  static {
+    ((Log4JLogger) FsDatasetImpl.LOG).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger) DataNode.LOG).getLogger().setLevel(Level.ALL);
+  }
+
+  /**
+   *
+   * @param blockSize
+   * @param perVolumeCapacity limit the capacity of each volume to the given
+   *                          value. If negative, then don't limit.
+   * @throws IOException
+   */
+  private void startCluster(int blockSize, int numDatanodes, long perVolumeCapacity) throws IOException {
+    initConfig(blockSize);
+
+    cluster = new MiniDFSCluster
+        .Builder(conf)
+        .storagesPerDatanode(STORAGES_PER_DATANODE)
+        .numDataNodes(numDatanodes)
+        .build();
+    fs = cluster.getFileSystem();
+    client = fs.getClient();
+    cluster.waitActive();
+
+    if (perVolumeCapacity >= 0) {
+      try (FsDatasetSpi.FsVolumeReferences volumes =
+          cluster.getDataNodes().get(0).getFSDataset().getFsVolumeReferences()) {
+        singletonVolumeRef = volumes.get(0).obtainReference();
+      }
+      singletonVolume = ((FsVolumeImpl) singletonVolumeRef.getVolume());
+      singletonVolume.setCapacityForTesting(perVolumeCapacity);
+    }
+  }
+
+  @After
+  public void shutdownCluster() throws IOException {
+    if (singletonVolumeRef != null) {
+      singletonVolumeRef.close();
+      singletonVolumeRef = null;
+    }
+
+    if (client != null) {
+      client.close();
+      client = null;
+    }
+
+    if (fs != null) {
+      fs.close();
+      fs = null;
+    }
+
+    if (cluster != null) {
+      cluster.shutdown();
+      cluster = null;
+    }
+  }
+
+  private void createFileAndTestSpaceReservation(
+      final String fileNamePrefix, final int fileBlockSize)
+      throws IOException, InterruptedException {
+    // Enough for 1 block + meta files + some delta.
+    final long configuredCapacity = fileBlockSize * 2 - 1;
+    startCluster(BLOCK_SIZE, 1, configuredCapacity);
+    FSDataOutputStream out = null;
+    Path path = new Path("/" + fileNamePrefix + ".dat");
+
+    try {
+      out = fs.create(path, false, 4096, (short) 1, fileBlockSize);
+
+      byte[] buffer = new byte[rand.nextInt(fileBlockSize / 4)];
+      out.write(buffer);
+      out.hsync();
+      int bytesWritten = buffer.length;
+
+      // Check that space was reserved for a full block minus the bytesWritten.
+      assertThat(singletonVolume.getReservedForReplicas(),
+                 is((long) fileBlockSize - bytesWritten));
+      out.close();
+      out = null;
+
+      // Check that the reserved space has been released since we closed the
+      // file.
+      assertThat(singletonVolume.getReservedForReplicas(), is(0L));
+
+      // Reopen the file for appends and write 1 more byte.
+      out = fs.append(path);
+      out.write(buffer);
+      out.hsync();
+      bytesWritten += buffer.length;
+
+      // Check that space was again reserved for a full block minus the
+      // bytesWritten so far.
+      assertThat(singletonVolume.getReservedForReplicas(),
+                 is((long) fileBlockSize - bytesWritten));
+
+      // Write once again and again verify the available space. This ensures
+      // that the reserved space is progressively adjusted to account for bytes
+      // written to disk.
+      out.write(buffer);
+      out.hsync();
+      bytesWritten += buffer.length;
+      assertThat(singletonVolume.getReservedForReplicas(),
+                 is((long) fileBlockSize - bytesWritten));
+    } finally {
+      if (out != null) {
+        out.close();
+      }
+    }
+  }
+
+  @Test (timeout=300000)
+  public void testWithDefaultBlockSize()
+      throws IOException, InterruptedException {
+    createFileAndTestSpaceReservation(GenericTestUtils.getMethodName(), BLOCK_SIZE);
+  }
+
+  @Test (timeout=300000)
+  public void testWithNonDefaultBlockSize()
+      throws IOException, InterruptedException {
+    // Same test as previous one, but with a non-default block size.
+    createFileAndTestSpaceReservation(GenericTestUtils.getMethodName(), BLOCK_SIZE * 2);
+  }
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Test (timeout=300000)
+  public void testWithLimitedSpace() throws IOException {
+    // Cluster with just enough space for a full block + meta.
+    startCluster(BLOCK_SIZE, 1, 2 * BLOCK_SIZE - 1);
+    final String methodName = GenericTestUtils.getMethodName();
+    Path file1 = new Path("/" + methodName + ".01.dat");
+    Path file2 = new Path("/" + methodName + ".02.dat");
+
+    // Create two files.
+    FSDataOutputStream os1 = null, os2 = null;
+
+    try {
+      os1 = fs.create(file1);
+      os2 = fs.create(file2);
+
+      // Write one byte to the first file.
+      byte[] data = new byte[1];
+      os1.write(data);
+      os1.hsync();
+
+      // Try to write one byte to the second file.
+      // The block allocation must fail.
+      thrown.expect(RemoteException.class);
+      os2.write(data);
+      os2.hsync();
+    } finally {
+      if (os1 != null) {
+        os1.close();
+      }
+
+      // os2.close() will fail as no block was allocated.
+    }
+  }
+
+  /**
+   * Ensure that reserved space is released when the client goes away
+   * unexpectedly.
+   *
+   * The verification is done for each replica in the write pipeline.
+   *
+   * @throws IOException
+   */
+  @Test(timeout=300000)
+  public void testSpaceReleasedOnUnexpectedEof()
+      throws IOException, InterruptedException, TimeoutException {
+    final short replication = 3;
+    startCluster(BLOCK_SIZE, replication, -1);
+
+    final String methodName = GenericTestUtils.getMethodName();
+    final Path file = new Path("/" + methodName + ".01.dat");
+
+    // Write 1 byte to the file and kill the writer.
+    FSDataOutputStream os = fs.create(file, replication);
+    os.write(new byte[1]);
+    os.hsync();
+    DFSTestUtil.abortStream((DFSOutputStream) os.getWrappedStream());
+
+    // Ensure all space reserved for the replica was released on each
+    // DataNode.
+    for (DataNode dn : cluster.getDataNodes()) {
+      try (FsDatasetSpi.FsVolumeReferences volumes =
+          dn.getFSDataset().getFsVolumeReferences()) {
+        final FsVolumeImpl volume = (FsVolumeImpl) volumes.get(0);
+        GenericTestUtils.waitFor(new Supplier<Boolean>() {
+          @Override
+          public Boolean get() {
+            return (volume.getReservedForReplicas() == 0);
+          }
+        }, 500, Integer.MAX_VALUE); // Wait until the test times out.
+      }
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test(timeout = 30000)
+  public void testRBWFileCreationError() throws Exception {
+
+    final short replication = 1;
+    startCluster(BLOCK_SIZE, replication, -1);
+
+    final FsVolumeImpl fsVolumeImpl = (FsVolumeImpl) cluster.getDataNodes()
+        .get(0).getFSDataset().getFsVolumeReferences().get(0);
+    final String methodName = GenericTestUtils.getMethodName();
+    final Path file = new Path("/" + methodName + ".01.dat");
+
+    // Mock BlockPoolSlice so that RBW file creation gives IOExcception
+    BlockPoolSlice blockPoolSlice = Mockito.mock(BlockPoolSlice.class);
+    Mockito.when(blockPoolSlice.createRbwFile((Block) Mockito.any()))
+        .thenThrow(new IOException("Synthetic IO Exception Throgh MOCK"));
+
+    Field field = FsVolumeImpl.class.getDeclaredField("bpSlices");
+    field.setAccessible(true);
+    Map<String, BlockPoolSlice> bpSlices = (Map<String, BlockPoolSlice>) field
+        .get(fsVolumeImpl);
+    bpSlices.put(fsVolumeImpl.getBlockPoolList()[0], blockPoolSlice);
+
+    try {
+      // Write 1 byte to the file
+      FSDataOutputStream os = fs.create(file, replication);
+      os.write(new byte[1]);
+      os.hsync();
+      os.close();
+      fail("Expecting IOException file creation failure");
+    } catch (IOException e) {
+      // Exception can be ignored (expected)
+    }
+
+    // Ensure RBW space reserved is released
+    assertTrue(
+        "Expected ZERO but got " + fsVolumeImpl.getReservedForReplicas(),
+        fsVolumeImpl.getReservedForReplicas() == 0);
+
+    // Reserve some bytes to verify double clearing space should't happen
+    fsVolumeImpl.reserveSpaceForReplica(1000);
+    try {
+      // Write 1 byte to the file
+      FSDataOutputStream os = fs.create(new Path("/" + methodName + ".02.dat"),
+          replication);
+      os.write(new byte[1]);
+      os.hsync();
+      os.close();
+      fail("Expecting IOException file creation failure");
+    } catch (IOException e) {
+      // Exception can be ignored (expected)
+    }
+
+    // Ensure RBW space reserved is released only once
+    assertTrue(fsVolumeImpl.getReservedForReplicas() == 1000);
+  }
+
+  @Test(timeout = 30000)
+  public void testReservedSpaceInJMXBean() throws Exception {
+
+    final short replication = 1;
+    startCluster(BLOCK_SIZE, replication, -1);
+
+    final String methodName = GenericTestUtils.getMethodName();
+    final Path file = new Path("/" + methodName + ".01.dat");
+
+    try (FSDataOutputStream os = fs.create(file, replication)) {
+      // Write 1 byte to the file
+      os.write(new byte[1]);
+      os.hsync();
+
+      final MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+      final ObjectName mxbeanName = new ObjectName(
+          "Hadoop:service=DataNode,name=DataNodeInfo");
+      final String volumeInfo = (String) mbs.getAttribute(mxbeanName,
+          "VolumeInfo");
+
+      // verify reserved space for Replicas in JMX bean volume info
+      assertTrue(volumeInfo.contains("reservedSpaceForReplicas"));
+    }
+  }
+
+  @Test(timeout = 300000)
+  public void testTmpSpaceReserve() throws Exception {
+
+    final short replication = 2;
+    startCluster(BLOCK_SIZE, replication, -1);
+    final int byteCount1 = 100;
+    final int byteCount2 = 200;
+
+    final String methodName = GenericTestUtils.getMethodName();
+
+    // Test positive scenario
+    {
+      final Path file = new Path("/" + methodName + ".01.dat");
+
+      try (FSDataOutputStream os = fs.create(file, (short) 1)) {
+        // Write test data to the file
+        os.write(new byte[byteCount1]);
+        os.hsync();
+      }
+
+      BlockLocation[] blockLocations = fs.getFileBlockLocations(file, 0, 10);
+      String firstReplicaNode = blockLocations[0].getNames()[0];
+
+      int newReplicaDNIndex = 0;
+      if (firstReplicaNode.equals(cluster.getDataNodes().get(0)
+          .getDisplayName())) {
+        newReplicaDNIndex = 1;
+      }
+
+      FsVolumeImpl fsVolumeImpl = (FsVolumeImpl) cluster.getDataNodes()
+          .get(newReplicaDNIndex).getFSDataset().getFsVolumeReferences().get(0);
+
+      performReReplication(file, true);
+
+      assertEquals("Wrong reserve space for Tmp ", byteCount1,
+          fsVolumeImpl.getRecentReserved());
+
+      assertEquals("Reserved Tmp space is not released", 0,
+          fsVolumeImpl.getReservedForReplicas());
+    }
+
+    // Test when file creation fails
+    {
+      final Path file = new Path("/" + methodName + ".01.dat");
+
+      try (FSDataOutputStream os = fs.create(file, (short) 1)) {
+        // Write test data to the file
+        os.write(new byte[byteCount2]);
+        os.hsync();
+      }
+
+      BlockLocation[] blockLocations = fs.getFileBlockLocations(file, 0, 10);
+      String firstReplicaNode = blockLocations[0].getNames()[0];
+
+      int newReplicaDNIndex = 0;
+      if (firstReplicaNode.equals(cluster.getDataNodes().get(0)
+          .getDisplayName())) {
+        newReplicaDNIndex = 1;
+      }
+
+      BlockPoolSlice blockPoolSlice = Mockito.mock(BlockPoolSlice.class);
+      Mockito.when(blockPoolSlice.createTmpFile((Block) Mockito.any()))
+          .thenThrow(new IOException("Synthetic IO Exception Throgh MOCK"));
+
+      final FsVolumeImpl fsVolumeImpl = (FsVolumeImpl) cluster.getDataNodes()
+          .get(newReplicaDNIndex).getFSDataset().getFsVolumeReferences().get(0);
+
+      // Reserve some bytes to verify double clearing space should't happen
+      fsVolumeImpl.reserveSpaceForReplica(1000);
+
+      Field field = FsVolumeImpl.class.getDeclaredField("bpSlices");
+      field.setAccessible(true);
+      @SuppressWarnings("unchecked")
+      Map<String, BlockPoolSlice> bpSlices = (Map<String, BlockPoolSlice>) field
+          .get(fsVolumeImpl);
+      bpSlices.put(fsVolumeImpl.getBlockPoolList()[0], blockPoolSlice);
+
+      performReReplication(file, false);
+
+      assertEquals("Wrong reserve space for Tmp ", byteCount2,
+          fsVolumeImpl.getRecentReserved());
+
+      assertEquals("Tmp space is not released OR released twice", 1000,
+          fsVolumeImpl.getReservedForReplicas());
+    }
+  }
+
+  private void performReReplication(Path filePath, boolean waitForSuccess)
+      throws Exception {
+    fs.setReplication(filePath, (short) 2);
+
+    Thread.sleep(4000);
+    BlockLocation[] blockLocations = fs.getFileBlockLocations(filePath, 0, 10);
+
+    if (waitForSuccess) {
+      // Wait for the re replication
+      while (blockLocations[0].getNames().length < 2) {
+        Thread.sleep(2000);
+        blockLocations = fs.getFileBlockLocations(filePath, 0, 10);
+      }
+    }
+  }
+
+  /**
+   * Stress test to ensure we are not leaking reserved space.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @Test (timeout=600000)
+  public void stressTest() throws IOException, InterruptedException {
+    final int numWriters = 5;
+    startCluster(SMALL_BLOCK_SIZE, 1, SMALL_BLOCK_SIZE * numWriters * 10);
+    Writer[] writers = new Writer[numWriters];
+
+    // Start a few writers and let them run for a while.
+    for (int i = 0; i < numWriters; ++i) {
+      writers[i] = new Writer(client, SMALL_BLOCK_SIZE);
+      writers[i].start();
+    }
+
+    Thread.sleep(60000);
+
+    // Stop the writers.
+    for (Writer w : writers) {
+      w.stopWriter();
+    }
+    int filesCreated = 0;
+    int numFailures = 0;
+    for (Writer w : writers) {
+      w.join();
+      filesCreated += w.getFilesCreated();
+      numFailures += w.getNumFailures();
+    }
+
+    LOG.info("Stress test created " + filesCreated +
+             " files and hit " + numFailures + " failures");
+
+    // Check no space was leaked.
+    assertThat(singletonVolume.getReservedForReplicas(), is(0L));
+  }
+
+  private static class Writer extends Daemon {
+    private volatile boolean keepRunning;
+    private final DFSClient localClient;
+    private int filesCreated = 0;
+    private int numFailures = 0;
+    byte[] data;
+
+    Writer(DFSClient client, int blockSize) throws IOException {
+      localClient = client;
+      keepRunning = true;
+      filesCreated = 0;
+      numFailures = 0;
+
+      // At least some of the files should span a block boundary.
+      data = new byte[blockSize * 2];
+    }
+
+    @Override
+    public void run() {
+      /**
+       * Create a file, write up to 3 blocks of data and close the file.
+       * Do this in a loop until we are told to stop.
+       */
+      while (keepRunning) {
+        OutputStream os = null;
+        try {
+          String filename = "/file-" + rand.nextLong();
+          os = localClient.create(filename, false);
+          os.write(data, 0, rand.nextInt(data.length));
+          IOUtils.closeQuietly(os);
+          os = null;
+          localClient.delete(filename, false);
+          Thread.sleep(50);     // Sleep for a bit to avoid killing the system.
+          ++filesCreated;
+        } catch (IOException ioe) {
+          // Just ignore the exception and keep going.
+          ++numFailures;
+        } catch (InterruptedException ie) {
+          return;
+        } finally {
+          if (os != null) {
+            IOUtils.closeQuietly(os);
+          }
+        }
+      }
+    }
+
+    public void stopWriter() {
+      keepRunning = false;
+    }
+
+    public int getFilesCreated() {
+      return filesCreated;
+    }
+
+    public int getNumFailures() {
+      return numFailures;
+    }
+  }
+}


[06/50] [abbrv] hadoop git commit: YARN-4078. Add getPendingResourceRequestForAttempt in YarnScheduler interface. Contributed by Naganarasimha G R

Posted by zh...@apache.org.
YARN-4078. Add getPendingResourceRequestForAttempt in YarnScheduler interface. Contributed by Naganarasimha G R


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

Branch: refs/heads/HDFS-7285
Commit: 452079af8bc56195945e28b8cf76620f0aca01c3
Parents: 2ffe2db
Author: Jian He <ji...@apache.org>
Authored: Wed Sep 16 14:58:32 2015 +0800
Committer: Jian He <ji...@apache.org>
Committed: Wed Sep 16 14:59:20 2015 +0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +++
 .../scheduler/AbstractYarnScheduler.java        |  1 +
 .../scheduler/YarnScheduler.java                |  6 ++++++
 .../resourcemanager/webapp/dao/AppInfo.java     |  6 ++----
 .../TestWorkPreservingRMRestart.java            | 21 ++++++--------------
 .../scheduler/TestAbstractYarnScheduler.java    | 14 ++++++-------
 .../capacity/TestCapacityScheduler.java         | 10 ++++------
 7 files changed, 29 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/452079af/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index b044078..c5143ff 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -842,6 +842,9 @@ Release 2.8.0 - UNRELEASED
     YARN-4151. Fix findbugs errors in hadoop-yarn-server-common module.
     (Meng Ding via wangda)
 
+    YARN-4078. Add getPendingResourceRequestForAttempt in YarnScheduler interface.
+    (Naganarasimha G R via jianhe)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/452079af/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index 0b6b8ef..4c34c47 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -687,6 +687,7 @@ public abstract class AbstractYarnScheduler
     }
   }
 
+  @Override
   public List<ResourceRequest> getPendingResourceRequestsForAttempt(
       ApplicationAttemptId attemptId) {
     SchedulerApplicationAttempt attempt = getApplicationAttempt(attemptId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/452079af/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
index f2753e6..e3c79f7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
@@ -337,4 +337,10 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
    * @throws YarnException
    */
   void setClusterMaxPriority(Configuration conf) throws YarnException;
+
+  /**
+   * @param attemptId
+   */
+  List<ResourceRequest> getPendingResourceRequestsForAttempt(
+      ApplicationAttemptId attemptId);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/452079af/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
index 7f80315..a035aeb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
@@ -37,7 +37,6 @@ 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.RMAppMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
@@ -177,9 +176,8 @@ public class AppInfo {
             allocatedVCores = usedResources.getVirtualCores();
             runningContainers = resourceReport.getNumUsedContainers();
           }
-          resourceRequests =
-              ((AbstractYarnScheduler) rm.getRMContext().getScheduler())
-                .getPendingResourceRequestsForAttempt(attempt.getAppAttemptId());
+          resourceRequests = rm.getRMContext().getScheduler()
+              .getPendingResourceRequestsForAttempt(attempt.getAppAttemptId());
         }
       }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/452079af/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
index 77e9838..a9a88da 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
@@ -23,14 +23,10 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
-import java.io.File;
-import java.io.FileWriter;
 import java.io.IOException;
-import java.io.PrintWriter;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -62,10 +58,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueNotFoundException;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
@@ -73,9 +69,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQu
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSParentQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
 import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.Records;
@@ -616,8 +610,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     // Wait for RM to settle down on recovering containers;
     Thread.sleep(3000);
 
-    AbstractYarnScheduler scheduler =
-        (AbstractYarnScheduler) rm2.getResourceScheduler();
+    YarnScheduler scheduler = rm2.getResourceScheduler();
     // Previous AM failed, The failed AM should once again release the
     // just-recovered containers.
     assertNull(scheduler.getRMContainer(runningContainer.getContainerId()));
@@ -669,8 +662,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     // Wait for RM to settle down on recovering containers;
     Thread.sleep(3000);
 
-    AbstractYarnScheduler scheduler =
-        (AbstractYarnScheduler) rm2.getResourceScheduler();
+    YarnScheduler scheduler = rm2.getResourceScheduler();
 
     // scheduler should not recover containers for finished apps.
     assertNull(scheduler.getRMContainer(runningContainer.getContainerId()));
@@ -724,9 +716,8 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     MockAM am1_1 = MockRM.launchAndRegisterAM(app1_1, rm1, nm1);
     
     RMAppAttempt attempt0 = app1_1.getCurrentAppAttempt();
-    AbstractYarnScheduler scheduler =
-        ((AbstractYarnScheduler) rm1.getResourceScheduler());
-    
+    YarnScheduler scheduler = rm1.getResourceScheduler();
+
     Assert.assertTrue(scheduler.getRMContainer(
         attempt0.getMasterContainer().getId()).isAMContainer());
 
@@ -742,7 +733,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     // Wait for RM to settle down on recovering containers;
     waitForNumContainersToRecover(2, rm2, am1_1.getApplicationAttemptId());
 
-    scheduler = ((AbstractYarnScheduler) rm2.getResourceScheduler());
+    scheduler = rm2.getResourceScheduler();
     Assert.assertTrue(scheduler.getRMContainer(
         attempt0.getMasterContainer().getId()).isAMContainer());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/452079af/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAbstractYarnScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAbstractYarnScheduler.java
index ffd1c1f..7c33f78 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAbstractYarnScheduler.java
@@ -79,7 +79,7 @@ public class TestAbstractYarnScheduler extends ParameterizedSchedulerTestBase {
     try {
       rm.start();
       testMaximumAllocationMemoryHelper(
-          (AbstractYarnScheduler) rm.getResourceScheduler(),
+          rm.getResourceScheduler(),
           node1MaxMemory, node2MaxMemory, node3MaxMemory,
           configuredMaxMemory, configuredMaxMemory, configuredMaxMemory,
           configuredMaxMemory, configuredMaxMemory, configuredMaxMemory);
@@ -94,7 +94,7 @@ public class TestAbstractYarnScheduler extends ParameterizedSchedulerTestBase {
     try {
       rm.start();
       testMaximumAllocationMemoryHelper(
-          (AbstractYarnScheduler) rm.getResourceScheduler(),
+          rm.getResourceScheduler(),
           node1MaxMemory, node2MaxMemory, node3MaxMemory,
           configuredMaxMemory, configuredMaxMemory, configuredMaxMemory,
           node2MaxMemory, node3MaxMemory, node2MaxMemory);
@@ -104,7 +104,7 @@ public class TestAbstractYarnScheduler extends ParameterizedSchedulerTestBase {
   }
 
   private void testMaximumAllocationMemoryHelper(
-       AbstractYarnScheduler scheduler,
+       YarnScheduler scheduler,
        final int node1MaxMemory, final int node2MaxMemory,
        final int node3MaxMemory, final int... expectedMaxMemory)
        throws Exception {
@@ -166,7 +166,7 @@ public class TestAbstractYarnScheduler extends ParameterizedSchedulerTestBase {
     try {
       rm.start();
       testMaximumAllocationVCoresHelper(
-          (AbstractYarnScheduler) rm.getResourceScheduler(),
+          rm.getResourceScheduler(),
           node1MaxVCores, node2MaxVCores, node3MaxVCores,
           configuredMaxVCores, configuredMaxVCores, configuredMaxVCores,
           configuredMaxVCores, configuredMaxVCores, configuredMaxVCores);
@@ -181,7 +181,7 @@ public class TestAbstractYarnScheduler extends ParameterizedSchedulerTestBase {
     try {
       rm.start();
       testMaximumAllocationVCoresHelper(
-          (AbstractYarnScheduler) rm.getResourceScheduler(),
+          rm.getResourceScheduler(),
           node1MaxVCores, node2MaxVCores, node3MaxVCores,
           configuredMaxVCores, configuredMaxVCores, configuredMaxVCores,
           node2MaxVCores, node3MaxVCores, node2MaxVCores);
@@ -191,7 +191,7 @@ public class TestAbstractYarnScheduler extends ParameterizedSchedulerTestBase {
   }
 
   private void testMaximumAllocationVCoresHelper(
-      AbstractYarnScheduler scheduler,
+      YarnScheduler scheduler,
       final int node1MaxVCores, final int node2MaxVCores,
       final int node3MaxVCores, final int... expectedMaxVCores)
       throws Exception {
@@ -494,7 +494,7 @@ public class TestAbstractYarnScheduler extends ParameterizedSchedulerTestBase {
   }
 
   private void verifyMaximumResourceCapability(
-      Resource expectedMaximumResource, AbstractYarnScheduler scheduler) {
+      Resource expectedMaximumResource, YarnScheduler scheduler) {
 
     final Resource schedulerMaximumResourceCapability = scheduler
         .getMaximumResourceCapability();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/452079af/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
index 76a1351..fb7fce4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
@@ -113,6 +113,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicat
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
@@ -1950,8 +1951,7 @@ public class TestCapacityScheduler {
   @Test
   public void testMoveAllAppsInvalidDestination() throws Exception {
     MockRM rm = setUpMove();
-    AbstractYarnScheduler scheduler =
-        (AbstractYarnScheduler) rm.getResourceScheduler();
+    YarnScheduler scheduler = rm.getResourceScheduler();
 
     // submit an app
     RMApp app = rm.submitApp(GB, "test-move-1", "user_0", null, "a1");
@@ -2009,8 +2009,7 @@ public class TestCapacityScheduler {
   @Test
   public void testMoveAllAppsInvalidSource() throws Exception {
     MockRM rm = setUpMove();
-    AbstractYarnScheduler scheduler =
-        (AbstractYarnScheduler) rm.getResourceScheduler();
+    YarnScheduler scheduler = rm.getResourceScheduler();
 
     // submit an app
     RMApp app = rm.submitApp(GB, "test-move-1", "user_0", null, "a1");
@@ -2113,8 +2112,7 @@ public class TestCapacityScheduler {
   @Test
   public void testKillAllAppsInvalidSource() throws Exception {
     MockRM rm = setUpMove();
-    AbstractYarnScheduler scheduler =
-        (AbstractYarnScheduler) rm.getResourceScheduler();
+    YarnScheduler scheduler = rm.getResourceScheduler();
 
     // submit an app
     RMApp app = rm.submitApp(GB, "test-move-1", "user_0", null, "a1");


[07/50] [abbrv] hadoop git commit: HDFS-9072. Fix random failures in TestJMXGet. (J.Andreina via stevel)

Posted by zh...@apache.org.
HDFS-9072. Fix random failures in TestJMXGet.  (J.Andreina via stevel)


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

Branch: refs/heads/HDFS-7285
Commit: 44776437493db3dc7796fcd23a0301bc48a1781c
Parents: 452079a
Author: Steve Loughran <st...@apache.org>
Authored: Wed Sep 16 10:08:18 2015 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Sep 16 10:08:33 2015 +0100

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../src/test/java/org/apache/hadoop/tools/TestJMXGet.java         | 3 +++
 2 files changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/44776437/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 a3b9b44..589c50c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1352,6 +1352,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9067. o.a.h.hdfs.server.datanode.fsdataset.impl.TestLazyWriter
     is failing in trunk (Surendra Singh Lilhore via vinayakumarb)
 
+    HDFS-9072. Fix random failures in TestJMXGet.
+    (J.Andreina via stevel)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44776437/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java
index eb439f4..370b650 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java
@@ -103,10 +103,12 @@ public class TestJMXGet {
     assertTrue("error printAllValues", checkPrintAllValues(jmx));
 
     //get some data from different source
+    DFSTestUtil.waitForMetric(jmx, "NumLiveDataNodes", numDatanodes);
     assertEquals(numDatanodes, Integer.parseInt(
         jmx.getValue("NumLiveDataNodes")));
     assertGauge("CorruptBlocks", Long.parseLong(jmx.getValue("CorruptBlocks")),
                 getMetrics("FSNamesystem"));
+    DFSTestUtil.waitForMetric(jmx, "NumOpenConnections", numDatanodes);
     assertEquals(numDatanodes, Integer.parseInt(
         jmx.getValue("NumOpenConnections")));
 
@@ -151,6 +153,7 @@ public class TestJMXGet {
     String serviceName = "DataNode";
     jmx.setService(serviceName);
     jmx.init();
+    DFSTestUtil.waitForMetric(jmx, "BytesWritten", fileSize);
     assertEquals(fileSize, Integer.parseInt(jmx.getValue("BytesWritten")));
 
     cluster.shutdown();


[12/50] [abbrv] hadoop git commit: HADOOP-12417. TestWebDelegationToken failing with port in use. Contributed by Mingliang Liu.

Posted by zh...@apache.org.
HADOOP-12417. TestWebDelegationToken failing with port in use. Contributed by Mingliang Liu.


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

Branch: refs/heads/HDFS-7285
Commit: 0832b38d482238e87e2529d80d09881a5c90ccff
Parents: 941001b
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Sep 16 18:38:37 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Wed Sep 16 18:38:37 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt               | 3 +++
 .../security/token/delegation/web/TestWebDelegationToken.java | 7 +------
 2 files changed, 4 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0832b38d/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 8ae0128..ecd8833 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1129,6 +1129,9 @@ Release 2.8.0 - UNRELEASED
 
     HADOOP-12407. Test failing: hadoop.ipc.TestSaslRPC. (stevel)
 
+    HADOOP-12417. TestWebDelegationToken failing with port in use.
+    (Mingliang Liu via wheat9)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0832b38d/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java
index d18f968..73562b5 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java
@@ -176,13 +176,8 @@ public class TestWebDelegationToken {
 
   protected Server createJettyServer() {
     try {
-      InetAddress localhost = InetAddress.getLocalHost();
-      ServerSocket ss = new ServerSocket(0, 50, localhost);
-      int port = ss.getLocalPort();
-      ss.close();
       jetty = new Server(0);
       jetty.getConnectors()[0].setHost("localhost");
-      jetty.getConnectors()[0].setPort(port);
       return jetty;
     } catch (Exception ex) {
       throw new RuntimeException("Could not setup Jetty: " + ex.getMessage(),
@@ -192,7 +187,7 @@ public class TestWebDelegationToken {
 
   protected String getJettyURL() {
     Connector c = jetty.getConnectors()[0];
-    return "http://" + c.getHost() + ":" + c.getPort();
+    return "http://" + c.getHost() + ":" + c.getLocalPort();
   }
 
   @Before


[40/50] [abbrv] hadoop git commit: HDFS-9111. Move hdfs-client protobuf convert methods from PBHelper to PBHelperClient. Contributed by Mingliang Liu.

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java
index 6d9dc23..740b5cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.AcceptRecoveryRequestProto;
@@ -161,7 +162,7 @@ public class QJournalProtocolTranslatorPB implements ProtocolMetaInterface,
         .setSegmentTxnId(segmentTxId)
         .setFirstTxnId(firstTxnId)
         .setNumTxns(numTxns)
-        .setRecords(PBHelper.getByteString(records))
+        .setRecords(PBHelperClient.getByteString(records))
         .build();
     try {
       rpcProxy.journal(NULL_CONTROLLER, req);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
index e09ba32..3559065 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
@@ -65,7 +65,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
-import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@@ -1048,7 +1048,7 @@ public final class CacheManager {
       Expiration expiry = info.getExpiration();
       if (expiry != null) {
         assert (!expiry.isRelative());
-        b.setExpiration(PBHelper.convert(expiry));
+        b.setExpiration(PBHelperClient.convert(expiry));
       }
 
       directives.add(b.build());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
index 7c3c895..0663b8e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
-import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -305,7 +305,7 @@ public class EncryptionZoneManager {
     }
 
     final HdfsProtos.ZoneEncryptionInfoProto proto =
-        PBHelper.convert(suite, version, keyName);
+        PBHelperClient.convert(suite, version, keyName);
     final XAttr ezXAttr = XAttrHelper
         .buildXAttr(CRYPTO_XATTR_ENCRYPTION_ZONE, proto.toByteArray());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
index ced085a..24d3360 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
@@ -29,7 +29,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
-import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.security.AccessControlException;
 
@@ -284,7 +283,7 @@ class FSDirXAttrOp {
             HdfsProtos.ZoneEncryptionInfoProto.parseFrom(xattr.getValue());
         fsd.ezManager.addEncryptionZone(inode.getId(),
             PBHelperClient.convert(ezProto.getSuite()),
-            PBHelper.convert(ezProto.getCryptoProtocolVersion()),
+            PBHelperClient.convert(ezProto.getCryptoProtocolVersion()),
             ezProto.getKeyName());
       }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index e25e0e0..4dc5326 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -48,7 +48,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
-import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -1134,7 +1133,7 @@ public class FSDirectory implements Closeable {
                       xattr.getValue());
               ezManager.unprotectedAddEncryptionZone(inode.getId(),
                   PBHelperClient.convert(ezProto.getSuite()),
-                  PBHelper.convert(ezProto.getCryptoProtocolVersion()),
+                  PBHelperClient.convert(ezProto.getCryptoProtocolVersion()),
                   ezProto.getKeyName());
             } catch (InvalidProtocolBufferException e) {
               NameNode.LOG.warn("Error parsing protocol buffer of " +
@@ -1261,7 +1260,7 @@ public class FSDirectory implements Closeable {
       throws IOException {
     // Make the PB for the xattr
     final HdfsProtos.PerFileEncryptionInfoProto proto =
-        PBHelper.convertPerFileEncInfo(info);
+        PBHelperClient.convertPerFileEncInfo(info);
     final byte[] protoBytes = proto.toByteArray();
     final XAttr fileEncryptionAttr =
         XAttrHelper.buildXAttr(CRYPTO_XATTR_FILE_ENCRYPTION_INFO, protoBytes);
@@ -1327,7 +1326,7 @@ public class FSDirectory implements Closeable {
         HdfsProtos.PerFileEncryptionInfoProto fileProto =
             HdfsProtos.PerFileEncryptionInfoProto.parseFrom(
                 fileXAttr.getValue());
-        return PBHelper.convert(fileProto, suite, version, keyName);
+        return PBHelperClient.convert(fileProto, suite, version, keyName);
       } catch (InvalidProtocolBufferException e) {
         throw new IOException("Could not parse file encryption info for " +
             "inode " + inode, e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/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 125e1cf..8a8a6e6 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
@@ -105,7 +105,7 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 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.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.util.XMLUtils;
@@ -410,7 +410,7 @@ public abstract class FSEditLogOp {
       return null;
     }
     XAttrEditLogProto proto = XAttrEditLogProto.parseDelimitedFrom(in);
-    return PBHelper.convertXAttrs(proto.getXAttrsList());
+    return PBHelperClient.convertXAttrs(proto.getXAttrsList());
   }
 
   @SuppressWarnings("unchecked")
@@ -554,7 +554,7 @@ public abstract class FSEditLogOp {
       if (this.opCode == OP_ADD) {
         AclEditLogUtil.write(aclEntries, out);
         XAttrEditLogProto.Builder b = XAttrEditLogProto.newBuilder();
-        b.addAllXAttrs(PBHelper.convertXAttrProto(xAttrs));
+        b.addAllXAttrs(PBHelperClient.convertXAttrProto(xAttrs));
         b.build().writeDelimitedTo(out);
         FSImageSerialization.writeString(clientName,out);
         FSImageSerialization.writeString(clientMachine,out);
@@ -1631,7 +1631,7 @@ public abstract class FSEditLogOp {
       permissions.write(out);
       AclEditLogUtil.write(aclEntries, out);
       XAttrEditLogProto.Builder b = XAttrEditLogProto.newBuilder();
-      b.addAllXAttrs(PBHelper.convertXAttrProto(xAttrs));
+      b.addAllXAttrs(PBHelperClient.convertXAttrProto(xAttrs));
       b.build().writeDelimitedTo(out);
     }
     
@@ -4158,7 +4158,7 @@ public abstract class FSEditLogOp {
     void readFields(DataInputStream in, int logVersion) throws IOException {
       XAttrEditLogProto p = XAttrEditLogProto.parseDelimitedFrom(in);
       src = p.getSrc();
-      xAttrs = PBHelper.convertXAttrs(p.getXAttrsList());
+      xAttrs = PBHelperClient.convertXAttrs(p.getXAttrsList());
       readRpcIds(in, logVersion);
     }
 
@@ -4168,7 +4168,7 @@ public abstract class FSEditLogOp {
       if (src != null) {
         b.setSrc(src);
       }
-      b.addAllXAttrs(PBHelper.convertXAttrProto(xAttrs));
+      b.addAllXAttrs(PBHelperClient.convertXAttrProto(xAttrs));
       b.build().writeDelimitedTo(out);
       // clientId and callId
       writeRpcIds(rpcClientId, rpcCallId, out);
@@ -4211,7 +4211,7 @@ public abstract class FSEditLogOp {
     void readFields(DataInputStream in, int logVersion) throws IOException {
       XAttrEditLogProto p = XAttrEditLogProto.parseDelimitedFrom(in);
       src = p.getSrc();
-      xAttrs = PBHelper.convertXAttrs(p.getXAttrsList());
+      xAttrs = PBHelperClient.convertXAttrs(p.getXAttrsList());
       readRpcIds(in, logVersion);
     }
 
@@ -4221,7 +4221,7 @@ public abstract class FSEditLogOp {
       if (src != null) {
         b.setSrc(src);
       }
-      b.addAllXAttrs(PBHelper.convertXAttrProto(xAttrs));
+      b.addAllXAttrs(PBHelperClient.convertXAttrProto(xAttrs));
       b.build().writeDelimitedTo(out);
       // clientId and callId
       writeRpcIds(rpcClientId, rpcCallId, out);
@@ -4267,7 +4267,7 @@ public abstract class FSEditLogOp {
         throw new IOException("Failed to read fields from SetAclOp");
       }
       src = p.getSrc();
-      aclEntries = PBHelper.convertAclEntry(p.getEntriesList());
+      aclEntries = PBHelperClient.convertAclEntry(p.getEntriesList());
     }
 
     @Override
@@ -4275,7 +4275,7 @@ public abstract class FSEditLogOp {
       AclEditLogProto.Builder b = AclEditLogProto.newBuilder();
       if (src != null)
         b.setSrc(src);
-      b.addAllEntries(PBHelper.convertAclEntryProto(aclEntries));
+      b.addAllEntries(PBHelperClient.convertAclEntryProto(aclEntries));
       b.build().writeDelimitedTo(out);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
index ac88919..0ae739c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
-import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
@@ -337,7 +336,7 @@ public final class FSImageFormatPBINode {
       BlockInfo[] blocks = new BlockInfo[bp.size()];
       for (int i = 0, e = bp.size(); i < e; ++i) {
         blocks[i] =
-            new BlockInfoContiguous(PBHelper.convert(bp.get(i)), replication);
+            new BlockInfoContiguous(PBHelperClient.convert(bp.get(i)), replication);
       }
       final PermissionStatus permissions = loadPermission(f.getPermission(),
           parent.getLoaderContext().getStringTable());
@@ -447,7 +446,7 @@ public final class FSImageFormatPBINode {
             XATTR_NAMESPACE_EXT_OFFSET);
         xAttrCompactBuilder.setName(v);
         if (a.getValue() != null) {
-          xAttrCompactBuilder.setValue(PBHelper.getByteString(a.getValue()));
+          xAttrCompactBuilder.setValue(PBHelperClient.getByteString(a.getValue()));
         }
         b.addXAttrs(xAttrCompactBuilder.build());
       }
@@ -636,7 +635,7 @@ public final class FSImageFormatPBINode {
 
       if (n.getBlocks() != null) {
         for (Block block : n.getBlocks()) {
-          b.addBlocks(PBHelper.convert(block));
+          b.addBlocks(PBHelperClient.convert(block));
         }
       }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
index 91ebaaf..cf21411 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
-import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -244,7 +244,7 @@ public class FSImageFormatPBSnapshot {
         List<BlockProto> bpl = pbf.getBlocksList();
         BlockInfo[] blocks = new BlockInfo[bpl.size()];
         for(int j = 0, e = bpl.size(); j < e; ++j) {
-          Block blk = PBHelper.convert(bpl.get(j));
+          Block blk = PBHelperClient.convert(bpl.get(j));
           BlockInfo storedBlock = bm.getStoredBlock(blk);
           if(storedBlock == null) {
             storedBlock = bm.addBlockCollection(
@@ -524,7 +524,7 @@ public class FSImageFormatPBSnapshot {
               .setFileSize(diff.getFileSize());
           if(diff.getBlocks() != null) {
             for(Block block : diff.getBlocks()) {
-              fb.addBlocks(PBHelper.convert(block));
+              fb.addBlocks(PBHelperClient.convert(block));
             }
           }
           INodeFileAttributes copy = diff.snapshotINode;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
index 3d2e8b9..851e5b9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
@@ -154,7 +154,7 @@ public class TestPBHelper {
   public void testConvertDatanodeID() {
     DatanodeID dn = DFSTestUtil.getLocalDatanodeID();
     DatanodeIDProto dnProto = PBHelperClient.convert(dn);
-    DatanodeID dn2 = PBHelper.convert(dnProto);
+    DatanodeID dn2 = PBHelperClient.convert(dnProto);
     compare(dn, dn2);
   }
   
@@ -176,8 +176,8 @@ public class TestPBHelper {
   @Test
   public void testConvertBlock() {
     Block b = new Block(1, 100, 3);
-    BlockProto bProto = PBHelper.convert(b);
-    Block b2 = PBHelper.convert(bProto);
+    BlockProto bProto = PBHelperClient.convert(b);
+    Block b2 = PBHelperClient.convert(bProto);
     assertEquals(b, b2);
   }
 
@@ -399,7 +399,7 @@ public class TestPBHelper {
         "identifier".getBytes(), "password".getBytes(), new Text("kind"),
         new Text("service"));
     TokenProto tokenProto = PBHelperClient.convert(token);
-    Token<BlockTokenIdentifier> token2 = PBHelper.convert(tokenProto);
+    Token<BlockTokenIdentifier> token2 = PBHelperClient.convert(tokenProto);
     compare(token, token2);
   }
   
@@ -489,16 +489,16 @@ public class TestPBHelper {
   @Test
   public void testConvertLocatedBlock() {
     LocatedBlock lb = createLocatedBlock();
-    LocatedBlockProto lbProto = PBHelper.convert(lb);
-    LocatedBlock lb2 = PBHelper.convert(lbProto);
+    LocatedBlockProto lbProto = PBHelperClient.convert(lb);
+    LocatedBlock lb2 = PBHelperClient.convert(lbProto);
     compare(lb,lb2);
   }
 
   @Test
   public void testConvertLocatedBlockNoStorageMedia() {
     LocatedBlock lb = createLocatedBlockNoStorageMedia();
-    LocatedBlockProto lbProto = PBHelper.convert(lb);
-    LocatedBlock lb2 = PBHelper.convert(lbProto);
+    LocatedBlockProto lbProto = PBHelperClient.convert(lb);
+    LocatedBlock lb2 = PBHelperClient.convert(lbProto);
     compare(lb,lb2);
   }
 
@@ -508,8 +508,8 @@ public class TestPBHelper {
     for (int i=0;i<3;i++) {
       lbl.add(createLocatedBlock());
     }
-    List<LocatedBlockProto> lbpl = PBHelper.convertLocatedBlock2(lbl);
-    List<LocatedBlock> lbl2 = PBHelper.convertLocatedBlock(lbpl);
+    List<LocatedBlockProto> lbpl = PBHelperClient.convertLocatedBlock2(lbl);
+    List<LocatedBlock> lbl2 = PBHelperClient.convertLocatedBlock(lbpl);
     assertEquals(lbl.size(), lbl2.size());
     for (int i=0;i<lbl.size();i++) {
       compare(lbl.get(i), lbl2.get(2));
@@ -522,8 +522,8 @@ public class TestPBHelper {
     for (int i=0;i<3;i++) {
       lbl[i] = createLocatedBlock();
     }
-    LocatedBlockProto [] lbpl = PBHelper.convertLocatedBlock(lbl);
-    LocatedBlock [] lbl2 = PBHelper.convertLocatedBlock(lbpl);
+    LocatedBlockProto [] lbpl = PBHelperClient.convertLocatedBlock(lbl);
+    LocatedBlock [] lbl2 = PBHelperClient.convertLocatedBlock(lbpl);
     assertEquals(lbl.length, lbl2.length);
     for (int i=0;i<lbl.length;i++) {
       compare(lbl[i], lbl2[i]);
@@ -551,8 +551,8 @@ public class TestPBHelper {
     DatanodeStorage dns1 = new DatanodeStorage(
         "id1", DatanodeStorage.State.NORMAL, StorageType.SSD);
 
-    DatanodeStorageProto proto = PBHelper.convert(dns1);
-    DatanodeStorage dns2 = PBHelper.convert(proto);
+    DatanodeStorageProto proto = PBHelperClient.convert(dns1);
+    DatanodeStorage dns2 = PBHelperClient.convert(proto);
     compare(dns1, dns2);
   }
   
@@ -625,7 +625,7 @@ public class TestPBHelper {
             .setPermission(FsAction.NONE)
             .build() };
     AclEntry[] actual = Lists.newArrayList(
-        PBHelper.convertAclEntry(PBHelper.convertAclEntryProto(Lists
+        PBHelperClient.convertAclEntry(PBHelperClient.convertAclEntryProto(Lists
             .newArrayList(e1, e2, e3)))).toArray(new AclEntry[0]);
     Assert.assertArrayEquals(expected, actual);
   }
@@ -637,6 +637,6 @@ public class TestPBHelper {
         .setType(AclEntryType.OTHER).build();
     AclStatus s = new AclStatus.Builder().owner("foo").group("bar").addEntry(e)
         .build();
-    Assert.assertEquals(s, PBHelper.convert(PBHelper.convert(s)));
+    Assert.assertEquals(s, PBHelperClient.convert(PBHelperClient.convert(s)));
   }
 }


[31/50] [abbrv] hadoop git commit: MAPREDUCE-6483. Replace deprecated method NameNode.getUri() with DFSUtilClient.getNNUri() in TestMRCredentials. Contributed by Mingliang Liu.

Posted by zh...@apache.org.
MAPREDUCE-6483. Replace deprecated method NameNode.getUri() with DFSUtilClient.getNNUri() in TestMRCredentials. Contributed by Mingliang Liu.


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

Branch: refs/heads/HDFS-7285
Commit: 602335dfe61a08be02ffa5a9c5b6c3f148926a7d
Parents: f97dc03
Author: Haohui Mai <wh...@apache.org>
Authored: Fri Sep 18 13:36:09 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Sep 18 13:36:09 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                             | 3 +++
 .../org/apache/hadoop/mapreduce/security/TestMRCredentials.java  | 4 ++--
 2 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/602335df/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index a80a8f6..c67e494 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -384,6 +384,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6477. Replace usage of deprecated NameNode.DEFAULT_PORT in
     TestFileSystem. (Mingliang Liu via wheat9)
 
+    MAPREDUCE-6483. Replace deprecated method NameNode.getUri() with
+    DFSUtilClient.getNNUri() in TestMRCredentials. (Mingliang Liu via wheat9)
+
   OPTIMIZATIONS
 
     MAPREDUCE-6376. Add avro binary support for jhist files (Ray Chiang via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/602335df/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestMRCredentials.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestMRCredentials.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestMRCredentials.java
index 72639e2..85d60f0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestMRCredentials.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestMRCredentials.java
@@ -29,6 +29,7 @@ import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.mapred.JobConf;
@@ -111,9 +112,8 @@ public class TestMRCredentials {
     Configuration jobConf =  new JobConf(mrCluster.getConfig());
 
     // provide namenodes names for the job to get the delegation tokens for
-    //String nnUri = dfsCluster.getNameNode().getUri(namenode).toString();
     NameNode nn = dfsCluster.getNameNode();
-    URI nnUri = NameNode.getUri(nn.getNameNodeAddress());
+    URI nnUri = DFSUtilClient.getNNUri(nn.getNameNodeAddress());
     jobConf.set(JobContext.JOB_NAMENODES, nnUri + "," + nnUri.toString());
 
 


[09/50] [abbrv] hadoop git commit: YARN-3433. Jersey tests failing with Port in Use -again. (Brahma Reddy Battula)

Posted by zh...@apache.org.
YARN-3433. Jersey tests failing with Port in Use -again.  (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/bf2f2b4f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bf2f2b4f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bf2f2b4f

Branch: refs/heads/HDFS-7285
Commit: bf2f2b4fc436ea5990e6fc78eb18091b9458e75a
Parents: 08024d1
Author: Steve Loughran <st...@apache.org>
Authored: Wed Sep 16 10:30:49 2015 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Sep 16 10:31:02 2015 +0100

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                       |  3 +++
 .../org/apache/hadoop/yarn/webapp/JerseyTestBase.java | 14 +++++---------
 2 files changed, 8 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf2f2b4f/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index c5143ff..e5cb40e 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -928,6 +928,9 @@ Release 2.7.2 - UNRELEASED
     YARN-4158. Remove duplicate close for LogWriter in
     AppLogAggregatorImpl#uploadLogsForContainers (Zhihai Xu via jlowe)
 
+    YARN-3433. Jersey tests failing with Port in Use -again.
+    (Brahma Reddy Battula)
+
 Release 2.7.1 - 2015-07-06
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf2f2b4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/JerseyTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/JerseyTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/JerseyTestBase.java
index 0b177f9..7a225a3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/JerseyTestBase.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/JerseyTestBase.java
@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.yarn.webapp;
 
+import java.io.IOException;
+
+import org.apache.hadoop.net.ServerSocketUtil;
 import org.junit.Before;
 import com.sun.jersey.test.framework.JerseyTest;
 import com.sun.jersey.test.framework.WebAppDescriptor;
@@ -28,15 +31,8 @@ public abstract class JerseyTestBase extends JerseyTest {
   }
 
   @Before
-  public void initializeJerseyPort() {
-    int jerseyPort = 9998;
-    String port = System.getProperty("jersey.test.port");
-    if(null != port) {
-      jerseyPort = Integer.parseInt(port) + 10;
-      if(jerseyPort > 65535) {
-        jerseyPort = 9998;
-      }
-    }
+  public void initializeJerseyPort() throws IOException {
+    int jerseyPort = ServerSocketUtil.getPort(9998, 10);
     System.setProperty("jersey.test.port", Integer.toString(jerseyPort));
   }
 }


[10/50] [abbrv] hadoop git commit: HDFS-7995. Implement chmod in the HDFS Web UI. Contributed by Ravi Prakash and Haohui Mai.

Posted by zh...@apache.org.
HDFS-7995. Implement chmod in the HDFS Web UI. Contributed by Ravi Prakash and Haohui Mai.


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

Branch: refs/heads/HDFS-7285
Commit: 07f304467d317d0de2cdf6ddda625ca103153146
Parents: bf2f2b4
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Sep 16 11:50:17 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Wed Sep 16 11:50:28 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../src/main/webapps/hdfs/explorer.html         | 51 ++++++++++++++++--
 .../src/main/webapps/hdfs/explorer.js           | 56 +++++++++++++++++++-
 .../src/main/webapps/static/hadoop.css          | 19 +++++++
 4 files changed, 123 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/07f30446/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 78c83d1..a328805 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -926,6 +926,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-7986. Allow files / directories to be deleted from the NameNode UI.
     (Ravi Prakash via wheat9)
 
+    HDFS-7995. Implement chmod in the HDFS Web UI.
+    (Ravi Prakash and Haohui Mai via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/07f30446/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
index 2fd7db6..1283ea6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
@@ -87,7 +87,6 @@
 	<button type="button" class="close" onclick="$('#alert-panel').hide();">&times;</button>
 	<div class="alert-body" id="alert-panel-body"></div>
       </div>
-
     <div class="modal" id="btn-create-directory" tabindex="-1" role="dialog"
       aria-hidden="true">
       <div class="modal-dialog">
@@ -169,6 +168,46 @@
 
     </div>
 
+    <script type="text/x-template" id="explorer-popover-perm-info">
+      <div class="explorer-popover-perm-body">
+        <table class="table table-striped">
+          <thead>
+            <tr>
+              <th class="text-center">User</th>
+              <th class="text-center">Group</th>
+              <th class="text-center">Other</th>
+            </tr>
+          </thead>
+          <tbody>
+            <tr>
+              <td><label><input type="checkbox" data-bit="8" /> Read</label></td>
+              <td><label><input type="checkbox" data-bit="5" /> Read</label></td>
+              <td><label><input type="checkbox" data-bit="2" /> Read</label></td>
+            </tr>
+            <tr>
+              <td><label><input type="checkbox" data-bit="7" /> Write</label></td>
+              <td><label><input type="checkbox" data-bit="4" /> Write</label></td>
+              <td><label><input type="checkbox" data-bit="1" /> Write</label></td>
+            </tr>
+            <tr>
+              <td><label><input type="checkbox" data-bit="6" /> Execute</label></td>
+              <td><label><input type="checkbox" data-bit="3" /> Execute</label></td>
+              <td><label><input type="checkbox" data-bit="0" /> Execute</label></td>
+            </tr>
+          </tbody>
+        </table>
+        <div style="text-align: right; margin-right: 10px">
+          <label><input type="checkbox" id="explorer-perm-sticky" data-bit="9" /> Sticky bit</label>
+        </div>
+        <hr/>
+        <div style="text-align: right">
+          <button type="button" class="btn" id="explorer-perm-cancel">Cancel</button>
+          <button type="button" class="btn btn-success" id="explorer-set-perm-button"
+                  data-complete-text="Updating...">Set</button>
+        </div>
+      </div>
+    </script>
+
     <script type="text/x-dust-template" id="tmpl-explorer">
       <table class="table">
         <thead>
@@ -181,13 +220,17 @@
             <th>Replication</th>
             <th>Block Size</th>
             <th>Name</th>
-            <th>Delete</th>
+            <th></th>
           </tr>
         </thead>
         <tbody>
           {#FileStatus}
-          <tr inode-path="{pathSuffix}" class="explorer-entry">
-            <td>{type|helper_to_directory}{permission|helper_to_permission}{aclBit|helper_to_acl_bit}</td>
+          <tr inode-path="{pathSuffix}" data-permission="{permission}"
+            class="explorer-entry">
+            <td><span class="explorer-perm-links editable-click">
+              {type|helper_to_directory}{permission|helper_to_permission}
+              {aclBit|helper_to_acl_bit}
+              </span></td>
             <td>{owner}</td>
             <td>{group}</td>
             <td>{length|fmt_bytes}</td>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/07f30446/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
index e469ead..521c62d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
@@ -22,6 +22,7 @@
   // in the preview.
   var TAIL_CHUNK_SIZE = 32768;
 
+  //This stores the current directory which is being browsed
   var current_directory = "";
 
   function show_err_msg(msg) {
@@ -101,6 +102,50 @@
     $('#delete-modal').modal();
   }
 
+  /* This method loads the checkboxes on the permission info modal. It accepts
+   * the octal permissions, eg. '644' or '755' and infers the checkboxes that
+   * should be true and false
+   */
+  function view_perm_details(e, filename, abs_path, perms) {
+    $('.explorer-perm-links').popover('destroy');
+    e.popover({html: true, content: $('#explorer-popover-perm-info').html(), trigger: 'focus'})
+      .on('shown.bs.popover', function(e) {
+        var popover = $(this), parent = popover.parent();
+        //Convert octal to binary permissions
+        var bin_perms = parseInt(perms, 8).toString(2);
+        bin_perms = bin_perms.length == 9 ? "0" + bin_perms : bin_perms;
+        parent.find('#explorer-perm-cancel').on('click', function() { popover.popover('destroy'); });
+        parent.find('#explorer-set-perm-button').off().click(function() { set_permissions(abs_path); });
+        parent.find('input[type=checkbox]').each(function(idx, element) {
+          var e = $(element);
+          e.prop('checked', bin_perms.charAt(9 - e.attr('data-bit')) == '1');
+        });
+      })
+      .popover('show');
+  }
+
+  // Use WebHDFS to set permissions on an absolute path
+  function set_permissions(abs_path) {
+    var p = 0;
+    $.each($('.popover .explorer-popover-perm-body input:checked'), function(idx, e) {
+      p |= 1 << (+$(e).attr('data-bit'));
+    });
+
+    var permission_mask = p.toString(8);
+
+    // PUT /webhdfs/v1/<path>?op=SETPERMISSION&permission=<permission>
+    var url = '/webhdfs/v1' + encode_path(abs_path) +
+      '?op=SETPERMISSION' + '&permission=' + permission_mask;
+
+    $.ajax(url, { type: 'PUT'
+      }).done(function(data) {
+        browse_directory(current_directory);
+      }).error(network_error_handler(url))
+      .complete(function() {
+        $('.explorer-perm-links').popover('destroy');
+      });
+  }
+
   function encode_path(abs_path) {
     abs_path = encodeURIComponent(abs_path);
     var re = /%2F/g;
@@ -198,6 +243,14 @@
           }
         });
 
+        //Set the handler for changing permissions
+        $('.explorer-perm-links').click(function() {
+          var filename = $(this).closest('tr').attr('inode-path');
+          var abs_path = append_path(current_directory, filename);
+          var perms = $(this).closest('tr').attr('data-permission');
+          view_perm_details($(this), filename, abs_path, perms);
+        });
+
         $('.explorer-entry .glyphicon-trash').click(function() {
           var inode_name = $(this).closest('tr').attr('inode-path');
           var absolute_file_path = append_path(current_directory, inode_name);
@@ -223,8 +276,7 @@
   }
 
   $('#btn-create-directory').on('show.bs.modal', function(event) {
-    var modal = $(this)
-    $('#new_directory_pwd').html(current_directory);
+    $('#new_directory_pwd').text(current_directory);
   });
 
   $('#btn-create-directory-send').click(function () {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/07f30446/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css
index e22611e..58c3cb5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css
@@ -267,3 +267,22 @@ header.bs-docs-nav, header.bs-docs-nav .navbar-brand {
 
 .explorer-entry .explorer-browse-links { cursor: pointer; }
 .explorer-entry .glyphicon-trash { cursor: pointer; }
+
+.popover {
+  max-width: 400px;
+}
+
+.explorer-popover-perm-body table  {
+  width: 30rem;
+}
+
+.explorer-popover-perm-body table > tbody > tr > td {
+  text-align: center;
+}
+
+.explorer-popover-perm-body label {
+    display:inline;
+    margin-bottom: 0;
+    font-weight: normal;
+    cursor: pointer;
+}
\ No newline at end of file


[30/50] [abbrv] hadoop git commit: Update CHANGES.txt to reflect commit of MR-5982 to 2.7.2

Posted by zh...@apache.org.
Update CHANGES.txt to reflect commit of MR-5982 to 2.7.2


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

Branch: refs/heads/HDFS-7285
Commit: f97dc034fe5db5919294465509cfb63d73bcd58e
Parents: 88d89267
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Sep 18 19:24:10 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Sep 18 19:24:10 2015 +0000

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f97dc034/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 0c2c8a4..a80a8f6 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -565,9 +565,6 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-5002. AM could potentially allocate a reduce container to a map
     attempt (Chang Li via jlowe)
 
-    MAPREDUCE-5982. Task attempts that fail from the ASSIGNED state can
-    disappear (Chang Li via jlowe)
-
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -606,6 +603,9 @@ Release 2.7.2 - UNRELEASED
     position/key information for uncompressed input sometimes. (Zhihai Xu via
     jlowe)
 
+    MAPREDUCE-5982. Task attempts that fail from the ASSIGNED state can
+    disappear (Chang Li via jlowe)
+
 Release 2.7.1 - 2015-07-06 
 
   INCOMPATIBLE CHANGES


[05/50] [abbrv] hadoop git commit: HADOOP-12374. Updated expunge command description. (WeiWei Yang via eyang)

Posted by zh...@apache.org.
HADOOP-12374. Updated expunge command description.
(WeiWei Yang via eyang)


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

Branch: refs/heads/HDFS-7285
Commit: 2ffe2db95ede7f30aeaece4619db7eb08b84280e
Parents: 77aaf4c
Author: Eric Yang <ey...@apache.org>
Authored: Tue Sep 15 20:03:24 2015 -0700
Committer: Eric Yang <ey...@apache.org>
Committed: Tue Sep 15 20:03:24 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt               | 3 +++
 .../src/main/java/org/apache/hadoop/fs/shell/Delete.java      | 7 +++++--
 .../hadoop-common/src/site/markdown/FileSystemShell.md        | 2 +-
 3 files changed, 9 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ffe2db9/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 b637358..8ae0128 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -819,6 +819,9 @@ Release 2.8.0 - UNRELEASED
 
   BUG FIXES
 
+    HADOOP-12374. Updated expunge command description.
+    (WeiWei Yang via eyang)
+
     HADOOP-12352. Delay in checkpointing Trash can leave trash for 2 intervals
     before deleting (Casey Brotherton via harsh)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ffe2db9/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Delete.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Delete.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Delete.java
index ec45d17..6983eb9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Delete.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Delete.java
@@ -215,11 +215,14 @@ class Delete {
     }
   }
 
-  /** empty the trash */
+  // delete files from the trash that are older
+  // than the retention threshold.
   static class Expunge extends FsCommand {
     public static final String NAME = "expunge";
     public static final String USAGE = "";
-    public static final String DESCRIPTION = "Empty the Trash";
+    public static final String DESCRIPTION =
+        "Delete files from the trash that are older " +
+            "than the retention threshold";
 
     // TODO: should probably allow path arguments for the filesystems
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ffe2db9/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md b/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
index d6d00e4..2ea5aba 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
@@ -271,7 +271,7 @@ expunge
 
 Usage: `hadoop fs -expunge`
 
-Empty the Trash. Refer to the [HDFS Architecture Guide](../hadoop-hdfs/HdfsDesign.html) for more information on the Trash feature.
+If trash is enabled when a file is deleted, HDFS instead moves the deleted file to a trash directory. This command causes HDFS to permanently delete files from the trash that are older than the retention threshold. Refer to the [File Deletes and Undeletes Guide](../hadoop-hdfs/HdfsDesign.html#File_Deletes_and_Undeletes) in Space Reclamation section for more information on the Trash feature.
 
 find
 ----


[20/50] [abbrv] hadoop git commit: YARN-2597 MiniYARNCluster should propagate reason for AHS not starting

Posted by zh...@apache.org.
YARN-2597 MiniYARNCluster should propagate reason for AHS not starting


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

Branch: refs/heads/HDFS-7285
Commit: a7201d635fc45b169ca3326bad48a3f781efe604
Parents: 723c31d
Author: Steve Loughran <st...@apache.org>
Authored: Fri Sep 18 09:45:17 2015 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Sep 18 09:45:35 2015 +0100

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../hadoop/yarn/server/MiniYARNCluster.java     | 42 ++++++++++----------
 2 files changed, 24 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7201d63/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 90b0e82..8fe686d 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -445,6 +445,9 @@ Release 2.8.0 - UNRELEASED
     YARN-4149. yarn logs -am should provide an option to fetch all the log files
     (Varun Vasudev via xgong)
 
+    YARN-2597. MiniYARNCluster should propagate reason for AHS not starting.
+    (stevel)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7201d63/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
index 319047d..a348390 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.service.ServiceStateException;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
@@ -708,32 +709,31 @@ public class MiniYARNCluster extends CompositeService {
 
     @Override
     protected synchronized void serviceStart() throws Exception {
-      try {
-        new Thread() {
-          public void run() {
-            appHistoryServer.start();
-          };
-        }.start();
-        int waitCount = 0;
-        while (appHistoryServer.getServiceState() == STATE.INITED
-            && waitCount++ < 60) {
-          LOG.info("Waiting for Timeline Server to start...");
-          Thread.sleep(1500);
-        }
-        if (appHistoryServer.getServiceState() != STATE.STARTED) {
-          // AHS could have failed.
-          throw new IOException(
-              "ApplicationHistoryServer failed to start. Final state is "
-                  + appHistoryServer.getServiceState());
-        }
-        super.serviceStart();
-      } catch (Throwable t) {
-        throw new YarnRuntimeException(t);
+
+      new Thread() {
+        public void run() {
+          appHistoryServer.start();
+        };
+      }.start();
+      int waitCount = 0;
+      while (appHistoryServer.getServiceState() == STATE.INITED
+          && waitCount++ < 60) {
+        LOG.info("Waiting for Timeline Server to start...");
+        Thread.sleep(1500);
+      }
+      if (appHistoryServer.getServiceState() != STATE.STARTED) {
+        // AHS could have failed.
+        IOException ioe = new IOException(
+            "ApplicationHistoryServer failed to start. Final state is "
+            + appHistoryServer.getServiceState());
+        ioe.initCause(appHistoryServer.getFailureCause());
+        throw ioe;
       }
       LOG.info("MiniYARN ApplicationHistoryServer address: "
           + getConfig().get(YarnConfiguration.TIMELINE_SERVICE_ADDRESS));
       LOG.info("MiniYARN ApplicationHistoryServer web address: "
           + getConfig().get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS));
+      super.serviceStart();
     }
 
     @Override


[38/50] [abbrv] hadoop git commit: YARN-4113. RM should respect retry-interval when uses RetryPolicies.RETRY_FOREVER. (Sunil G via wangda)

Posted by zh...@apache.org.
YARN-4113. RM should respect retry-interval when uses RetryPolicies.RETRY_FOREVER. (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/b00392dd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b00392dd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b00392dd

Branch: refs/heads/HDFS-7285
Commit: b00392dd9cbb6778f2f3e669e96cf7133590dfe7
Parents: c9cb6a5
Author: Wangda Tan <wa...@apache.org>
Authored: Mon Sep 21 11:04:03 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Mon Sep 21 11:04:03 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 3 +++
 .../src/main/java/org/apache/hadoop/yarn/client/RMProxy.java      | 3 ++-
 .../src/main/java/org/apache/hadoop/yarn/client/ServerProxy.java  | 3 ++-
 3 files changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b00392dd/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index a3dfb85..3ef5b75 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -865,6 +865,9 @@ Release 2.8.0 - UNRELEASED
 
     YARN-4167. NPE on RMActiveServices#serviceStop when store is null. (Bibin A Chundatt via rohithsharmaks)
 
+    YARN-4113. RM should respect retry-interval when uses RetryPolicies.RETRY_FOREVER. 
+    (Sunil G via wangda)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b00392dd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RMProxy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RMProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RMProxy.java
index 28628f3..be08f2f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RMProxy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RMProxy.java
@@ -232,7 +232,8 @@ public class RMProxy<T> {
 
     RetryPolicy retryPolicy = null;
     if (waitForEver) {
-      retryPolicy = RetryPolicies.RETRY_FOREVER;
+      retryPolicy = RetryPolicies.retryForeverWithFixedSleep(
+          rmConnectionRetryIntervalMS, TimeUnit.MILLISECONDS);
     } else {
       retryPolicy =
           RetryPolicies.retryUpToMaximumTimeWithFixedSleep(rmConnectWaitMS,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b00392dd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ServerProxy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ServerProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ServerProxy.java
index de7fc7d..8c30fd7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ServerProxy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ServerProxy.java
@@ -63,7 +63,8 @@ public class ServerProxy {
     RetryPolicy retryPolicy = null;
     if (maxWaitTime == -1) {
       // wait forever.
-      retryPolicy = RetryPolicies.RETRY_FOREVER;
+      retryPolicy = RetryPolicies.retryForeverWithFixedSleep(retryIntervalMS,
+          TimeUnit.MILLISECONDS);
     } else {
       retryPolicy =
           RetryPolicies.retryUpToMaximumTimeWithFixedSleep(maxWaitTime,


[08/50] [abbrv] hadoop git commit: HDFS-9073. Fix failures in TestLazyPersistLockedMemory testReleaseOnEviction(). (J.Andreina via stevel)

Posted by zh...@apache.org.
HDFS-9073. Fix failures in TestLazyPersistLockedMemory   testReleaseOnEviction(). (J.Andreina via stevel)


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

Branch: refs/heads/HDFS-7285
Commit: 08024d1b0ba0d399ae910a7226a98b019be6bdfa
Parents: 4477643
Author: Steve Loughran <st...@apache.org>
Authored: Wed Sep 16 10:11:59 2015 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Sep 16 10:12:19 2015 +0100

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java  | 1 +
 2 files changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/08024d1b/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 589c50c..78c83d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1355,6 +1355,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9072. Fix random failures in TestJMXGet.
     (J.Andreina via stevel)
 
+    HDFS-9073. Fix failures in TestLazyPersistLockedMemory
+    testReleaseOnEviction(). (J.Andreina via stevel)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08024d1b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
index 6c49de5..9155344 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
@@ -480,6 +480,7 @@ public abstract class LazyPersistTestCase {
 
   protected final void verifyRamDiskJMXMetric(String metricName,
       long expectedValue) throws Exception {
+    waitForMetric(metricName, (int)expectedValue);
     assertEquals(expectedValue, Integer.parseInt(jmx.getValue(metricName)));
   }
 


[03/50] [abbrv] hadoop git commit: HDFS-7986. Allow files / directories to be deleted from the NameNode UI. Contributed by Ravi Prakash.

Posted by zh...@apache.org.
HDFS-7986. Allow files / directories to be deleted from the NameNode UI. Contributed by Ravi Prakash.


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

Branch: refs/heads/HDFS-7285
Commit: 6c52be78a0c6d6d86444933c6b0734dfc2477c32
Parents: 559c09d
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Sep 15 16:48:15 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Sep 15 16:48:15 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../src/main/webapps/hdfs/explorer.html         | 28 ++++++++++++++++--
 .../src/main/webapps/hdfs/explorer.js           | 31 +++++++++++++++++++-
 .../src/main/webapps/static/hadoop.css          |  5 +++-
 4 files changed, 63 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c52be78/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 4b310dd..a3b9b44 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -923,6 +923,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9082. Change the log level in WebHdfsFileSystem.initialize() from INFO
     to DEBUG. (Santhosh Nayak via cnauroth)
 
+    HDFS-7986. Allow files / directories to be deleted from the NameNode UI.
+    (Ravi Prakash via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c52be78/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
index e805915..2fd7db6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
@@ -117,6 +117,28 @@
       </div>
     </div>
 
+  <div class="modal" id="delete-modal" tabindex="-1" role="dialog" aria-hidden="true">
+    <div class="modal-dialog">
+      <div class="modal-content">
+        <div class="modal-header">
+          <button type="button" class="close" data-dismiss="modal"
+            aria-hidden="true">&times;</button>
+          <h4 class="modal-title" id="delete-modal-title">Delete</h4>
+        </div>
+        <div class="modal-body">
+          <div class="panel-body">
+            <div id="delete-prompt"></div>
+          </div>
+        </div>
+        <div class="modal-footer">
+          <button type="button" class="btn" data-dismiss="modal">Cancel</button>
+          <button type="button" class="btn btn-success" id="delete-button"
+            data-complete-text="Deleting...">Delete</button>
+        </div>
+      </div>
+    </div>
+  </div>
+
       <div class="row">
       <div class="col-xs-11">
         <form onsubmit="return false;">
@@ -159,11 +181,12 @@
             <th>Replication</th>
             <th>Block Size</th>
             <th>Name</th>
+            <th>Delete</th>
           </tr>
         </thead>
         <tbody>
           {#FileStatus}
-          <tr>
+          <tr inode-path="{pathSuffix}" class="explorer-entry">
             <td>{type|helper_to_directory}{permission|helper_to_permission}{aclBit|helper_to_acl_bit}</td>
             <td>{owner}</td>
             <td>{group}</td>
@@ -171,7 +194,8 @@
             <td>{#helper_date_tostring value="{modificationTime}"/}</td>
             <td>{replication}</td>
             <td>{blockSize|fmt_bytes}</td>
-            <td><a style="cursor:pointer" inode-type="{type}" class="explorer-browse-links" inode-path="{pathSuffix}">{pathSuffix}</a></td>
+            <td><a inode-type="{type}" class="explorer-browse-links">{pathSuffix}</a></td>
+            <td><span class="glyphicon glyphicon-trash"></span></td>
           </tr>
           {/FileStatus}
         </tbody>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c52be78/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
index 46f48b8..e469ead 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
@@ -78,6 +78,29 @@
     return data.RemoteException !== undefined ? data.RemoteException.message : "";
   }
 
+  function delete_path(inode_name, absolute_file_path) {
+    $('#delete-modal-title').text("Delete - " + inode_name);
+    $('#delete-prompt').text("Are you sure you want to delete " + inode_name
+      + " ?");
+
+    $('#delete-button').click(function() {
+      // DELETE /webhdfs/v1/<path>?op=DELETE&recursive=<true|false>
+      var url = '/webhdfs/v1' + encode_path(absolute_file_path) +
+        '?op=DELETE' + '&recursive=true';
+
+      $.ajax(url,
+        { type: 'DELETE'
+        }).done(function(data) {
+          browse_directory(current_directory);
+        }).error(network_error_handler(url)
+         ).complete(function() {
+           $('#delete-modal').modal('hide');
+           $('#delete-button').button('reset');
+        });
+    })
+    $('#delete-modal').modal();
+  }
+
   function encode_path(abs_path) {
     abs_path = encodeURIComponent(abs_path);
     var re = /%2F/g;
@@ -166,7 +189,7 @@
 
         $('.explorer-browse-links').click(function() {
           var type = $(this).attr('inode-type');
-          var path = $(this).attr('inode-path');
+          var path = $(this).closest('tr').attr('inode-path');
           var abs_path = append_path(current_directory, path);
           if (type == 'DIRECTORY') {
             browse_directory(abs_path);
@@ -174,6 +197,12 @@
             view_file_details(path, abs_path);
           }
         });
+
+        $('.explorer-entry .glyphicon-trash').click(function() {
+          var inode_name = $(this).closest('tr').attr('inode-path');
+          var absolute_file_path = append_path(current_directory, inode_name);
+          delete_path(inode_name, absolute_file_path);
+        })
       });
     }).error(network_error_handler(url));
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c52be78/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css
index c13fe3f..e22611e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css
@@ -263,4 +263,7 @@ header.bs-docs-nav, header.bs-docs-nav .navbar-brand {
 
 .dfshealth-node-legend li:before {
     padding-right: 5pt;
-}
\ No newline at end of file
+}
+
+.explorer-entry .explorer-browse-links { cursor: pointer; }
+.explorer-entry .glyphicon-trash { cursor: pointer; }


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

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index d921507,ae0a3f6..a7cacc7
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@@ -17,25 -17,150 +17,157 @@@
   */
  package org.apache.hadoop.hdfs.protocolPB;
  
+ import java.io.EOFException;
+ import java.io.IOException;
+ import java.io.InputStream;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.EnumSet;
++import java.util.HashMap;
+ import java.util.List;
++import java.util.Map;
++import java.util.Set;
+ 
+ import com.google.common.base.Preconditions;
  import com.google.common.collect.Lists;
+ import com.google.common.primitives.Shorts;
  import com.google.protobuf.ByteString;
  import com.google.protobuf.CodedInputStream;
+ 
 -import static com.google.common.base.Preconditions.checkNotNull;
 -
+ import org.apache.hadoop.crypto.CipherOption;
+ import org.apache.hadoop.crypto.CipherSuite;
+ import org.apache.hadoop.crypto.CryptoProtocolVersion;
+ import org.apache.hadoop.fs.CacheFlag;
+ import org.apache.hadoop.fs.ContentSummary;
+ import org.apache.hadoop.fs.CreateFlag;
+ import org.apache.hadoop.fs.FileEncryptionInfo;
+ import org.apache.hadoop.fs.FsServerDefaults;
+ import org.apache.hadoop.fs.Path;
  import org.apache.hadoop.fs.StorageType;
+ import org.apache.hadoop.fs.XAttr;
+ import org.apache.hadoop.fs.XAttrSetFlag;
+ import org.apache.hadoop.fs.permission.AclEntry;
+ import org.apache.hadoop.fs.permission.AclEntryScope;
+ import org.apache.hadoop.fs.permission.AclEntryType;
+ import org.apache.hadoop.fs.permission.AclStatus;
+ import org.apache.hadoop.fs.permission.FsAction;
+ import org.apache.hadoop.fs.permission.FsPermission;
+ import org.apache.hadoop.hdfs.DFSUtilClient;
+ import org.apache.hadoop.hdfs.inotify.Event;
+ import org.apache.hadoop.hdfs.inotify.EventBatch;
+ import org.apache.hadoop.hdfs.inotify.EventBatchList;
+ import org.apache.hadoop.hdfs.protocol.Block;
+ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+ import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+ import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
+ import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+ import org.apache.hadoop.hdfs.protocol.CachePoolStats;
+ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+ import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
  import org.apache.hadoop.hdfs.protocol.DatanodeID;
  import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+ import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
  import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo;
+ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+ import org.apache.hadoop.hdfs.protocol.EncryptionZone;
++import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
  import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+ import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+ import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
+ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
++import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+ import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
+ import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
+ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
+ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
+ import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryScopeProto;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryTypeProto;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.FsActionProto;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclStatusProto;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveEntryProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveStatsProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheFlagProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolStatsProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateFlagProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeReportTypeProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeStorageReportProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeActionProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeInfoProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
  import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmIdProto;
  import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto;
+ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.EncryptionZoneProto;
++import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos;
++import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECRecoveryInfoProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ContentSummaryProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CorruptFileBlocksProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CryptoProtocolVersionProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DataEncryptionKeyProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfosProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeLocalInfoProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto.StorageState;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
++import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsPermissionProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsServerDefaultsProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto.FileType;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlocksProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RollingUpgradeStatusProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportEntryProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryListingProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryStatusProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageReportProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypesProto;
+ import org.apache.hadoop.hdfs.protocol.proto.InotifyProtos;
+ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrProto;
+ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrProto.XAttrNamespaceProto;
+ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrSetFlagProto;
+ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+ import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
+ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+ import org.apache.hadoop.hdfs.server.protocol.StorageReport;
  import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId;
  import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
  import org.apache.hadoop.hdfs.util.ExactSizeInputStream;
+ import org.apache.hadoop.io.EnumSetWritable;
+ import org.apache.hadoop.io.Text;
++import org.apache.hadoop.io.erasurecode.ECSchema;
  import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
  import org.apache.hadoop.security.token.Token;
  import org.apache.hadoop.util.DataChecksum;
@@@ -264,4 -396,1934 +403,2033 @@@ public class PBHelperClient 
      assert size >= 0;
      return new ExactSizeInputStream(input, size);
    }
+ 
+   public static CipherOption convert(HdfsProtos.CipherOptionProto proto) {
+     if (proto != null) {
+       CipherSuite suite = null;
+       if (proto.getSuite() != null) {
+         suite = convert(proto.getSuite());
+       }
+       byte[] inKey = null;
+       if (proto.getInKey() != null) {
+         inKey = proto.getInKey().toByteArray();
+       }
+       byte[] inIv = null;
+       if (proto.getInIv() != null) {
+         inIv = proto.getInIv().toByteArray();
+       }
+       byte[] outKey = null;
+       if (proto.getOutKey() != null) {
+         outKey = proto.getOutKey().toByteArray();
+       }
+       byte[] outIv = null;
+       if (proto.getOutIv() != null) {
+         outIv = proto.getOutIv().toByteArray();
+       }
+       return new CipherOption(suite, inKey, inIv, outKey, outIv);
+     }
+     return null;
+   }
+ 
+   public static CipherSuite convert(HdfsProtos.CipherSuiteProto proto) {
+     switch (proto) {
+     case AES_CTR_NOPADDING:
+       return CipherSuite.AES_CTR_NOPADDING;
+     default:
+       // Set to UNKNOWN and stash the unknown enum value
+       CipherSuite suite = CipherSuite.UNKNOWN;
+       suite.setUnknownValue(proto.getNumber());
+       return suite;
+     }
+   }
+ 
+   public static HdfsProtos.CipherOptionProto convert(CipherOption option) {
+     if (option != null) {
+       HdfsProtos.CipherOptionProto.Builder builder = HdfsProtos.CipherOptionProto.
+           newBuilder();
+       if (option.getCipherSuite() != null) {
+         builder.setSuite(convert(option.getCipherSuite()));
+       }
+       if (option.getInKey() != null) {
+         builder.setInKey(ByteString.copyFrom(option.getInKey()));
+       }
+       if (option.getInIv() != null) {
+         builder.setInIv(ByteString.copyFrom(option.getInIv()));
+       }
+       if (option.getOutKey() != null) {
+         builder.setOutKey(ByteString.copyFrom(option.getOutKey()));
+       }
+       if (option.getOutIv() != null) {
+         builder.setOutIv(ByteString.copyFrom(option.getOutIv()));
+       }
+       return builder.build();
+     }
+     return null;
+   }
+ 
+   public static HdfsProtos.CipherSuiteProto convert(CipherSuite suite) {
+     switch (suite) {
+     case UNKNOWN:
+       return HdfsProtos.CipherSuiteProto.UNKNOWN;
+     case AES_CTR_NOPADDING:
+       return HdfsProtos.CipherSuiteProto.AES_CTR_NOPADDING;
+     default:
+       return null;
+     }
+   }
+ 
+   public static List<HdfsProtos.CipherOptionProto> convertCipherOptions(
+       List<CipherOption> options) {
+     if (options != null) {
+       List<HdfsProtos.CipherOptionProto> protos =
+           Lists.newArrayListWithCapacity(options.size());
+       for (CipherOption option : options) {
+         protos.add(convert(option));
+       }
+       return protos;
+     }
+     return null;
+   }
+ 
+   public static List<CipherOption> convertCipherOptionProtos(
+       List<HdfsProtos.CipherOptionProto> protos) {
+     if (protos != null) {
+       List<CipherOption> options =
+           Lists.newArrayListWithCapacity(protos.size());
+       for (HdfsProtos.CipherOptionProto proto : protos) {
+         options.add(convert(proto));
+       }
+       return options;
+     }
+     return null;
+   }
+ 
 -  public static LocatedBlock convert(LocatedBlockProto proto) {
++  public static LocatedBlock convertLocatedBlockProto(LocatedBlockProto proto) {
+     if (proto == null) return null;
+     List<DatanodeInfoProto> locs = proto.getLocsList();
+     DatanodeInfo[] targets = new DatanodeInfo[locs.size()];
+     for (int i = 0; i < locs.size(); i++) {
+       targets[i] = convert(locs.get(i));
+     }
+ 
+     final StorageType[] storageTypes = convertStorageTypes(
+         proto.getStorageTypesList(), locs.size());
+ 
+     final int storageIDsCount = proto.getStorageIDsCount();
+     final String[] storageIDs;
+     if (storageIDsCount == 0) {
+       storageIDs = null;
+     } else {
+       Preconditions.checkState(storageIDsCount == locs.size());
+       storageIDs = proto.getStorageIDsList().toArray(new String[storageIDsCount]);
+     }
+ 
++    int[] indices = null;
++    final int indexCount = proto.getBlockIndexCount();
++    if (indexCount > 0) {
++      indices = new int[indexCount];
++      for (int i = 0; i < indexCount; i++) {
++        indices[i] = proto.getBlockIndex(i);
++      }
++    }
++
+     // Set values from the isCached list, re-using references from loc
 -    List<DatanodeInfo> cachedLocs = new ArrayList<>(locs.size());
++    List<DatanodeInfo> cachedLocs = new ArrayList<DatanodeInfo>(locs.size());
+     List<Boolean> isCachedList = proto.getIsCachedList();
+     for (int i=0; i<isCachedList.size(); i++) {
+       if (isCachedList.get(i)) {
+         cachedLocs.add(targets[i]);
+       }
+     }
+ 
 -    LocatedBlock lb = new LocatedBlock(convert(proto.getB()), targets,
 -        storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(),
 -        cachedLocs.toArray(new DatanodeInfo[0]));
++    final LocatedBlock lb;
++    if (indices == null) {
++      lb = new LocatedBlock(PBHelperClient.convert(proto.getB()), targets,
++          storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(),
++          cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()]));
++    } else {
++      lb = new LocatedStripedBlock(PBHelperClient.convert(proto.getB()), targets,
++          storageIDs, storageTypes, indices, proto.getOffset(),
++          proto.getCorrupt(),
++          cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()]));
++      List<TokenProto> tokenProtos = proto.getBlockTokensList();
++      Token<BlockTokenIdentifier>[] blockTokens = new Token[indices.length];
++      for (int i = 0; i < indices.length; i++) {
++        blockTokens[i] = convert(tokenProtos.get(i));
++      }
++      ((LocatedStripedBlock) lb).setBlockTokens(blockTokens);
++    }
+     lb.setBlockToken(convert(proto.getBlockToken()));
+ 
+     return lb;
+   }
+ 
+   static public DatanodeInfo convert(DatanodeInfoProto di) {
+     if (di == null) return null;
+     return new DatanodeInfo(
+         convert(di.getId()),
+         di.hasLocation() ? di.getLocation() : null,
+         di.getCapacity(),  di.getDfsUsed(),  di.getRemaining(),
+         di.getBlockPoolUsed(), di.getCacheCapacity(), di.getCacheUsed(),
+         di.getLastUpdate(), di.getLastUpdateMonotonic(),
+         di.getXceiverCount(), convert(di.getAdminState()),
+         di.hasUpgradeDomain() ? di.getUpgradeDomain() : null);
+   }
+ 
+   public static StorageType[] convertStorageTypes(
+       List<StorageTypeProto> storageTypesList, int expectedSize) {
+     final StorageType[] storageTypes = new StorageType[expectedSize];
+     if (storageTypesList.size() != expectedSize) {
+      // missing storage types
+       Preconditions.checkState(storageTypesList.isEmpty());
+       Arrays.fill(storageTypes, StorageType.DEFAULT);
+     } else {
+       for (int i = 0; i < storageTypes.length; ++i) {
+         storageTypes[i] = convertStorageType(storageTypesList.get(i));
+       }
+     }
+     return storageTypes;
+   }
+ 
+   public static Token<BlockTokenIdentifier> convert(
+       TokenProto blockToken) {
+     return new Token<>(blockToken.getIdentifier()
+         .toByteArray(), blockToken.getPassword().toByteArray(), new Text(
+         blockToken.getKind()), new Text(blockToken.getService()));
+   }
+ 
+   // DatanodeId
+   public static DatanodeID convert(DatanodeIDProto dn) {
+     return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getDatanodeUuid(),
+         dn.getXferPort(), dn.getInfoPort(), dn.hasInfoSecurePort() ? dn
+         .getInfoSecurePort() : 0, dn.getIpcPort());
+   }
+ 
+   public static AdminStates convert(AdminState adminState) {
+     switch(adminState) {
+     case DECOMMISSION_INPROGRESS:
+       return AdminStates.DECOMMISSION_INPROGRESS;
+     case DECOMMISSIONED:
+       return AdminStates.DECOMMISSIONED;
+     case NORMAL:
+     default:
+       return AdminStates.NORMAL;
+     }
+   }
+ 
+   // LocatedBlocks
+   public static LocatedBlocks convert(LocatedBlocksProto lb) {
+     return new LocatedBlocks(
+         lb.getFileLength(), lb.getUnderConstruction(),
 -        convertLocatedBlock(lb.getBlocksList()),
 -        lb.hasLastBlock() ? convert(lb.getLastBlock()) : null,
++        convertLocatedBlocks(lb.getBlocksList()),
++        lb.hasLastBlock() ?
++            convertLocatedBlockProto(lb.getLastBlock()) : null,
+         lb.getIsLastBlockComplete(),
 -        lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) :
 -            null);
++        lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) : null,
++        lb.hasEcPolicy() ? convertErasureCodingPolicy(lb.getEcPolicy()) : null);
+   }
+ 
+   public static BlockStoragePolicy[] convertStoragePolicies(
+       List<BlockStoragePolicyProto> policyProtos) {
+     if (policyProtos == null || policyProtos.size() == 0) {
+       return new BlockStoragePolicy[0];
+     }
+     BlockStoragePolicy[] policies = new BlockStoragePolicy[policyProtos.size()];
+     int i = 0;
+     for (BlockStoragePolicyProto proto : policyProtos) {
+       policies[i++] = convert(proto);
+     }
+     return policies;
+   }
+ 
+   public static EventBatchList convert(GetEditsFromTxidResponseProto resp) throws
+     IOException {
+     final InotifyProtos.EventsListProto list = resp.getEventsList();
+     final long firstTxid = list.getFirstTxid();
+     final long lastTxid = list.getLastTxid();
+ 
+     List<EventBatch> batches = Lists.newArrayList();
+     if (list.getEventsList().size() > 0) {
+       throw new IOException("Can't handle old inotify server response.");
+     }
+     for (InotifyProtos.EventBatchProto bp : list.getBatchList()) {
+       long txid = bp.getTxid();
+       if ((txid != -1) && ((txid < firstTxid) || (txid > lastTxid))) {
+         throw new IOException("Error converting TxidResponseProto: got a " +
+             "transaction id " + txid + " that was outside the range of [" +
+             firstTxid + ", " + lastTxid + "].");
+       }
+       List<Event> events = Lists.newArrayList();
+       for (InotifyProtos.EventProto p : bp.getEventsList()) {
+         switch (p.getType()) {
+           case EVENT_CLOSE:
+             InotifyProtos.CloseEventProto close =
+                 InotifyProtos.CloseEventProto.parseFrom(p.getContents());
+             events.add(new Event.CloseEvent(close.getPath(),
+                 close.getFileSize(), close.getTimestamp()));
+             break;
+           case EVENT_CREATE:
+             InotifyProtos.CreateEventProto create =
+                 InotifyProtos.CreateEventProto.parseFrom(p.getContents());
+             events.add(new Event.CreateEvent.Builder()
+                 .iNodeType(createTypeConvert(create.getType()))
+                 .path(create.getPath())
+                 .ctime(create.getCtime())
+                 .ownerName(create.getOwnerName())
+                 .groupName(create.getGroupName())
+                 .perms(convert(create.getPerms()))
+                 .replication(create.getReplication())
+                 .symlinkTarget(create.getSymlinkTarget().isEmpty() ? null :
+                     create.getSymlinkTarget())
+                 .defaultBlockSize(create.getDefaultBlockSize())
+                 .overwrite(create.getOverwrite()).build());
+             break;
+           case EVENT_METADATA:
+             InotifyProtos.MetadataUpdateEventProto meta =
+                 InotifyProtos.MetadataUpdateEventProto.parseFrom(p.getContents());
+             events.add(new Event.MetadataUpdateEvent.Builder()
+                 .path(meta.getPath())
+                 .metadataType(metadataUpdateTypeConvert(meta.getType()))
+                 .mtime(meta.getMtime())
+                 .atime(meta.getAtime())
+                 .replication(meta.getReplication())
+                 .ownerName(
+                     meta.getOwnerName().isEmpty() ? null : meta.getOwnerName())
+                 .groupName(
+                     meta.getGroupName().isEmpty() ? null : meta.getGroupName())
+                 .perms(meta.hasPerms() ? convert(meta.getPerms()) : null)
+                 .acls(meta.getAclsList().isEmpty() ? null : convertAclEntry(
+                     meta.getAclsList()))
+                 .xAttrs(meta.getXAttrsList().isEmpty() ? null : convertXAttrs(
+                     meta.getXAttrsList()))
+                 .xAttrsRemoved(meta.getXAttrsRemoved())
+                 .build());
+             break;
+           case EVENT_RENAME:
+             InotifyProtos.RenameEventProto rename =
+                 InotifyProtos.RenameEventProto.parseFrom(p.getContents());
+             events.add(new Event.RenameEvent.Builder()
+                   .srcPath(rename.getSrcPath())
+                   .dstPath(rename.getDestPath())
+                   .timestamp(rename.getTimestamp())
+                   .build());
+             break;
+           case EVENT_APPEND:
+             InotifyProtos.AppendEventProto append =
+                 InotifyProtos.AppendEventProto.parseFrom(p.getContents());
+             events.add(new Event.AppendEvent.Builder().path(append.getPath())
+                 .newBlock(append.hasNewBlock() && append.getNewBlock())
+                 .build());
+             break;
+           case EVENT_UNLINK:
+             InotifyProtos.UnlinkEventProto unlink =
+                 InotifyProtos.UnlinkEventProto.parseFrom(p.getContents());
+             events.add(new Event.UnlinkEvent.Builder()
+                   .path(unlink.getPath())
+                   .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());
+         }
+       }
+       batches.add(new EventBatch(txid, events.toArray(new Event[0])));
+     }
+     return new EventBatchList(batches, resp.getEventsList().getFirstTxid(),
+         resp.getEventsList().getLastTxid(), resp.getEventsList().getSyncTxid());
+   }
+ 
+   // Located Block Arrays and Lists
 -  public static LocatedBlockProto[] convertLocatedBlock(LocatedBlock[] lb) {
++  public static LocatedBlockProto[] convertLocatedBlocks(LocatedBlock[] lb) {
++    if (lb == null) return null;
++    return convertLocatedBlocks2(Arrays.asList(lb))
++        .toArray(new LocatedBlockProto[lb.length]);
++  }
++
++  public static LocatedBlock[] convertLocatedBlocks(LocatedBlockProto[] lb) {
++    if (lb == null) return null;
++    return convertLocatedBlocks(Arrays.asList(lb))
++        .toArray(new LocatedBlock[lb.length]);
++  }
++
++  public static List<LocatedBlock> convertLocatedBlocks(
++      List<LocatedBlockProto> lb) {
+     if (lb == null) return null;
 -    return convertLocatedBlock2(Arrays.asList(lb)).toArray(
 -        new LocatedBlockProto[lb.length]);
++    final int len = lb.size();
++    List<LocatedBlock> result = new ArrayList<>(len);
++    for (LocatedBlockProto aLb : lb) {
++      result.add(convertLocatedBlockProto(aLb));
++    }
++    return result;
+   }
+ 
 -  public static List<LocatedBlockProto> convertLocatedBlock2(List<LocatedBlock> lb) {
++  public static List<LocatedBlockProto> convertLocatedBlocks2(
++      List<LocatedBlock> lb) {
+     if (lb == null) return null;
+     final int len = lb.size();
+     List<LocatedBlockProto> result = new ArrayList<>(len);
 -    for (int i = 0; i < len; ++i) {
 -      result.add(convert(lb.get(i)));
++    for (LocatedBlock aLb : lb) {
++      result.add(convertLocatedBlock(aLb));
+     }
+     return result;
+   }
+ 
 -  public static LocatedBlockProto convert(LocatedBlock b) {
++  public static LocatedBlockProto convertLocatedBlock(LocatedBlock b) {
+     if (b == null) return null;
+     Builder builder = LocatedBlockProto.newBuilder();
+     DatanodeInfo[] locs = b.getLocations();
+     List<DatanodeInfo> cachedLocs =
+         Lists.newLinkedList(Arrays.asList(b.getCachedLocations()));
+     for (int i = 0; i < locs.length; i++) {
+       DatanodeInfo loc = locs[i];
 -      builder.addLocs(i, convert(loc));
++      builder.addLocs(i, PBHelperClient.convert(loc));
+       boolean locIsCached = cachedLocs.contains(loc);
+       builder.addIsCached(locIsCached);
+       if (locIsCached) {
+         cachedLocs.remove(loc);
+       }
+     }
+     Preconditions.checkArgument(cachedLocs.size() == 0,
+         "Found additional cached replica locations that are not in the set of"
 -        + " storage-backed locations!");
++            + " storage-backed locations!");
+ 
+     StorageType[] storageTypes = b.getStorageTypes();
+     if (storageTypes != null) {
 -      for (int i = 0; i < storageTypes.length; ++i) {
 -        builder.addStorageTypes(convertStorageType(storageTypes[i]));
++      for (StorageType storageType : storageTypes) {
++        builder.addStorageTypes(convertStorageType(storageType));
+       }
+     }
+     final String[] storageIDs = b.getStorageIDs();
+     if (storageIDs != null) {
+       builder.addAllStorageIDs(Arrays.asList(storageIDs));
+     }
++    if (b instanceof LocatedStripedBlock) {
++      LocatedStripedBlock sb = (LocatedStripedBlock) b;
++      int[] indices = sb.getBlockIndices();
++      Token<BlockTokenIdentifier>[] blockTokens = sb.getBlockTokens();
++      for (int i = 0; i < indices.length; i++) {
++        builder.addBlockIndex(indices[i]);
++        builder.addBlockTokens(PBHelperClient.convert(blockTokens[i]));
++      }
++    }
+ 
 -    return builder.setB(convert(b.getBlock()))
 -        .setBlockToken(convert(b.getBlockToken()))
++    return builder.setB(PBHelperClient.convert(b.getBlock()))
++        .setBlockToken(PBHelperClient.convert(b.getBlockToken()))
+         .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build();
+   }
+ 
+   public static BlockStoragePolicy convert(BlockStoragePolicyProto proto) {
+     List<StorageTypeProto> cList = proto.getCreationPolicy()
+         .getStorageTypesList();
+     StorageType[] creationTypes = convertStorageTypes(cList, cList.size());
+     List<StorageTypeProto> cfList = proto.hasCreationFallbackPolicy() ? proto
+         .getCreationFallbackPolicy().getStorageTypesList() : null;
+     StorageType[] creationFallbackTypes = cfList == null ? StorageType
+         .EMPTY_ARRAY : convertStorageTypes(cfList, cfList.size());
+     List<StorageTypeProto> rfList = proto.hasReplicationFallbackPolicy() ?
+         proto.getReplicationFallbackPolicy().getStorageTypesList() : null;
+     StorageType[] replicationFallbackTypes = rfList == null ? StorageType
+         .EMPTY_ARRAY : convertStorageTypes(rfList, rfList.size());
+     return new BlockStoragePolicy((byte) proto.getPolicyId(), proto.getName(),
+         creationTypes, creationFallbackTypes, replicationFallbackTypes);
+   }
+ 
+   public static FsActionProto convert(FsAction v) {
+     return FsActionProto.valueOf(v != null ? v.ordinal() : 0);
+   }
+ 
+   public static XAttrProto convertXAttrProto(XAttr a) {
+     XAttrProto.Builder builder = XAttrProto.newBuilder();
+     builder.setNamespace(convert(a.getNameSpace()));
+     if (a.getName() != null) {
+       builder.setName(a.getName());
+     }
+     if (a.getValue() != null) {
+       builder.setValue(getByteString(a.getValue()));
+     }
+     return builder.build();
+   }
+ 
+   public static List<XAttr> convert(ListXAttrsResponseProto a) {
+     final List<XAttrProto> xAttrs = a.getXAttrsList();
+     return convertXAttrs(xAttrs);
+   }
+ 
+   public static List<XAttr> convert(GetXAttrsResponseProto a) {
+     List<XAttrProto> xAttrs = a.getXAttrsList();
+     return convertXAttrs(xAttrs);
+   }
+ 
+   public static List<XAttr> convertXAttrs(List<XAttrProto> xAttrSpec) {
+     ArrayList<XAttr> xAttrs = Lists.newArrayListWithCapacity(xAttrSpec.size());
+     for (XAttrProto a : xAttrSpec) {
+       XAttr.Builder builder = new XAttr.Builder();
+       builder.setNameSpace(convert(a.getNamespace()));
+       if (a.hasName()) {
+         builder.setName(a.getName());
+       }
+       if (a.hasValue()) {
+         builder.setValue(a.getValue().toByteArray());
+       }
+       xAttrs.add(builder.build());
+     }
+     return xAttrs;
+   }
+ 
+   static XAttrNamespaceProto convert(XAttr.NameSpace v) {
+     return XAttrNamespaceProto.valueOf(v.ordinal());
+   }
+ 
+   static XAttr.NameSpace convert(XAttrNamespaceProto v) {
+     return castEnum(v, XATTR_NAMESPACE_VALUES);
+   }
+ 
+   static <T extends Enum<T>, U extends Enum<U>> U castEnum(T from, U[] to) {
+     return to[from.ordinal()];
+   }
+ 
+   static InotifyProtos.MetadataUpdateType metadataUpdateTypeConvert(
+       Event.MetadataUpdateEvent.MetadataType type) {
+     switch (type) {
+     case TIMES:
+       return InotifyProtos.MetadataUpdateType.META_TYPE_TIMES;
+     case REPLICATION:
+       return InotifyProtos.MetadataUpdateType.META_TYPE_REPLICATION;
+     case OWNER:
+       return InotifyProtos.MetadataUpdateType.META_TYPE_OWNER;
+     case PERMS:
+       return InotifyProtos.MetadataUpdateType.META_TYPE_PERMS;
+     case ACLS:
+       return InotifyProtos.MetadataUpdateType.META_TYPE_ACLS;
+     case XATTRS:
+       return InotifyProtos.MetadataUpdateType.META_TYPE_XATTRS;
+     default:
+       return null;
+     }
+   }
+ 
+   private static Event.MetadataUpdateEvent.MetadataType metadataUpdateTypeConvert(
+       InotifyProtos.MetadataUpdateType type) {
+     switch (type) {
+     case META_TYPE_TIMES:
+       return Event.MetadataUpdateEvent.MetadataType.TIMES;
+     case META_TYPE_REPLICATION:
+       return Event.MetadataUpdateEvent.MetadataType.REPLICATION;
+     case META_TYPE_OWNER:
+       return Event.MetadataUpdateEvent.MetadataType.OWNER;
+     case META_TYPE_PERMS:
+       return Event.MetadataUpdateEvent.MetadataType.PERMS;
+     case META_TYPE_ACLS:
+       return Event.MetadataUpdateEvent.MetadataType.ACLS;
+     case META_TYPE_XATTRS:
+       return Event.MetadataUpdateEvent.MetadataType.XATTRS;
+     default:
+       return null;
+     }
+   }
+ 
+   static InotifyProtos.INodeType createTypeConvert(Event.CreateEvent.INodeType
+                                                        type) {
+     switch (type) {
+     case DIRECTORY:
+       return InotifyProtos.INodeType.I_TYPE_DIRECTORY;
+     case FILE:
+       return InotifyProtos.INodeType.I_TYPE_FILE;
+     case SYMLINK:
+       return InotifyProtos.INodeType.I_TYPE_SYMLINK;
+     default:
+       return null;
+     }
+   }
+ 
+   public static List<LocatedBlock> convertLocatedBlock(
+       List<LocatedBlockProto> lb) {
+     if (lb == null) return null;
+     final int len = lb.size();
+     List<LocatedBlock> result = new ArrayList<>(len);
+     for (int i = 0; i < len; ++i) {
 -      result.add(convert(lb.get(i)));
++      result.add(convertLocatedBlockProto(lb.get(i)));
+     }
+     return result;
+   }
+ 
+   public static List<AclEntry> convertAclEntry(List<AclEntryProto> aclSpec) {
+     ArrayList<AclEntry> r = Lists.newArrayListWithCapacity(aclSpec.size());
+     for (AclEntryProto e : aclSpec) {
+       AclEntry.Builder builder = new AclEntry.Builder();
+       builder.setType(convert(e.getType()));
+       builder.setScope(convert(e.getScope()));
+       builder.setPermission(convert(e.getPermissions()));
+       if (e.hasName()) {
+         builder.setName(e.getName());
+       }
+       r.add(builder.build());
+     }
+     return r;
+   }
+ 
+   static AclEntryScopeProto convert(AclEntryScope v) {
+     return AclEntryScopeProto.valueOf(v.ordinal());
+   }
+ 
+   private static AclEntryScope convert(AclEntryScopeProto v) {
+     return castEnum(v, ACL_ENTRY_SCOPE_VALUES);
+   }
+ 
+   static AclEntryTypeProto convert(AclEntryType e) {
+     return AclEntryTypeProto.valueOf(e.ordinal());
+   }
+ 
+   private static AclEntryType convert(AclEntryTypeProto v) {
+     return castEnum(v, ACL_ENTRY_TYPE_VALUES);
+   }
+ 
+   public static FsAction convert(FsActionProto v) {
+     return castEnum(v, FSACTION_VALUES);
+   }
+ 
+   public static FsPermission convert(FsPermissionProto p) {
+     return new FsPermissionExtension((short)p.getPerm());
+   }
+ 
+   private static Event.CreateEvent.INodeType createTypeConvert(
+       InotifyProtos.INodeType type) {
+     switch (type) {
+     case I_TYPE_DIRECTORY:
+       return Event.CreateEvent.INodeType.DIRECTORY;
+     case I_TYPE_FILE:
+       return Event.CreateEvent.INodeType.FILE;
+     case I_TYPE_SYMLINK:
+       return Event.CreateEvent.INodeType.SYMLINK;
+     default:
+       return null;
+     }
+   }
+ 
+   public static HdfsProtos.FileEncryptionInfoProto convert(
+       FileEncryptionInfo info) {
+     if (info == null) {
+       return null;
+     }
+     return HdfsProtos.FileEncryptionInfoProto.newBuilder()
+         .setSuite(convert(info.getCipherSuite()))
+         .setCryptoProtocolVersion(convert(info.getCryptoProtocolVersion()))
+         .setKey(getByteString(info.getEncryptedDataEncryptionKey()))
+         .setIv(getByteString(info.getIV()))
+         .setEzKeyVersionName(info.getEzKeyVersionName())
+         .setKeyName(info.getKeyName())
+         .build();
+   }
+ 
+   public static CryptoProtocolVersionProto convert(CryptoProtocolVersion
+       version) {
+     switch(version) {
+     case UNKNOWN:
+       return CryptoProtocolVersionProto.UNKNOWN_PROTOCOL_VERSION;
+     case ENCRYPTION_ZONES:
+       return CryptoProtocolVersionProto.ENCRYPTION_ZONES;
+     default:
+       return null;
+     }
+   }
+ 
+   public static FileEncryptionInfo convert(
+       HdfsProtos.FileEncryptionInfoProto proto) {
+     if (proto == null) {
+       return null;
+     }
+     CipherSuite suite = convert(proto.getSuite());
+     CryptoProtocolVersion version = convert(proto.getCryptoProtocolVersion());
+     byte[] key = proto.getKey().toByteArray();
+     byte[] iv = proto.getIv().toByteArray();
+     String ezKeyVersionName = proto.getEzKeyVersionName();
+     String keyName = proto.getKeyName();
+     return new FileEncryptionInfo(suite, version, key, iv, keyName,
+         ezKeyVersionName);
+   }
+ 
+   public static CryptoProtocolVersion convert(CryptoProtocolVersionProto
+       proto) {
+     switch(proto) {
+     case ENCRYPTION_ZONES:
+       return CryptoProtocolVersion.ENCRYPTION_ZONES;
+     default:
+       // Set to UNKNOWN and stash the unknown enum value
+       CryptoProtocolVersion version = CryptoProtocolVersion.UNKNOWN;
+       version.setUnknownValue(proto.getNumber());
+       return version;
+     }
+   }
+ 
+   public static List<XAttrProto> convertXAttrProto(
+       List<XAttr> xAttrSpec) {
+     if (xAttrSpec == null) {
+       return Lists.newArrayListWithCapacity(0);
+     }
+     ArrayList<XAttrProto> xAttrs = Lists.newArrayListWithCapacity(
+         xAttrSpec.size());
+     for (XAttr a : xAttrSpec) {
+       XAttrProto.Builder builder = XAttrProto.newBuilder();
+       builder.setNamespace(convert(a.getNameSpace()));
+       if (a.getName() != null) {
+         builder.setName(a.getName());
+       }
+       if (a.getValue() != null) {
+         builder.setValue(getByteString(a.getValue()));
+       }
+       xAttrs.add(builder.build());
+     }
+     return xAttrs;
+   }
+ 
+   /**
+    * The flag field in PB is a bitmask whose values are the same a the
+    * emum values of XAttrSetFlag
+    */
+   public static int convert(EnumSet<XAttrSetFlag> flag) {
+     int value = 0;
+     if (flag.contains(XAttrSetFlag.CREATE)) {
+       value |= XAttrSetFlagProto.XATTR_CREATE.getNumber();
+     }
+     if (flag.contains(XAttrSetFlag.REPLACE)) {
+       value |= XAttrSetFlagProto.XATTR_REPLACE.getNumber();
+     }
+     return value;
+   }
+ 
+   public static EncryptionZone convert(EncryptionZoneProto proto) {
+     return new EncryptionZone(proto.getId(), proto.getPath(),
+         convert(proto.getSuite()), convert(proto.getCryptoProtocolVersion()),
+         proto.getKeyName());
+   }
+ 
+   public static AclStatus convert(GetAclStatusResponseProto e) {
+     AclStatusProto r = e.getResult();
+     AclStatus.Builder builder = new AclStatus.Builder();
+     builder.owner(r.getOwner()).group(r.getGroup()).stickyBit(r.getSticky())
+         .addEntries(convertAclEntry(r.getEntriesList()));
+     if (r.hasPermission()) {
+       builder.setPermission(convert(r.getPermission()));
+     }
+     return builder.build();
+   }
+ 
+   public static List<AclEntryProto> convertAclEntryProto(
+       List<AclEntry> aclSpec) {
+     ArrayList<AclEntryProto> r = Lists.newArrayListWithCapacity(aclSpec.size());
+     for (AclEntry e : aclSpec) {
+       AclEntryProto.Builder builder = AclEntryProto.newBuilder();
+       builder.setType(convert(e.getType()));
+       builder.setScope(convert(e.getScope()));
+       builder.setPermissions(convert(e.getPermission()));
+       if (e.getName() != null) {
+         builder.setName(e.getName());
+       }
+       r.add(builder.build());
+     }
+     return r;
+   }
+ 
+   public static CachePoolEntry convert(CachePoolEntryProto proto) {
+     CachePoolInfo info = convert(proto.getInfo());
+     CachePoolStats stats = convert(proto.getStats());
+     return new CachePoolEntry(info, stats);
+   }
+ 
+   public static CachePoolInfo convert (CachePoolInfoProto proto) {
+     // Pool name is a required field, the rest are optional
 -    String poolName = checkNotNull(proto.getPoolName());
++    String poolName = Preconditions.checkNotNull(proto.getPoolName());
+     CachePoolInfo info = new CachePoolInfo(poolName);
+     if (proto.hasOwnerName()) {
+         info.setOwnerName(proto.getOwnerName());
+     }
+     if (proto.hasGroupName()) {
+       info.setGroupName(proto.getGroupName());
+     }
+     if (proto.hasMode()) {
+       info.setMode(new FsPermission((short)proto.getMode()));
+     }
+     if (proto.hasLimit())  {
+       info.setLimit(proto.getLimit());
+     }
+     if (proto.hasMaxRelativeExpiry()) {
+       info.setMaxRelativeExpiryMs(proto.getMaxRelativeExpiry());
+     }
+     return info;
+   }
+ 
+   public static CachePoolStats convert (CachePoolStatsProto proto) {
+     CachePoolStats.Builder builder = new CachePoolStats.Builder();
+     builder.setBytesNeeded(proto.getBytesNeeded());
+     builder.setBytesCached(proto.getBytesCached());
+     builder.setBytesOverlimit(proto.getBytesOverlimit());
+     builder.setFilesNeeded(proto.getFilesNeeded());
+     builder.setFilesCached(proto.getFilesCached());
+     return builder.build();
+   }
+ 
+   public static CachePoolInfoProto convert(CachePoolInfo info) {
+     CachePoolInfoProto.Builder builder = CachePoolInfoProto.newBuilder();
+     builder.setPoolName(info.getPoolName());
+     if (info.getOwnerName() != null) {
+       builder.setOwnerName(info.getOwnerName());
+     }
+     if (info.getGroupName() != null) {
+       builder.setGroupName(info.getGroupName());
+     }
+     if (info.getMode() != null) {
+       builder.setMode(info.getMode().toShort());
+     }
+     if (info.getLimit() != null) {
+       builder.setLimit(info.getLimit());
+     }
+     if (info.getMaxRelativeExpiryMs() != null) {
+       builder.setMaxRelativeExpiry(info.getMaxRelativeExpiryMs());
+     }
+     return builder.build();
+   }
+ 
+   public static CacheDirectiveInfoProto convert
+       (CacheDirectiveInfo info) {
+     CacheDirectiveInfoProto.Builder builder =
+         CacheDirectiveInfoProto.newBuilder();
+     if (info.getId() != null) {
+       builder.setId(info.getId());
+     }
+     if (info.getPath() != null) {
+       builder.setPath(info.getPath().toUri().getPath());
+     }
+     if (info.getReplication() != null) {
+       builder.setReplication(info.getReplication());
+     }
+     if (info.getPool() != null) {
+       builder.setPool(info.getPool());
+     }
+     if (info.getExpiration() != null) {
+       builder.setExpiration(convert(info.getExpiration()));
+     }
+     return builder.build();
+   }
+ 
+   public static CacheDirectiveInfoExpirationProto convert(
+       CacheDirectiveInfo.Expiration expiration) {
+     return CacheDirectiveInfoExpirationProto.newBuilder()
+         .setIsRelative(expiration.isRelative())
+         .setMillis(expiration.getMillis())
+         .build();
+   }
+ 
+   public static CacheDirectiveEntry convert(CacheDirectiveEntryProto proto) {
+     CacheDirectiveInfo info = convert(proto.getInfo());
+     CacheDirectiveStats stats = convert(proto.getStats());
+     return new CacheDirectiveEntry(info, stats);
+   }
+ 
+   public static CacheDirectiveStats convert(CacheDirectiveStatsProto proto) {
+     CacheDirectiveStats.Builder builder = new CacheDirectiveStats.Builder();
+     builder.setBytesNeeded(proto.getBytesNeeded());
+     builder.setBytesCached(proto.getBytesCached());
+     builder.setFilesNeeded(proto.getFilesNeeded());
+     builder.setFilesCached(proto.getFilesCached());
+     builder.setHasExpired(proto.getHasExpired());
+     return builder.build();
+   }
+ 
+   public static CacheDirectiveInfo convert
+       (CacheDirectiveInfoProto proto) {
+     CacheDirectiveInfo.Builder builder =
+         new CacheDirectiveInfo.Builder();
+     if (proto.hasId()) {
+       builder.setId(proto.getId());
+     }
+     if (proto.hasPath()) {
+       builder.setPath(new Path(proto.getPath()));
+     }
+     if (proto.hasReplication()) {
+       builder.setReplication(Shorts.checkedCast(
+           proto.getReplication()));
+     }
+     if (proto.hasPool()) {
+       builder.setPool(proto.getPool());
+     }
+     if (proto.hasExpiration()) {
+       builder.setExpiration(convert(proto.getExpiration()));
+     }
+     return builder.build();
+   }
+ 
+   public static CacheDirectiveInfo.Expiration convert(
+       CacheDirectiveInfoExpirationProto proto) {
+     if (proto.getIsRelative()) {
+       return CacheDirectiveInfo.Expiration.newRelative(proto.getMillis());
+     }
+     return CacheDirectiveInfo.Expiration.newAbsolute(proto.getMillis());
+   }
+ 
+   public static int convertCacheFlags(EnumSet<CacheFlag> flags) {
+     int value = 0;
+     if (flags.contains(CacheFlag.FORCE)) {
+       value |= CacheFlagProto.FORCE.getNumber();
+     }
+     return value;
+   }
+ 
+   public static SnapshotDiffReport convert(SnapshotDiffReportProto reportProto) {
+     if (reportProto == null) {
+       return null;
+     }
+     String snapshotDir = reportProto.getSnapshotRoot();
+     String fromSnapshot = reportProto.getFromSnapshot();
+     String toSnapshot = reportProto.getToSnapshot();
+     List<SnapshotDiffReportEntryProto> list = reportProto
+         .getDiffReportEntriesList();
+     List<DiffReportEntry> entries = new ArrayList<>();
+     for (SnapshotDiffReportEntryProto entryProto : list) {
+       DiffReportEntry entry = convert(entryProto);
+       if (entry != null)
+         entries.add(entry);
+     }
+     return new SnapshotDiffReport(snapshotDir, fromSnapshot, toSnapshot,
+         entries);
+   }
+ 
+   public static DiffReportEntry convert(SnapshotDiffReportEntryProto entry) {
+     if (entry == null) {
+       return null;
+     }
+     DiffType type = DiffType.getTypeFromLabel(entry
+         .getModificationLabel());
+     return type == null ? null : new DiffReportEntry(type, entry.getFullpath()
+         .toByteArray(), entry.hasTargetPath() ? entry.getTargetPath()
+         .toByteArray() : null);
+   }
+ 
+   public static SnapshottableDirectoryStatus[] convert(
+       SnapshottableDirectoryListingProto sdlp) {
+     if (sdlp == null)
+       return null;
+     List<SnapshottableDirectoryStatusProto> list = sdlp
+         .getSnapshottableDirListingList();
+     if (list.isEmpty()) {
+       return new SnapshottableDirectoryStatus[0];
+     } else {
+       SnapshottableDirectoryStatus[] result =
+           new SnapshottableDirectoryStatus[list.size()];
+       for (int i = 0; i < list.size(); i++) {
+         result[i] = convert(list.get(i));
+       }
+       return result;
+     }
+   }
+ 
+   public static SnapshottableDirectoryStatus convert(
+       SnapshottableDirectoryStatusProto sdirStatusProto) {
+     if (sdirStatusProto == null) {
+       return null;
+     }
+     final HdfsFileStatusProto status = sdirStatusProto.getDirStatus();
+     return new SnapshottableDirectoryStatus(
+         status.getModificationTime(),
+         status.getAccessTime(),
+         convert(status.getPermission()),
+         status.getOwner(),
+         status.getGroup(),
+         status.getPath().toByteArray(),
+         status.getFileId(),
+         status.getChildrenNum(),
+         sdirStatusProto.getSnapshotNumber(),
+         sdirStatusProto.getSnapshotQuota(),
+         sdirStatusProto.getParentFullpath().toByteArray());
+   }
+ 
+   // DataEncryptionKey
+   public static DataEncryptionKey convert(DataEncryptionKeyProto bet) {
+     String encryptionAlgorithm = bet.getEncryptionAlgorithm();
+     return new DataEncryptionKey(bet.getKeyId(),
+         bet.getBlockPoolId(),
+         bet.getNonce().toByteArray(),
+         bet.getEncryptionKey().toByteArray(),
+         bet.getExpiryDate(),
+         encryptionAlgorithm.isEmpty() ? null : encryptionAlgorithm);
+   }
+ 
+   public static Token<DelegationTokenIdentifier> convertDelegationToken(
+       TokenProto blockToken) {
+     return new Token<>(blockToken.getIdentifier()
+         .toByteArray(), blockToken.getPassword().toByteArray(), new Text(
+         blockToken.getKind()), new Text(blockToken.getService()));
+   }
+ 
+   // Arrays of DatanodeId
+   public static DatanodeIDProto[] convert(DatanodeID[] did) {
+     if (did == null)
+       return null;
+     final int len = did.length;
+     DatanodeIDProto[] result = new DatanodeIDProto[len];
+     for (int i = 0; i < len; ++i) {
+       result[i] = convert(did[i]);
+     }
+     return result;
+   }
+ 
+   public static FsPermissionProto convert(FsPermission p) {
+     return FsPermissionProto.newBuilder().setPerm(p.toExtendedShort()).build();
+   }
+ 
+   public static HdfsFileStatus convert(HdfsFileStatusProto fs) {
+     if (fs == null)
+       return null;
+     return new HdfsLocatedFileStatus(
+         fs.getLength(), fs.getFileType().equals(FileType.IS_DIR),
+         fs.getBlockReplication(), fs.getBlocksize(),
+         fs.getModificationTime(), fs.getAccessTime(),
+         convert(fs.getPermission()), fs.getOwner(), fs.getGroup(),
+         fs.getFileType().equals(FileType.IS_SYMLINK) ?
+             fs.getSymlink().toByteArray() : null,
+         fs.getPath().toByteArray(),
+         fs.hasFileId()? fs.getFileId(): HdfsConstants.GRANDFATHER_INODE_ID,
+         fs.hasLocations() ? convert(fs.getLocations()) : null,
+         fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
+         fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,
+         fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy()
 -            : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
++            : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
++    fs.hasEcPolicy() ? convertErasureCodingPolicy(fs.getEcPolicy()) : null);
+   }
+ 
+   public static CorruptFileBlocks convert(CorruptFileBlocksProto c) {
+     if (c == null)
+       return null;
+     List<String> fileList = c.getFilesList();
+     return new CorruptFileBlocks(fileList.toArray(new String[fileList.size()]),
+         c.getCookie());
+   }
+ 
+   public static ContentSummary convert(ContentSummaryProto cs) {
+     if (cs == null) return null;
+     ContentSummary.Builder builder = new ContentSummary.Builder();
+     builder.length(cs.getLength()).
+         fileCount(cs.getFileCount()).
+         directoryCount(cs.getDirectoryCount()).
+         quota(cs.getQuota()).
+         spaceConsumed(cs.getSpaceConsumed()).
+         spaceQuota(cs.getSpaceQuota());
+     if (cs.hasTypeQuotaInfos()) {
+       for (HdfsProtos.StorageTypeQuotaInfoProto info :
+           cs.getTypeQuotaInfos().getTypeQuotaInfoList()) {
+         StorageType type = convertStorageType(info.getType());
+         builder.typeConsumed(type, info.getConsumed());
+         builder.typeQuota(type, info.getQuota());
+       }
+     }
+     return builder.build();
+   }
+ 
+   public static RollingUpgradeActionProto convert(RollingUpgradeAction a) {
+     switch (a) {
+     case QUERY:
+       return RollingUpgradeActionProto.QUERY;
+     case PREPARE:
+       return RollingUpgradeActionProto.START;
+     case FINALIZE:
+       return RollingUpgradeActionProto.FINALIZE;
+     default:
+       throw new IllegalArgumentException("Unexpected value: " + a);
+     }
+   }
+ 
+   public static RollingUpgradeInfo convert(RollingUpgradeInfoProto proto) {
+     RollingUpgradeStatusProto status = proto.getStatus();
+     return new RollingUpgradeInfo(status.getBlockPoolId(),
+         proto.getCreatedRollbackImages(),
+         proto.getStartTime(), proto.getFinalizeTime());
+   }
+ 
+   public static DatanodeStorageReport[] convertDatanodeStorageReports(
+       List<DatanodeStorageReportProto> protos) {
+     final DatanodeStorageReport[] reports
+         = new DatanodeStorageReport[protos.size()];
+     for(int i = 0; i < reports.length; i++) {
+       reports[i] = convertDatanodeStorageReport(protos.get(i));
+     }
+     return reports;
+   }
+ 
+   public static DatanodeStorageReport convertDatanodeStorageReport(
+       DatanodeStorageReportProto proto) {
+     return new DatanodeStorageReport(
+         convert(proto.getDatanodeInfo()),
+         convertStorageReports(proto.getStorageReportsList()));
+   }
+ 
+   public static StorageReport[] convertStorageReports(
+       List<StorageReportProto> list) {
+     final StorageReport[] report = new StorageReport[list.size()];
+     for (int i = 0; i < report.length; i++) {
+       report[i] = convert(list.get(i));
+     }
+     return report;
+   }
+ 
+   public static StorageReport convert(StorageReportProto p) {
+     return new StorageReport(
+         p.hasStorage() ?
+             convert(p.getStorage()) :
+             new DatanodeStorage(p.getStorageUuid()),
+         p.getFailed(), p.getCapacity(), p.getDfsUsed(), p.getRemaining(),
+         p.getBlockPoolUsed());
+   }
+ 
+   public static DatanodeStorage convert(DatanodeStorageProto s) {
+     return new DatanodeStorage(s.getStorageUuid(),
+         convertState(s.getState()), convertStorageType(s.getStorageType()));
+   }
+ 
+   private static State convertState(StorageState state) {
+     switch(state) {
+     case READ_ONLY_SHARED:
+       return State.READ_ONLY_SHARED;
+     case NORMAL:
+     default:
+       return State.NORMAL;
+     }
+   }
+ 
+   public static SafeModeActionProto convert(
+       SafeModeAction a) {
+     switch (a) {
+     case SAFEMODE_LEAVE:
+       return SafeModeActionProto.SAFEMODE_LEAVE;
+     case SAFEMODE_ENTER:
+       return SafeModeActionProto.SAFEMODE_ENTER;
+     case SAFEMODE_GET:
+       return SafeModeActionProto.SAFEMODE_GET;
+     default:
+       throw new IllegalArgumentException("Unexpected SafeModeAction :" + a);
+     }
+   }
+ 
+   public static DatanodeInfo[] convert(List<DatanodeInfoProto> list) {
+     DatanodeInfo[] info = new DatanodeInfo[list.size()];
+     for (int i = 0; i < info.length; i++) {
+       info[i] = convert(list.get(i));
+     }
+     return info;
+   }
+ 
+   public static long[] convert(GetFsStatsResponseProto res) {
+     long[] result = new long[7];
+     result[ClientProtocol.GET_STATS_CAPACITY_IDX] = res.getCapacity();
+     result[ClientProtocol.GET_STATS_USED_IDX] = res.getUsed();
+     result[ClientProtocol.GET_STATS_REMAINING_IDX] = res.getRemaining();
+     result[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX] = res.getUnderReplicated();
+     result[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX] = res.getCorruptBlocks();
+     result[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX] = res.getMissingBlocks();
+     result[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX] =
+         res.getMissingReplOneBlocks();
+     return result;
+   }
+ 
+   public static DatanodeReportTypeProto
+     convert(DatanodeReportType t) {
+     switch (t) {
+     case ALL: return DatanodeReportTypeProto.ALL;
+     case LIVE: return DatanodeReportTypeProto.LIVE;
+     case DEAD: return DatanodeReportTypeProto.DEAD;
+     case DECOMMISSIONING: return DatanodeReportTypeProto.DECOMMISSIONING;
+     default:
+       throw new IllegalArgumentException("Unexpected data type report:" + t);
+     }
+   }
+ 
+   public static DirectoryListing convert(DirectoryListingProto dl) {
+     if (dl == null)
+       return null;
+     List<HdfsFileStatusProto> partList =  dl.getPartialListingList();
+     return new DirectoryListing(partList.isEmpty() ?
+         new HdfsLocatedFileStatus[0] :
+         convert(partList.toArray(new HdfsFileStatusProto[partList.size()])),
+         dl.getRemainingEntries());
+   }
+ 
+   public static HdfsFileStatus[] convert(HdfsFileStatusProto[] fs) {
+     if (fs == null) return null;
+     final int len = fs.length;
+     HdfsFileStatus[] result = new HdfsFileStatus[len];
+     for (int i = 0; i < len; ++i) {
+       result[i] = convert(fs[i]);
+     }
+     return result;
+   }
+ 
+   // The creatFlag field in PB is a bitmask whose values are the same a the
+   // emum values of CreateFlag
+   public static int convertCreateFlag(EnumSetWritable<CreateFlag> flag) {
+     int value = 0;
+     if (flag.contains(CreateFlag.APPEND)) {
+       value |= CreateFlagProto.APPEND.getNumber();
+     }
+     if (flag.contains(CreateFlag.CREATE)) {
+       value |= CreateFlagProto.CREATE.getNumber();
+     }
+     if (flag.contains(CreateFlag.OVERWRITE)) {
+       value |= CreateFlagProto.OVERWRITE.getNumber();
+     }
+     if (flag.contains(CreateFlag.LAZY_PERSIST)) {
+       value |= CreateFlagProto.LAZY_PERSIST.getNumber();
+     }
+     if (flag.contains(CreateFlag.NEW_BLOCK)) {
+       value |= CreateFlagProto.NEW_BLOCK.getNumber();
+     }
+     return value;
+   }
+ 
+   public static FsServerDefaults convert(FsServerDefaultsProto fs) {
+     if (fs == null) return null;
+     return new FsServerDefaults(
+         fs.getBlockSize(), fs.getBytesPerChecksum(),
+         fs.getWritePacketSize(), (short) fs.getReplication(),
+         fs.getFileBufferSize(),
+         fs.getEncryptDataTransfer(),
+         fs.getTrashInterval(),
+         convert(fs.getChecksumType()));
+   }
+ 
+   public static List<CryptoProtocolVersionProto> convert(
+       CryptoProtocolVersion[] versions) {
+     List<CryptoProtocolVersionProto> protos =
+         Lists.newArrayListWithCapacity(versions.length);
+     for (CryptoProtocolVersion v: versions) {
+       protos.add(convert(v));
+     }
+     return protos;
+   }
+ 
+   static List<StorageTypesProto> convert(StorageType[][] types) {
+     List<StorageTypesProto> list = Lists.newArrayList();
+     if (types != null) {
+       for (StorageType[] ts : types) {
+         StorageTypesProto.Builder builder = StorageTypesProto.newBuilder();
+         builder.addAllStorageTypes(convertStorageTypes(ts));
+         list.add(builder.build());
+       }
+     }
+     return list;
+   }
+ 
+   public static BlockStoragePolicyProto convert(BlockStoragePolicy policy) {
+     BlockStoragePolicyProto.Builder builder = BlockStoragePolicyProto
+         .newBuilder().setPolicyId(policy.getId()).setName(policy.getName());
+     // creation storage types
+     StorageTypesProto creationProto = convert(policy.getStorageTypes());
+     Preconditions.checkArgument(creationProto != null);
+     builder.setCreationPolicy(creationProto);
+     // creation fallback
+     StorageTypesProto creationFallbackProto = convert(
+         policy.getCreationFallbacks());
+     if (creationFallbackProto != null) {
+       builder.setCreationFallbackPolicy(creationFallbackProto);
+     }
+     // replication fallback
+     StorageTypesProto replicationFallbackProto = convert(
+         policy.getReplicationFallbacks());
+     if (replicationFallbackProto != null) {
+       builder.setReplicationFallbackPolicy(replicationFallbackProto);
+     }
+     return builder.build();
+   }
+ 
+   public static StorageTypesProto convert(StorageType[] types) {
+     if (types == null || types.length == 0) {
+       return null;
+     }
+     List<StorageTypeProto> list = convertStorageTypes(types);
+     return StorageTypesProto.newBuilder().addAllStorageTypes(list).build();
+   }
+ 
+   public static DatanodeID[] convert(DatanodeIDProto[] did) {
+     if (did == null) return null;
+     final int len = did.length;
+     DatanodeID[] result = new DatanodeID[len];
+     for (int i = 0; i < len; ++i) {
+       result[i] = convert(did[i]);
+     }
+     return result;
+   }
+ 
+   // Block
+   public static BlockProto convert(Block b) {
+     return BlockProto.newBuilder().setBlockId(b.getBlockId())
+         .setGenStamp(b.getGenerationStamp()).setNumBytes(b.getNumBytes())
+         .build();
+   }
+ 
+   public static Block convert(BlockProto b) {
+     return new Block(b.getBlockId(), b.getNumBytes(), b.getGenStamp());
+   }
+ 
+   static public DatanodeInfo[] convert(DatanodeInfoProto di[]) {
+     if (di == null) return null;
+     DatanodeInfo[] result = new DatanodeInfo[di.length];
+     for (int i = 0; i < di.length; i++) {
+       result[i] = convert(di[i]);
+     }
+     return result;
+   }
+ 
+   public static DatanodeStorageReportProto convertDatanodeStorageReport(
+       DatanodeStorageReport report) {
+     return DatanodeStorageReportProto.newBuilder()
+         .setDatanodeInfo(convert(report.getDatanodeInfo()))
+         .addAllStorageReports(convertStorageReports(report.getStorageReports()))
+         .build();
+   }
+ 
+   public static List<DatanodeStorageReportProto> convertDatanodeStorageReports(
+       DatanodeStorageReport[] reports) {
+     final List<DatanodeStorageReportProto> protos
+         = new ArrayList<>(reports.length);
+     for(int i = 0; i < reports.length; i++) {
+       protos.add(convertDatanodeStorageReport(reports[i]));
+     }
+     return protos;
+   }
+ 
+   public static LocatedBlock[] convertLocatedBlock(LocatedBlockProto[] lb) {
+     if (lb == null) return null;
+     return convertLocatedBlock(Arrays.asList(lb)).toArray(
+         new LocatedBlock[lb.length]);
+   }
+ 
+   public static LocatedBlocksProto convert(LocatedBlocks lb) {
+     if (lb == null) {
+       return null;
+     }
+     LocatedBlocksProto.Builder builder =
+         LocatedBlocksProto.newBuilder();
+     if (lb.getLastLocatedBlock() != null) {
 -      builder.setLastBlock(convert(lb.getLastLocatedBlock()));
++      builder.setLastBlock(
++          convertLocatedBlock(lb.getLastLocatedBlock()));
+     }
+     if (lb.getFileEncryptionInfo() != null) {
+       builder.setFileEncryptionInfo(convert(lb.getFileEncryptionInfo()));
+     }
++    if (lb.getErasureCodingPolicy() != null) {
++      builder.setEcPolicy(convertErasureCodingPolicy(lb.getErasureCodingPolicy()));
++    }
+     return builder.setFileLength(lb.getFileLength())
+         .setUnderConstruction(lb.isUnderConstruction())
 -        .addAllBlocks(convertLocatedBlock2(lb.getLocatedBlocks()))
++        .addAllBlocks(convertLocatedBlocks2(lb.getLocatedBlocks()))
+         .setIsLastBlockComplete(lb.isLastBlockComplete()).build();
+   }
+ 
+   public static DataEncryptionKeyProto convert(DataEncryptionKey bet) {
+     DataEncryptionKeyProto.Builder b = DataEncryptionKeyProto.newBuilder()
+         .setKeyId(bet.keyId)
+         .setBlockPoolId(bet.blockPoolId)
+         .setNonce(ByteString.copyFrom(bet.nonce))
+         .setEncryptionKey(ByteString.copyFrom(bet.encryptionKey))
+         .setExpiryDate(bet.expiryDate);
+     if (bet.encryptionAlgorithm != null) {
+       b.setEncryptionAlgorithm(bet.encryptionAlgorithm);
+     }
+     return b.build();
+   }
+ 
+   public static FsServerDefaultsProto convert(FsServerDefaults fs) {
+     if (fs == null) return null;
+     return FsServerDefaultsProto.newBuilder().
+       setBlockSize(fs.getBlockSize()).
+       setBytesPerChecksum(fs.getBytesPerChecksum()).
+       setWritePacketSize(fs.getWritePacketSize())
+       .setReplication(fs.getReplication())
+       .setFileBufferSize(fs.getFileBufferSize())
+       .setEncryptDataTransfer(fs.getEncryptDataTransfer())
+       .setTrashInterval(fs.getTrashInterval())
+       .setChecksumType(convert(fs.getChecksumType()))
+       .build();
+   }
+ 
+   public static EnumSetWritable<CreateFlag> convertCreateFlag(int flag) {
+     EnumSet<CreateFlag> result =
+        EnumSet.noneOf(CreateFlag.class);
+     if ((flag & CreateFlagProto.APPEND_VALUE) == CreateFlagProto.APPEND_VALUE) {
+       result.add(CreateFlag.APPEND);
+     }
+     if ((flag & CreateFlagProto.CREATE_VALUE) == CreateFlagProto.CREATE_VALUE) {
+       result.add(CreateFlag.CREATE);
+     }
+     if ((flag & CreateFlagProto.OVERWRITE_VALUE)
+         == CreateFlagProto.OVERWRITE_VALUE) {
+       result.add(CreateFlag.OVERWRITE);
+     }
+     if ((flag & CreateFlagProto.LAZY_PERSIST_VALUE)
+         == CreateFlagProto.LAZY_PERSIST_VALUE) {
+       result.add(CreateFlag.LAZY_PERSIST);
+     }
+     if ((flag & CreateFlagProto.NEW_BLOCK_VALUE)
+         == CreateFlagProto.NEW_BLOCK_VALUE) {
+       result.add(CreateFlag.NEW_BLOCK);
+     }
+     return new EnumSetWritable<CreateFlag>(result, CreateFlag.class);
+   }
+ 
+   public static EnumSet<CacheFlag> convertCacheFlags(int flags) {
+     EnumSet<CacheFlag> result = EnumSet.noneOf(CacheFlag.class);
+     if ((flags & CacheFlagProto.FORCE_VALUE) == CacheFlagProto.FORCE_VALUE) {
+       result.add(CacheFlag.FORCE);
+     }
+     return result;
+   }
+ 
+   public static HdfsFileStatusProto convert(HdfsFileStatus fs) {
+     if (fs == null)
+       return null;
+     FileType fType = FileType.IS_FILE;
+     if (fs.isDir()) {
+       fType = FileType.IS_DIR;
+     } else if (fs.isSymlink()) {
+       fType = FileType.IS_SYMLINK;
+     }
+ 
+     HdfsFileStatusProto.Builder builder =
+      HdfsFileStatusProto.newBuilder().
+       setLength(fs.getLen()).
+       setFileType(fType).
+       setBlockReplication(fs.getReplication()).
+       setBlocksize(fs.getBlockSize()).
+       setModificationTime(fs.getModificationTime()).
+       setAccessTime(fs.getAccessTime()).
+       setPermission(convert(fs.getPermission())).
+       setOwner(fs.getOwner()).
+       setGroup(fs.getGroup()).
+       setFileId(fs.getFileId()).
+       setChildrenNum(fs.getChildrenNum()).
+       setPath(ByteString.copyFrom(fs.getLocalNameInBytes())).
+       setStoragePolicy(fs.getStoragePolicy());
+     if (fs.isSymlink())  {
+       builder.setSymlink(ByteString.copyFrom(fs.getSymlinkInBytes()));
+     }
+     if (fs.getFileEncryptionInfo() != null) {
+       builder.setFileEncryptionInfo(convert(fs.getFileEncryptionInfo()));
+     }
+     if (fs instanceof HdfsLocatedFileStatus) {
+       final HdfsLocatedFileStatus lfs = (HdfsLocatedFileStatus) fs;
+       LocatedBlocks locations = lfs.getBlockLocations();
+       if (locations != null) {
+         builder.setLocations(convert(locations));
+       }
+     }
++    if(fs.getErasureCodingPolicy() != null) {
++      builder.setEcPolicy(convertErasureCodingPolicy(fs.getErasureCodingPolicy()));
++    }
+     return builder.build();
+   }
+ 
+   public static SnapshottableDirectoryStatusProto convert(
+       SnapshottableDirectoryStatus status) {
+     if (status == null) {
+       return null;
+     }
+     int snapshotNumber = status.getSnapshotNumber();
+     int snapshotQuota = status.getSnapshotQuota();
+     byte[] parentFullPath = status.getParentFullPath();
+     ByteString parentFullPathBytes = ByteString.copyFrom(
+         parentFullPath == null ? DFSUtilClient.EMPTY_BYTES : parentFullPath);
+     HdfsFileStatusProto fs = convert(status.getDirStatus());
+     SnapshottableDirectoryStatusProto.Builder builder =
+         SnapshottableDirectoryStatusProto
+         .newBuilder().setSnapshotNumber(snapshotNumber)
+         .setSnapshotQuota(snapshotQuota).setParentFullpath(parentFullPathBytes)
+         .setDirStatus(fs);
+     return builder.build();
+   }
+ 
+   public static HdfsFileStatusProto[] convert(HdfsFileStatus[] fs) {
+     if (fs == null) return null;
+     final int len = fs.length;
+     HdfsFileStatusProto[] result = new HdfsFileStatusProto[len];
+     for (int i = 0; i < len; ++i) {
+       result[i] = convert(fs[i]);
+     }
+     return result;
+   }
+ 
+   public static DirectoryListingProto convert(DirectoryListing d) {
+     if (d == null)
+       return null;
+     return DirectoryListingProto.newBuilder().
+         addAllPartialListing(Arrays.asList(
+             convert(d.getPartialListing()))).
+         setRemainingEntries(d.getRemainingEntries()).
+         build();
+   }
+ 
+   public static GetFsStatsResponseProto convert(long[] fsStats) {
+     GetFsStatsResponseProto.Builder result = GetFsStatsResponseProto
+         .newBuilder();
+     if (fsStats.length >= ClientProtocol.GET_STATS_CAPACITY_IDX + 1)
+       result.setCapacity(fsStats[ClientProtocol.GET_STATS_CAPACITY_IDX]);
+     if (fsStats.length >= ClientProtocol.GET_STATS_USED_IDX + 1)
+       result.setUsed(fsStats[ClientProtocol.GET_STATS_USED_IDX]);
+     if (fsStats.length >= ClientProtocol.GET_STATS_REMAINING_IDX + 1)
+       result.setRemaining(fsStats[ClientProtocol.GET_STATS_REMAINING_IDX]);
+     if (fsStats.length >= ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX + 1)
+       result.setUnderReplicated(
+               fsStats[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX]);
+     if (fsStats.length >= ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX + 1)
+       result.setCorruptBlocks(
+           fsStats[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX]);
+     if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX + 1)
+       result.setMissingBlocks(
+           fsStats[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX]);
+     if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX + 1)
+       result.setMissingReplOneBlocks(
+           fsStats[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX]);
+     return result.build();
+   }
+ 
+   public static DatanodeReportType convert(DatanodeReportTypeProto t) {
+     switch (t) {
+     case ALL: return DatanodeReportType.ALL;
+     case LIVE: return DatanodeReportType.LIVE;
+     case DEAD: return DatanodeReportType.DEAD;
+     case DECOMMISSIONING: return DatanodeReportType.DECOMMISSIONING;
+     default:
+       throw new IllegalArgumentException("Unexpected data type report:" + t);
+     }
+   }
+ 
+   public static SafeModeAction convert(
+       SafeModeActionProto a) {
+     switch (a) {
+     case SAFEMODE_LEAVE:
+       return SafeModeAction.SAFEMODE_LEAVE;
+     case SAFEMODE_ENTER:
+       return SafeModeAction.SAFEMODE_ENTER;
+     case SAFEMODE_GET:
+       return SafeModeAction.SAFEMODE_GET;
+     default:
+       throw new IllegalArgumentException("Unexpected SafeModeAction :" + a);
+     }
+   }
+ 
+   public static RollingUpgradeAction convert(RollingUpgradeActionProto a) {
+     switch (a) {
+     case QUERY:
+       return RollingUpgradeAction.QUERY;
+     case START:
+       return RollingUpgradeAction.PREPARE;
+     case FINALIZE:
+       return RollingUpgradeAction.FINALIZE;
+     default:
+       throw new IllegalArgumentException("Unexpected value: " + a);
+     }
+   }
+ 
+   public static RollingUpgradeStatusProto convertRollingUpgradeStatus(
+       RollingUpgradeStatus status) {
+     return RollingUpgradeStatusProto.newBuilder()
+         .setBlockPoolId(status.getBlockPoolId())
+         .setFinalized(status.isFinalized())
+         .build();
+   }
+ 
+   public static RollingUpgradeStatus convert(RollingUpgradeStatusProto proto) {
+     return new RollingUpgradeStatus(proto.getBlockPoolId(),
+         proto.getFinalized());
+   }
+ 
+   public static RollingUpgradeInfoProto convert(RollingUpgradeInfo info) {
+     return RollingUpgradeInfoProto.newBuilder()
+         .setStatus(convertRollingUpgradeStatus(info))
+         .setCreatedRollbackImages(info.createdRollbackImages())
+         .setStartTime(info.getStartTime())
+         .setFinalizeTime(info.getFinalizeTime())
+         .build();
+   }
+ 
+   public static CorruptFileBlocksProto convert(CorruptFileBlocks c) {
+     if (c == null)
+       return null;
+     return CorruptFileBlocksProto.newBuilder().
+         addAllFiles(Arrays.asList(c.getFiles())).
+         setCookie(c.getCookie()).
+         build();
+   }
+ 
+   public static ContentSummaryProto convert(ContentSummary cs) {
+     if (cs == null) return null;
+     ContentSummaryProto.Builder builder = ContentSummaryProto.newBuilder();
+         builder.setLength(cs.getLength()).
+         setFileCount(cs.getFileCount()).
+         setDirectoryCount(cs.getDirectoryCount()).
+         setQuota(cs.getQuota()).
+         setSpaceConsumed(cs.getSpaceConsumed()).
+         setSpaceQuota(cs.getSpaceQuota());
+ 
+     if (cs.isTypeQuotaSet() || cs.isTypeConsumedAvailable()) {
+       HdfsProtos.StorageTypeQuotaInfosProto.Builder isb =
+           HdfsProtos.StorageTypeQuotaInfosProto.newBuilder();
+       for (StorageType t: StorageType.getTypesSupportingQuota()) {
+         HdfsProtos.StorageTypeQuotaInfoProto info =
+             HdfsProtos.StorageTypeQuotaInfoProto.newBuilder().
+                 setType(convertStorageType(t)).
+                 setConsumed(cs.getTypeConsumed(t)).
+                 setQuota(cs.getTypeQuota(t)).
+                 build();
+         isb.addTypeQuotaInfo(info);
+       }
+       builder.setTypeQuotaInfos(isb);
+     }
+     return builder.build();
+   }
+ 
+   public static DatanodeStorageProto convert(DatanodeStorage s) {
+     return DatanodeStorageProto.newBuilder()
+         .setState(convertState(s.getState()))
+         .setStorageType(convertStorageType(s.getStorageType()))
+         .setStorageUuid(s.getStorageID()).build();
+   }
+ 
+   private static StorageState convertState(State state) {
+     switch(state) {
+     case READ_ONLY_SHARED:
+       return StorageState.READ_ONLY_SHARED;
+     case NORMAL:
+     default:
+       return StorageState.NORMAL;
+     }
+   }
+ 
+   public static StorageReportProto convert(StorageReport r) {
+     StorageReportProto.Builder builder = StorageReportProto.newBuilder()
+         .setBlockPoolUsed(r.getBlockPoolUsed()).setCapacity(r.getCapacity())
+         .setDfsUsed(r.getDfsUsed()).setRemaining(r.getRemaining())
+         .setStorageUuid(r.getStorage().getStorageID())
+         .setStorage(convert(r.getStorage()));
+     return builder.build();
+   }
+ 
+   public static List<StorageReportProto> convertStorageReports(StorageReport[] storages) {
+     final List<StorageReportProto> protos = new ArrayList<StorageReportProto>(
+         storages.length);
+     for(int i = 0; i < storages.length; i++) {
+       protos.add(convert(storages[i]));
+     }
+     return protos;
+   }
+ 
+   public static SnapshottableDirectoryListingProto convert(
+       SnapshottableDirectoryStatus[] status) {
+     if (status == null)
+       return null;
+     SnapshottableDirectoryStatusProto[] protos =
+         new SnapshottableDirectoryStatusProto[status.length];
+     for (int i = 0; i < status.length; i++) {
+       protos[i] = convert(status[i]);
+     }
+     List<SnapshottableDirectoryStatusProto> protoList = Arrays.asList(protos);
+     return SnapshottableDirectoryListingProto.newBuilder()
+         .addAllSnapshottableDirListing(protoList).build();
+   }
+ 
+   public static SnapshotDiffReportEntryProto convert(DiffReportEntry entry) {
+     if (entry == null) {
+       return null;
+     }
+     ByteString sourcePath = ByteString
+         .copyFrom(entry.getSourcePath() == null ? DFSUtilClient.EMPTY_BYTES : entry
+             .getSourcePath());
+     String modification = entry.getType().getLabel();
+     SnapshotDiffReportEntryProto.Builder builder = SnapshotDiffReportEntryProto
+         .newBuilder().setFullpath(sourcePath)
+         .setModificationLabel(modification);
+     if (entry.getType() == DiffType.RENAME) {
+       ByteString targetPath = ByteString
+           .copyFrom(entry.getTargetPath() == null ? DFSUtilClient.EMPTY_BYTES : entry
+               .getTargetPath());
+       builder.setTargetPath(targetPath);
+     }
+     return builder.build();
+   }
+ 
+   public static SnapshotDiffReportProto convert(SnapshotDiffReport report) {
+     if (report == null) {
+       return null;
+     }
+     List<DiffReportEntry> entries = report.getDiffList();
+     List<SnapshotDiffReportEntryProto> entryProtos = new ArrayList<>();
+     for (DiffReportEntry entry : entries) {
+       SnapshotDiffReportEntryProto entryProto = convert(entry);
+       if (entryProto != null)
+         entryProtos.add(entryProto);
+     }
+ 
+     SnapshotDiffReportProto reportProto = SnapshotDiffReportProto.newBuilder()
+         .setSnapshotRoot(report.getSnapshotRoot())
+         .setFromSnapshot(report.getFromSnapshot())
+         .setToSnapshot(report.getLaterSnapshotName())
+         .addAllDiffReportEntries(entryProtos).build();
+     return reportProto;
+   }
+ 
+   public static CacheDirectiveStatsProto convert(CacheDirectiveStats stats) {
+     CacheDirectiveStatsProto.Builder builder =
+         CacheDirectiveStatsProto.newBuilder();
+     builder.setBytesNeeded(stats.getBytesNeeded());
+     builder.setBytesCached(stats.getBytesCached());
+     builder.setFilesNeeded(stats.getFilesNeeded());
+     builder.setFilesCached(stats.getFilesCached());
+     builder.setHasExpired(stats.hasExpired());
+     return builder.build();
+   }
+ 
+   public static CacheDirectiveEntryProto convert(CacheDirectiveEntry entry) {
+     CacheDirectiveEntryProto.Builder builder =
+         CacheDirectiveEntryProto.newBuilder();
+     builder.setInfo(convert(entry.getInfo()));
+     builder.setStats(convert(entry.getStats()));
+     return builder.build();
+   }
+ 
+   public static boolean[] convertBooleanList(
+     List<Boolean> targetPinningsList) {
+     final boolean[] targetPinnings = new boolean[targetPinningsList.size()];
+     for (int i = 0; i < targetPinningsList.size(); i++) {
+       targetPinnings[i] = targetPinningsList.get(i);
+     }
+     return targetPinnings;
+   }
+ 
+   public static CachePoolStatsProto convert(CachePoolStats stats) {
+     CachePoolStatsProto.Builder builder = CachePoolStatsProto.newBuilder();
+     builder.setBytesNeeded(stats.getBytesNeeded());
+     builder.setBytesCached(stats.getBytesCached());
+     builder.setBytesOverlimit(stats.getBytesOverlimit());
+     builder.setFilesNeeded(stats.getFilesNeeded());
+     builder.setFilesCached(stats.getFilesCached());
+     return builder.build();
+   }
+ 
+   public static CachePoolEntryProto convert(CachePoolEntry entry) {
+     CachePoolEntryProto.Builder builder = CachePoolEntryProto.newBuilder();
+     builder.setInfo(convert(entry.getInfo()));
+     builder.setStats(convert(entry.getStats()));
+     return builder.build();
+   }
+ 
+   public static DatanodeLocalInfoProto convert(DatanodeLocalInfo info) {
+     DatanodeLocalInfoProto.Builder builder = DatanodeLocalInfoProto.newBuilder();
+     builder.setSoftwareVersion(info.getSoftwareVersion());
+     builder.setConfigVersion(info.getConfigVersion());
+     builder.setUptime(info.getUptime());
+     return builder.build();
+   }
+ 
+   public static GetAclStatusResponseProto convert(AclStatus e) {
+     AclStatusProto.Builder builder = AclStatusProto.newBuilder();
+     builder.setOwner(e.getOwner())
+         .setGroup(e.getGroup()).setSticky(e.isStickyBit())
+         .addAllEntries(convertAclEntryProto(e.getEntries()));
+     if (e.getPermission() != null) {
+       builder.setPermission(convert(e.getPermission()));
+     }
+     AclStatusProto r = builder.build();
+     return GetAclStatusResponseProto.newBuilder().setResult(r).build();
+   }
+ 
+   public static EnumSet<XAttrSetFlag> convert(int flag) {
+     EnumSet<XAttrSetFlag> result =
+         EnumSet.noneOf(XAttrSetFlag.class);
+     if ((flag & XAttrSetFlagProto.XATTR_CREATE_VALUE) ==
+         XAttrSetFlagProto.XATTR_CREATE_VALUE) {
+       result.add(XAttrSetFlag.CREATE);
+     }
+     if ((flag & XAttrSetFlagProto.XATTR_REPLACE_VALUE) ==
+         XAttrSetFlagProto.XATTR_REPLACE_VALUE) {
+       result.add(XAttrSetFlag.REPLACE);
+     }
+     return result;
+   }
+ 
+   public static XAttr convertXAttr(XAttrProto a) {
+     XAttr.Builder builder = new XAttr.Builder();
+     builder.setNameSpace(convert(a.getNamespace()));
+     if (a.hasName()) {
+       builder.setName(a.getName());
+     }
+     if (a.hasValue()) {
+       builder.setValue(a.getValue().toByteArray());
+     }
+     return builder.build();
+   }
+ 
+   public static GetXAttrsResponseProto convertXAttrsResponse(
+       List<XAttr> xAttrs) {
+     GetXAttrsResponseProto.Builder builder = GetXAttrsResponseProto
+         .newBuilder();
+     if (xAttrs != null) {
+       builder.addAllXAttrs(convertXAttrProto(xAttrs));
+     }
+     return builder.build();
+   }
+ 
+   public static ListXAttrsResponseProto convertListXAttrsResponse(
+     List<XAttr> names) {
+     ListXAttrsResponseProto.Builder builder =
+       ListXAttrsResponseProto.newBuilder();
+     if (names != null) {
+       builder.addAllXAttrs(convertXAttrProto(names));
+     }
+     return builder.build();
+   }
+ 
+   public static EncryptionZoneProto convert(EncryptionZone zone) {
+     return EncryptionZoneProto.newBuilder()
+         .setId(zone.getId())
+         .setPath(zone.getPath())
+         .setSuite(convert(zone.getSuite()))
+         .setCryptoProtocolVersion(convert(zone.getVersion()))
+         .setKeyName(zone.getKeyName())
+         .build();
+   }
+ 
+   public static SlotId convert(ShortCircuitShmSlotProto slotId) {
+     return new SlotId(convert(slotId.getShmId()),
+         slotId.getSlotIdx());
+   }
+ 
+   public static GetEditsFromTxidResponseProto convertEditsResponse(EventBatchList el) {
+     InotifyProtos.EventsListProto.Builder builder =
+         InotifyProtos.EventsListProto.newBuilder();
+     for (EventBatch b : el.getBatches()) {
+       List<InotifyProtos.EventProto> events = Lists.newArrayList();
+       for (Event e : b.getEvents()) {
+         switch (e.getEventType()) {
+           case CLOSE:
+             Event.CloseEvent ce = (Event.CloseEvent) e;
+             events.add(InotifyProtos.EventProto.newBuilder()
+                 .setType(InotifyProtos.EventType.EVENT_CLOSE)
+                 .setContents(
+                     InotifyProtos.CloseEventProto.newBuilder()
+                         .setPath(ce.getPath())
+                         .setFileSize(ce.getFileSize())
+                         .setTimestamp(ce.getTimestamp()).build().toByteString()
+                 ).build());
+             break;
+           case CREATE:
+             Event.CreateEvent ce2 = (Event.CreateEvent) e;
+             events.add(InotifyProtos.EventProto.newBuilder()
+                 .setType(InotifyProtos.EventType.EVENT_CREATE)
+                 .setContents(
+                     InotifyProtos.CreateEventProto.newBuilder()
+                         .setType(createTypeConvert(ce2.getiNodeType()))
+                         .setPath(ce2.getPath())
+                         .setCtime(ce2.getCtime())
+                         .setOwnerName(ce2.getOwnerName())
+                         .setGroupName(ce2.getGroupName())
+                         .setPerms(convert(ce2.getPerms()))
+                         .setReplication(ce2.getReplication())
+                         .setSymlinkTarget(ce2.getSymlinkTarget() == null ?
+                             "" : ce2.getSymlinkTarget())
+                         .setDefaultBlockSize(ce2.getDefaultBlockSize())
+                         .setOverwrite(ce2.getOverwrite()).build().toByteString()
+                 ).build());
+             break;
+           case METADATA:
+             Event.MetadataUpdateEvent me = (Event.MetadataUpdateEvent) e;
+             InotifyProtos.MetadataUpdateEventProto.Builder metaB =
+                 InotifyProtos.MetadataUpdateEventProto.newBuilder()
+                     .setPath(me.getPath())
+                     .setType(metadataUpdateTypeConvert(me.getMetadataType()))
+                     .setMtime(me.getMtime())
+                     .setAtime(me.getAtime())
+                     .setReplication(me.getReplication())
+                     .setOwnerName(me.getOwnerName() == null ? "" :
+                         me.getOwnerName())
+                     .setGroupName(me.getGroupName() == null ? "" :
+                         me.getGroupName())
+                     .addAllAcls(me.getAcls() == null ?
+                         Lists.<AclEntryProto>newArrayList() :
+                         convertAclEntryProto(me.getAcls()))
+                     .addAllXAttrs(me.getxAttrs() == null ?
+                         Lists.<XAttrProto>newArrayList() :
+                         convertXAttrProto(me.getxAttrs()))
+                     .setXAttrsRemoved(me.isxAttrsRemoved());
+             if (me.getPerms() != null) {
+               metaB.setPerms(convert(me.getPerms()));
+             }
+             events.add(InotifyProtos.EventProto.newBuilder()
+                 .setType(InotifyProtos.EventType.EVENT_METADATA)
+                 .setContents(metaB.build().toByteString())
+                 .build());
+             break;
+           case RENAME:
+             Event.RenameEvent re = (Event.RenameEvent) e;
+             events.add(InotifyProtos.EventProto.newBuilder()
+                 .setType(InotifyProtos.EventType.EVENT_RENAME)
+                 .setContents(
+                     InotifyProtos.RenameEventProto.newBuilder()
+                         .setSrcPath(re.getSrcPath())
+                         .setDestPath(re.getDstPath())
+                         .setTimestamp(re.getTimestamp()).build().toByteString()
+                 ).build());
+             break;
+           case APPEND:
+             Event.AppendEvent re2 = (Event.AppendEvent) e;
+             events.add(InotifyProtos.EventProto.newBuilder()
+                 .setType(InotifyProtos.EventType.EVENT_APPEND)
+                 .setContents(InotifyProtos.AppendEventProto.newBuilder()
+                     .setPath(re2.getPath())
+                     .setNewBlock(re2.toNewBlock()).build().toByteString())
+                 .build());
+             break;
+           case UNLINK:
+             Event.UnlinkEvent ue = (Event.UnlinkEvent) e;
+             events.add(InotifyProtos.EventProto.newBuilder()
+                 .setType(InotifyProtos.EventType.EVENT_UNLINK)
+                 .setContents(
+                     InotifyProtos.UnlinkEventProto.newBuilder()
+                         .setPath(ue.getPath())
+                         .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);
+         }
+       }
+       builder.addBatch(InotifyProtos.EventBatchProto.newBuilder().
+           setTxid(b.getTxid()).
+           addAllEvents(events));
+     }
+     builder.setFirstTxid(el.getFirstTxid());
+     builder.setLastTxid(el.getLastTxid());
+     builder.setSyncTxid(el.getSyncTxid());
+     return GetEditsFromTxidResponseProto.newBuilder().setEventsList(
+         builder.build()).build();
+   }
+ 
+   public static CryptoProtocolVersion[] convertCryptoProtocolVersions(
+       List<CryptoProtocolVersionProto> protos) {
+     List<CryptoProtocolVersion> versions =
+         Lists.newArrayListWithCapacity(protos.size());
+     for (CryptoProtocolVersionProto p: protos) {
+       versions.add(convert(p));
+     }
+     return versions.toArray(new CryptoProtocolVersion[]{});
+   }
+ 
+   public static HdfsProtos.PerFileEncryptionInfoProto convertPerFileEncInfo(
+       FileEncryptionInfo info) {
+     if (info == null) {
+       return null;
+     }
+     return HdfsProtos.PerFileEncryptionInfoProto.newBuilder()
+         .setKey(getByteString(info.getEncryptedDataEncryptionKey()))
+         .setIv(getByteString(info.getIV()))
+         .setEzKeyVersionName(info.getEzKeyVersionName())
+         .build();
+   }
+ 
+   public static HdfsProtos.ZoneEncryptionInfoProto convert(
+       CipherSuite suite, CryptoProtocolVersion version, String keyName) {
+     if (suite == null || version == null || keyName == null) {
+       return null;
+     }
+     return HdfsProtos.ZoneEncryptionInfoProto.newBuilder()
+         .setSuite(convert(suite))
+         .setCryptoProtocolVersion(convert(version))
+         .setKeyName(keyName)
+         .build();
+   }
+ 
+   public static FileEncryptionInfo convert(
+       HdfsProtos.PerFileEncryptionInfoProto fileProto,
+       CipherSuite suite, CryptoProtocolVersion version, String keyName) {
+     if (fileProto == null || suite == null || version == null ||
+         keyName == null) {
+       return null;
+     }
+     byte[] key = fileProto.getKey().toByteArray();
+     byte[] iv = fileProto.getIv().toByteArray();
+     String ezKeyVersionName = fileProto.getEzKeyVersionName();
+     return new FileEncryptionInf

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
----------------------------------------------------------------------


[24/50] [abbrv] hadoop git commit: Disable caching for JarURLConnection to avoid sharing JarFile with other users when loading resource from URL in Configuration class. Contributed by Zhihai Xu

Posted by zh...@apache.org.
Disable caching for JarURLConnection to avoid sharing JarFile with other users when loading resource from URL in Configuration class. 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/e690a32e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e690a32e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e690a32e

Branch: refs/heads/HDFS-7285
Commit: e690a32e57bc14b0cca0e2d39da513841d81b4fc
Parents: 288c885
Author: Zhihai Xu <zx...@apache.org>
Authored: Fri Sep 18 08:24:03 2015 -0700
Committer: Zhihai Xu <zx...@apache.org>
Committed: Fri Sep 18 08:24:53 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt          |  4 ++++
 .../main/java/org/apache/hadoop/conf/Configuration.java  | 11 ++++++++++-
 2 files changed, 14 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e690a32e/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 54d7b6b..2bf5c9d 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -782,6 +782,10 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12413. AccessControlList should avoid calling getGroupNames in
     isUserInList with empty groups. (Zhihai Xu via cnauroth)
 
+    HADOOP-12404. Disable caching for JarURLConnection to avoid sharing
+    JarFile with other users when loading resource from URL in Configuration
+    class. (zxu)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e690a32e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
index 0b45429..8801c6c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
@@ -34,7 +34,9 @@ import java.io.Reader;
 import java.io.Writer;
 import java.lang.ref.WeakReference;
 import java.net.InetSocketAddress;
+import java.net.JarURLConnection;
 import java.net.URL;
+import java.net.URLConnection;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -2531,7 +2533,14 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     if (url == null) {
       return null;
     }
-    return parse(builder, url.openStream(), url.toString());
+
+    URLConnection connection = url.openConnection();
+    if (connection instanceof JarURLConnection) {
+      // Disable caching for JarURLConnection to avoid sharing JarFile
+      // with other users.
+      connection.setUseCaches(false);
+    }
+    return parse(builder, connection.getInputStream(), url.toString());
   }
 
   private Document parse(DocumentBuilder builder, InputStream is,


[16/50] [abbrv] hadoop git commit: HDFS-9022. Move NameNode.getAddress() and NameNode.getUri() to hadoop-hdfs-client. Contributed by Mingliang Liu.

Posted by zh...@apache.org.
HDFS-9022. Move NameNode.getAddress() and NameNode.getUri() to hadoop-hdfs-client. Contributed by Mingliang Liu.


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

Branch: refs/heads/HDFS-7285
Commit: 9eee97508f350ed4629abb04e7781514ffa04070
Parents: 3f82f58
Author: Haohui Mai <wh...@apache.org>
Authored: Thu Sep 17 13:41:18 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Thu Sep 17 14:18:00 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   | 41 ++++++++++++++++
 .../hadoop/hdfs/nfs/mount/RpcProgramMountd.java |  4 +-
 .../hadoop/hdfs/nfs/nfs3/DFSClientCache.java    |  4 +-
 .../apache/hadoop/hdfs/nfs/nfs3/TestWrites.java |  7 +--
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  7 ++-
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    | 10 ++--
 .../org/apache/hadoop/hdfs/NameNodeProxies.java | 12 ++---
 .../hadoop/hdfs/server/namenode/DfsServlet.java |  3 +-
 .../hdfs/server/namenode/ImageServlet.java      |  5 +-
 .../hadoop/hdfs/server/namenode/NameNode.java   | 49 ++++----------------
 .../hdfs/server/namenode/NamenodeFsck.java      |  2 +-
 .../server/namenode/ha/BootstrapStandby.java    |  3 +-
 .../namenode/ha/IPFailoverProxyProvider.java    |  4 +-
 .../hdfs/tools/DFSZKFailoverController.java     |  3 +-
 .../org/apache/hadoop/hdfs/tools/GetGroups.java |  4 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  4 +-
 .../hadoop/hdfs/TestDFSClientFailover.java      |  2 +-
 .../hadoop/hdfs/TestDFSShellGenericOptions.java | 13 +++---
 .../hadoop/hdfs/TestDefaultNameNodePort.java    | 18 +++----
 .../org/apache/hadoop/hdfs/TestFileStatus.java  |  5 +-
 .../org/apache/hadoop/hdfs/TestGetBlocks.java   |  7 ++-
 .../apache/hadoop/hdfs/TestPersistBlocks.java   |  5 +-
 .../datanode/TestDataNodeRollingUpgrade.java    |  8 ++--
 .../server/namenode/NNThroughputBenchmark.java  |  3 +-
 .../hdfs/server/namenode/TestINodeFile.java     |  4 +-
 .../namenode/ha/TestFailureToReadEdits.java     |  3 +-
 27 files changed, 124 insertions(+), 109 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
index b032250..359886e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderFactory;
 import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.net.BasicInetPeer;
@@ -33,6 +34,7 @@ import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
@@ -587,4 +589,43 @@ public class DFSUtilClient {
       }
     }
   }
+
+  public static InetSocketAddress getNNAddress(String address) {
+    return NetUtils.createSocketAddr(address,
+        HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT);
+  }
+
+  public static InetSocketAddress getNNAddress(Configuration conf) {
+    URI filesystemURI = FileSystem.getDefaultUri(conf);
+    return getNNAddress(filesystemURI);
+  }
+
+  /**
+   * @return address of file system
+   */
+  public static InetSocketAddress getNNAddress(URI filesystemURI) {
+    String authority = filesystemURI.getAuthority();
+    if (authority == null) {
+      throw new IllegalArgumentException(String.format(
+          "Invalid URI for NameNode address (check %s): %s has no authority.",
+          FileSystem.FS_DEFAULT_NAME_KEY, filesystemURI.toString()));
+    }
+    if (!HdfsConstants.HDFS_URI_SCHEME.equalsIgnoreCase(
+        filesystemURI.getScheme())) {
+      throw new IllegalArgumentException(String.format(
+          "Invalid URI for NameNode address (check %s): " +
+          "%s is not of scheme '%s'.", FileSystem.FS_DEFAULT_NAME_KEY,
+          filesystemURI.toString(), HdfsConstants.HDFS_URI_SCHEME));
+    }
+    return getNNAddress(authority);
+  }
+
+  public static URI getNNUri(InetSocketAddress namenode) {
+    int port = namenode.getPort();
+    String portString =
+        (port == HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT) ?
+        "" : (":" + port);
+    return URI.create(HdfsConstants.HDFS_URI_SCHEME + "://"
+        + namenode.getHostName() + portString);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java
index 2814cb0..869fb73 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java
@@ -27,10 +27,10 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.mount.MountEntry;
 import org.apache.hadoop.mount.MountInterface;
 import org.apache.hadoop.mount.MountResponse;
@@ -90,7 +90,7 @@ public class RpcProgramMountd extends RpcProgram implements MountInterface {
     UserGroupInformation.setConfiguration(config);
     SecurityUtil.login(config, NfsConfigKeys.DFS_NFS_KEYTAB_FILE_KEY,
         NfsConfigKeys.DFS_NFS_KERBEROS_PRINCIPAL_KEY);
-    this.dfsClient = new DFSClient(NameNode.getAddress(config), config);
+    this.dfsClient = new DFSClient(DFSUtilClient.getNNAddress(config), config);
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java
index 79072f4..b946bce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java
@@ -33,8 +33,8 @@ import org.apache.commons.logging.Log;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSInputStream;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ShutdownHookManager;
@@ -173,7 +173,7 @@ class DFSClientCache {
         return ugi.doAs(new PrivilegedExceptionAction<DFSClient>() {
           @Override
           public DFSClient run() throws IOException {
-            return new DFSClient(NameNode.getAddress(config), config);
+            return new DFSClient(DFSUtilClient.getNNAddress(config), config);
           }
         });
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java
index 56603b9..3c193ae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java
@@ -28,6 +28,7 @@ import java.util.Arrays;
 import java.util.concurrent.ConcurrentNavigableMap;
 
 import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
@@ -35,7 +36,6 @@ import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.hdfs.nfs.nfs3.OpenFileCtx.COMMIT_STATUS;
 import org.apache.hadoop.hdfs.nfs.nfs3.OpenFileCtx.CommitCtx;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.nfs.nfs3.FileHandle;
 import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
 import org.apache.hadoop.nfs.nfs3.Nfs3Constant.WriteStableHow;
@@ -480,7 +480,7 @@ public class TestWrites {
     try {
       cluster = new MiniDFSCluster.Builder(config).numDataNodes(1).build();
       cluster.waitActive();
-      client = new DFSClient(NameNode.getAddress(config), config);
+      client = new DFSClient(DFSUtilClient.getNNAddress(config), config);
 
       // Use emphral port in case tests are running in parallel
       config.setInt("nfs3.mountd.port", 0);
@@ -596,7 +596,8 @@ public class TestWrites {
       nfs3.startServiceInternal(false);
       nfsd = (RpcProgramNfs3) nfs3.getRpcProgram();
 
-      DFSClient dfsClient = new DFSClient(NameNode.getAddress(config), config);
+      DFSClient dfsClient = new DFSClient(DFSUtilClient.getNNAddress(config),
+          config);
       HdfsFileStatus status = dfsClient.getFileInfo("/");
       FileHandle rootHandle = new FileHandle(status.getFileId());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/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 a328805..4912f50 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -929,6 +929,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-7995. Implement chmod in the HDFS Web UI.
     (Ravi Prakash and Haohui Mai via wheat9)
 
+    HDFS-9022. Move NameNode.getAddress() and NameNode.getUri() to
+    hadoop-hdfs-client. (Mingliang Liu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/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 95e9ad4..b38ec00 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
@@ -149,7 +149,6 @@ import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.io.DataOutputBuffer;
@@ -250,17 +249,17 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       = new HashMap<Long, DFSOutputStream>();
 
   /**
-   * Same as this(NameNode.getAddress(conf), conf);
+   * Same as this(NameNode.getNNAddress(conf), conf);
    * @see #DFSClient(InetSocketAddress, Configuration)
    * @deprecated Deprecated at 0.21
    */
   @Deprecated
   public DFSClient(Configuration conf) throws IOException {
-    this(NameNode.getAddress(conf), conf);
+    this(DFSUtilClient.getNNAddress(conf), conf);
   }
   
   public DFSClient(InetSocketAddress address, Configuration conf) throws IOException {
-    this(NameNode.getUri(address), conf);
+    this(DFSUtilClient.getNNUri(address), conf);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index 5c8a700..fe9e342 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -411,7 +411,7 @@ public class DFSUtil {
           NameNode.initializeGenericKeys(confForNn, nsId, nnId);
           String principal = SecurityUtil.getServerPrincipal(confForNn
               .get(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY),
-              NameNode.getAddress(confForNn).getHostName());
+              DFSUtilClient.getNNAddress(confForNn).getHostName());
           principals.add(principal);
         }
       } else {
@@ -419,7 +419,7 @@ public class DFSUtil {
         NameNode.initializeGenericKeys(confForNn, nsId, null);
         String principal = SecurityUtil.getServerPrincipal(confForNn
             .get(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY),
-            NameNode.getAddress(confForNn).getHostName());
+            DFSUtilClient.getNNAddress(confForNn).getHostName());
         principals.add(principal);
       }
     }
@@ -495,7 +495,8 @@ public class DFSUtil {
     // Use default address as fall back
     String defaultAddress;
     try {
-      defaultAddress = NetUtils.getHostPortString(NameNode.getAddress(conf));
+      defaultAddress = NetUtils.getHostPortString(
+          DFSUtilClient.getNNAddress(conf));
     } catch (IllegalArgumentException e) {
       defaultAddress = null;
     }
@@ -531,7 +532,8 @@ public class DFSUtil {
     // Use default address as fall back
     String defaultAddress;
     try {
-      defaultAddress = NetUtils.getHostPortString(NameNode.getAddress(conf));
+      defaultAddress = NetUtils.getHostPortString(
+          DFSUtilClient.getNNAddress(conf));
     } catch (IllegalArgumentException e) {
       defaultAddress = null;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
index 7a4ec4f..80efa19 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
@@ -165,8 +165,8 @@ public class NameNodeProxies {
   
     if (failoverProxyProvider == null) {
       // Non-HA case
-      return createNonHAProxy(conf, NameNode.getAddress(nameNodeUri), xface,
-          UserGroupInformation.getCurrentUser(), true,
+      return createNonHAProxy(conf, DFSUtilClient.getNNAddress(nameNodeUri),
+          xface, UserGroupInformation.getCurrentUser(), true,
           fallbackToSimpleAuth);
     } else {
       // HA case
@@ -183,10 +183,10 @@ public class NameNodeProxies {
                                                                 HdfsConstants.HDFS_URI_SCHEME);
       } else {
         dtService = SecurityUtil.buildTokenService(
-            NameNode.getAddress(nameNodeUri));
+            DFSUtilClient.getNNAddress(nameNodeUri));
       }
       return new ProxyAndInfo<T>(proxy, dtService,
-          NameNode.getAddress(nameNodeUri));
+          DFSUtilClient.getNNAddress(nameNodeUri));
     }
   }
   
@@ -249,10 +249,10 @@ public class NameNodeProxies {
                                                                 HdfsConstants.HDFS_URI_SCHEME);
       } else {
         dtService = SecurityUtil.buildTokenService(
-            NameNode.getAddress(nameNodeUri));
+            DFSUtilClient.getNNAddress(nameNodeUri));
       }
       return new ProxyAndInfo<T>(proxy, dtService,
-          NameNode.getAddress(nameNodeUri));
+          DFSUtilClient.getNNAddress(nameNodeUri));
     } else {
       LOG.warn("Currently creating proxy using " +
       		"LossyRetryInvocationHandler requires NN HA setup");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java
index 402dcdd..8edaed6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java
@@ -26,6 +26,7 @@ import javax.servlet.http.HttpServletRequest;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
@@ -77,7 +78,7 @@ abstract class DfsServlet extends HttpServlet {
       NameNodeHttpServer.getNameNodeAddressFromContext(context);
     Configuration conf = new HdfsConfiguration(
         NameNodeHttpServer.getConfFromContext(context));
-    return NameNodeProxies.createProxy(conf, NameNode.getUri(nnAddr),
+    return NameNodeProxies.createProxy(conf, DFSUtilClient.getNNUri(nnAddr),
         ClientProtocol.class).getProxy();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java
index 9dc20b5..cdf34c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java
@@ -32,6 +32,7 @@ import javax.servlet.http.HttpServletResponse;
 
 import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -237,7 +238,7 @@ public class ImageServlet extends HttpServlet {
 
     validRequestors.add(SecurityUtil.getServerPrincipal(conf
         .get(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY),
-        NameNode.getAddress(conf).getHostName()));
+        DFSUtilClient.getNNAddress(conf).getHostName()));
     try {
       validRequestors.add(
           SecurityUtil.getServerPrincipal(conf
@@ -261,7 +262,7 @@ public class ImageServlet extends HttpServlet {
       for (Configuration otherNnConf : otherNnConfs) {
         validRequestors.add(SecurityUtil.getServerPrincipal(otherNnConf
                 .get(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY),
-            NameNode.getAddress(otherNnConf).getHostName()));
+            DFSUtilClient.getNNAddress(otherNnConf).getHostName()));
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index df25d59..0431fee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -445,10 +445,6 @@ public class NameNode implements NameNodeStatusMXBean {
     return clientNamenodeAddress;
   }
 
-  public static InetSocketAddress getAddress(String address) {
-    return NetUtils.createSocketAddr(address, DFS_NAMENODE_RPC_PORT_DEFAULT);
-  }
-  
   /**
    * Set the configuration property for the service rpc address
    * to address
@@ -470,45 +466,18 @@ public class NameNode implements NameNodeStatusMXBean {
                                                         boolean fallback) {
     String addr = conf.getTrimmed(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY);
     if (addr == null || addr.isEmpty()) {
-      return fallback ? getAddress(conf) : null;
+      return fallback ? DFSUtilClient.getNNAddress(conf) : null;
     }
-    return getAddress(addr);
-  }
-
-  public static InetSocketAddress getAddress(Configuration conf) {
-    URI filesystemURI = FileSystem.getDefaultUri(conf);
-    return getAddress(filesystemURI);
+    return DFSUtilClient.getNNAddress(addr);
   }
 
-
+  @Deprecated
   /**
-   * @return address of file system
+   * @deprecated Use {@link DFSUtilClient#getNNUri(InetSocketAddress)} instead.
    */
-  public static InetSocketAddress getAddress(URI filesystemURI) {
-    String authority = filesystemURI.getAuthority();
-    if (authority == null) {
-      throw new IllegalArgumentException(String.format(
-          "Invalid URI for NameNode address (check %s): %s has no authority.",
-          FileSystem.FS_DEFAULT_NAME_KEY, filesystemURI.toString()));
-    }
-    if (!HdfsConstants.HDFS_URI_SCHEME.equalsIgnoreCase(
-        filesystemURI.getScheme())) {
-      throw new IllegalArgumentException(String.format(
-          "Invalid URI for NameNode address (check %s): %s is not of scheme '%s'.",
-          FileSystem.FS_DEFAULT_NAME_KEY, filesystemURI.toString(),
-          HdfsConstants.HDFS_URI_SCHEME));
-    }
-    return getAddress(authority);
-  }
-
   public static URI getUri(InetSocketAddress namenode) {
-    int port = namenode.getPort();
-    String portString = (port == DFS_NAMENODE_RPC_PORT_DEFAULT) ?
-        "" : (":" + port);
-    return URI.create(HdfsConstants.HDFS_URI_SCHEME + "://"
-        + namenode.getHostName()+portString);
+    return DFSUtilClient.getNNUri(namenode);
   }
-
   //
   // Common NameNode methods implementation for the active name-node role.
   //
@@ -529,7 +498,7 @@ public class NameNode implements NameNodeStatusMXBean {
   }
 
   protected InetSocketAddress getRpcServerAddress(Configuration conf) {
-    return getAddress(conf);
+    return DFSUtilClient.getNNAddress(conf);
   }
   
   /** Given a configuration get the bind host of the service rpc server
@@ -564,7 +533,7 @@ public class NameNode implements NameNodeStatusMXBean {
 
   protected void setRpcServerAddress(Configuration conf,
       InetSocketAddress rpcAddress) {
-    FileSystem.setDefaultUri(conf, getUri(rpcAddress));
+    FileSystem.setDefaultUri(conf, DFSUtilClient.getNNUri(rpcAddress));
   }
 
   protected InetSocketAddress getHttpServerAddress(Configuration conf) {
@@ -1012,7 +981,7 @@ public class NameNode implements NameNodeStatusMXBean {
     checkAllowFormat(conf);
 
     if (UserGroupInformation.isSecurityEnabled()) {
-      InetSocketAddress socAddr = getAddress(conf);
+      InetSocketAddress socAddr = DFSUtilClient.getNNAddress(conf);
       SecurityUtil.login(conf, DFS_NAMENODE_KEYTAB_FILE_KEY,
           DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY, socAddr.getHostName());
     }
@@ -1115,7 +1084,7 @@ public class NameNode implements NameNodeStatusMXBean {
     }
 
     if (UserGroupInformation.isSecurityEnabled()) {
-      InetSocketAddress socAddr = getAddress(conf);
+      InetSocketAddress socAddr = DFSUtilClient.getNNAddress(conf);
       SecurityUtil.login(conf, DFS_NAMENODE_KEYTAB_FILE_KEY,
           DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY, socAddr.getHostName());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/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 b5dbc46..2dbc15a 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
@@ -758,7 +758,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
 
   private void copyBlocksToLostFound(String parent, HdfsFileStatus file,
         LocatedBlocks blocks) throws IOException {
-    final DFSClient dfs = new DFSClient(NameNode.getAddress(conf), conf);
+    final DFSClient dfs = new DFSClient(DFSUtilClient.getNNAddress(conf), conf);
     final String fullName = file.getFullName(parent);
     OutputStream fos = null;
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
index c22d7f1..a84dcf1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -101,7 +102,7 @@ public class BootstrapStandby implements Tool, Configurable {
     parseConfAndFindOtherNN();
     NameNode.checkAllowFormat(conf);
 
-    InetSocketAddress myAddr = NameNode.getAddress(conf);
+    InetSocketAddress myAddr = DFSUtilClient.getNNAddress(conf);
     SecurityUtil.login(conf, DFS_NAMENODE_KEYTAB_FILE_KEY,
         DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY, myAddr.getHostName());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java
index bc4e726..4e1cb9e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java
@@ -24,9 +24,9 @@ import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -91,7 +91,7 @@ public class IPFailoverProxyProvider<T> extends
     if (nnProxyInfo == null) {
       try {
         // Create a proxy that is not wrapped in RetryProxy
-        InetSocketAddress nnAddr = NameNode.getAddress(nameNodeUri);
+        InetSocketAddress nnAddr = DFSUtilClient.getNNAddress(nameNodeUri);
         nnProxyInfo = new ProxyInfo<T>(NameNodeProxies.createNonHAProxy(
             conf, nnAddr, xface, UserGroupInformation.getCurrentUser(), 
             false).getProxy(), nnAddr.toString());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
index 24e5bef..01cd934 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.ha.HealthMonitor;
 import org.apache.hadoop.ha.ZKFailoverController;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -166,7 +167,7 @@ public class DFSZKFailoverController extends ZKFailoverController {
 
   @Override
   public void loginAsFCUser() throws IOException {
-    InetSocketAddress socAddr = NameNode.getAddress(conf);
+    InetSocketAddress socAddr = DFSUtilClient.getNNAddress(conf);
     SecurityUtil.login(conf, DFS_NAMENODE_KEYTAB_FILE_KEY,
         DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY, socAddr.getHostName());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetGroups.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetGroups.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetGroups.java
index 3c6d4c5..e03e787 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetGroups.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetGroups.java
@@ -29,9 +29,9 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.NameNodeProxies;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.tools.GetGroupsBase;
 import org.apache.hadoop.tools.GetUserMappingsProtocol;
 import org.apache.hadoop.util.ToolRunner;
@@ -63,7 +63,7 @@ public class GetGroups extends GetGroupsBase {
   @Override
   protected InetSocketAddress getProtocolAddress(Configuration conf)
       throws IOException {
-    return NameNode.getAddress(conf);
+    return DFSUtilClient.getNNAddress(conf);
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/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 3082544..a7e80ca 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
@@ -1797,8 +1797,8 @@ public class DFSTestUtil {
       URI nameNodeUri, UserGroupInformation ugi)
       throws IOException {
     return NameNodeProxies.createNonHAProxy(conf,
-        NameNode.getAddress(nameNodeUri), NamenodeProtocol.class, ugi, false).
-        getProxy();
+        DFSUtilClient.getNNAddress(nameNodeUri), NamenodeProtocol.class, ugi,
+        false).getProxy();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java
index ff5554a..b098711 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java
@@ -300,7 +300,7 @@ public class TestDFSClientFailover {
         Class<T> xface) {
       try {
         this.proxy = NameNodeProxies.createNonHAProxy(conf,
-            NameNode.getAddress(uri), xface,
+            DFSUtilClient.getNNAddress(uri), xface,
             UserGroupInformation.getCurrentUser(), false).getProxy();
         this.xface = xface;
       } catch (IOException ioe) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellGenericOptions.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellGenericOptions.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellGenericOptions.java
index 2b37e2e..282dcf7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellGenericOptions.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellGenericOptions.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.Test;
 
@@ -53,7 +52,7 @@ public class TestDFSShellGenericOptions {
     }
   }
 
-  private void testFsOption(String [] args, String namenode) {        
+  private void testFsOption(String [] args, String namenode) {
     // prepare arguments to create a directory /data
     args[0] = "-fs";
     args[1] = namenode;
@@ -81,7 +80,7 @@ public class TestDFSShellGenericOptions {
       // prepare arguments to create a directory /data
       args[0] = "-conf";
       args[1] = siteFile.getPath();
-      execute(args, namenode); 
+      execute(args, namenode);
     } catch (FileNotFoundException e) {
       e.printStackTrace();
     } finally {
@@ -94,7 +93,7 @@ public class TestDFSShellGenericOptions {
     // prepare arguments to create a directory /data
     args[0] = "-D";
     args[1] = "fs.defaultFS="+namenode;
-    execute(args, namenode);        
+    execute(args, namenode);
   }
     
   private void execute(String [] args, String namenode) {
@@ -102,9 +101,9 @@ public class TestDFSShellGenericOptions {
     FileSystem fs=null;
     try {
       ToolRunner.run(shell, args);
-      fs = FileSystem.get(NameNode.getUri(NameNode.getAddress(namenode)),
-          shell.getConf());
-      assertTrue("Directory does not get created", 
+      fs = FileSystem.get(DFSUtilClient.getNNUri(
+          DFSUtilClient.getNNAddress(namenode)), shell.getConf());
+      assertTrue("Directory does not get created",
                  fs.isDirectory(new Path("/data")));
       fs.delete(new Path("/data"), true);
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDefaultNameNodePort.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDefaultNameNodePort.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDefaultNameNodePort.java
index 38be3c8..1d8d289 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDefaultNameNodePort.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDefaultNameNodePort.java
@@ -34,13 +34,13 @@ public class TestDefaultNameNodePort {
 
   @Test
   public void testGetAddressFromString() throws Exception {
-    assertEquals(NameNode.getAddress("foo").getPort(),
+    assertEquals(DFSUtilClient.getNNAddress("foo").getPort(),
                  HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT);
-    assertEquals(NameNode.getAddress("hdfs://foo/").getPort(),
+    assertEquals(DFSUtilClient.getNNAddress("hdfs://foo/").getPort(),
                  HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT);
-    assertEquals(NameNode.getAddress("hdfs://foo:555").getPort(),
+    assertEquals(DFSUtilClient.getNNAddress("hdfs://foo:555").getPort(),
                  555);
-    assertEquals(NameNode.getAddress("foo:555").getPort(),
+    assertEquals(DFSUtilClient.getNNAddress("foo:555").getPort(),
                  555);
   }
 
@@ -48,20 +48,20 @@ public class TestDefaultNameNodePort {
   public void testGetAddressFromConf() throws Exception {
     Configuration conf = new HdfsConfiguration();
     FileSystem.setDefaultUri(conf, "hdfs://foo/");
-    assertEquals(NameNode.getAddress(conf).getPort(),
+    assertEquals(DFSUtilClient.getNNAddress(conf).getPort(),
         HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT);
     FileSystem.setDefaultUri(conf, "hdfs://foo:555/");
-    assertEquals(NameNode.getAddress(conf).getPort(), 555);
+    assertEquals(DFSUtilClient.getNNAddress(conf).getPort(), 555);
     FileSystem.setDefaultUri(conf, "foo");
-    assertEquals(NameNode.getAddress(conf).getPort(),
+    assertEquals(DFSUtilClient.getNNAddress(conf).getPort(),
         HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT);
   }
 
   @Test
   public void testGetUri() {
-    assertEquals(NameNode.getUri(new InetSocketAddress("foo", 555)),
+    assertEquals(DFSUtilClient.getNNUri(new InetSocketAddress("foo", 555)),
                  URI.create("hdfs://foo:555"));
-    assertEquals(NameNode.getUri(new InetSocketAddress("foo",
+    assertEquals(DFSUtilClient.getNNUri(new InetSocketAddress("foo",
             HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT)),
         URI.create("hdfs://foo"));
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
index d6decd9..4f4897f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
@@ -35,12 +35,9 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.log4j.Level;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -73,7 +70,7 @@ public class TestFileStatus {
     cluster = new MiniDFSCluster.Builder(conf).build();
     fs = cluster.getFileSystem();
     fc = FileContext.getFileContext(cluster.getURI(0), conf);
-    dfsClient = new DFSClient(NameNode.getAddress(conf), conf);
+    dfsClient = new DFSClient(DFSUtilClient.getNNAddress(conf), conf);
     file1 = new Path("filestatus.dat");
     DFSTestUtil.createFile(fs, file1, fileSize, fileSize, blockSize, (short) 1,
         seed);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
index 103151e..741e641 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.ipc.RemoteException;
@@ -193,8 +192,8 @@ public class TestGetBlocks {
       DatanodeInfo[] dataNodes = null;
       boolean notWritten;
       do {
-        final DFSClient dfsclient = new DFSClient(NameNode.getAddress(CONF),
-            CONF);
+        final DFSClient dfsclient = new DFSClient(
+            DFSUtilClient.getNNAddress(CONF), CONF);
         locatedBlocks = dfsclient.getNamenode()
             .getBlockLocations("/tmp.txt", 0, fileLen).getLocatedBlocks();
         assertEquals(2, locatedBlocks.size());
@@ -216,7 +215,7 @@ public class TestGetBlocks {
       InetSocketAddress addr = new InetSocketAddress("localhost",
           cluster.getNameNodePort());
       NamenodeProtocol namenode = NameNodeProxies.createProxy(CONF,
-          NameNode.getUri(addr), NamenodeProtocol.class).getProxy();
+          DFSUtilClient.getNNUri(addr), NamenodeProtocol.class).getProxy();
 
       // get blocks of size fileLen from dataNodes[0]
       BlockWithLocations[] locs;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java
index 0266b5e..43b578f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
@@ -225,7 +224,7 @@ public class TestPersistBlocks {
     try {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
       FileSystem fs = cluster.getFileSystem();
-      NameNode.getAddress(conf).getPort();
+      DFSUtilClient.getNNAddress(conf).getPort();
       // Creating a file with 4096 blockSize to write multiple blocks
       stream = fs.create(FILE_PATH, true, BLOCK_SIZE, (short) 1, BLOCK_SIZE);
       stream.write(DATA_BEFORE_RESTART);
@@ -274,7 +273,7 @@ public class TestPersistBlocks {
     try {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
       FileSystem fs = cluster.getFileSystem();
-      NameNode.getAddress(conf).getPort();
+      DFSUtilClient.getNNAddress(conf).getPort();
       // Creating a file with 4096 blockSize to write multiple blocks
       stream = fs.create(FILE_PATH, true, BLOCK_SIZE, (short) 1, BLOCK_SIZE);
       stream.write(DATA_BEFORE_RESTART, 0, DATA_BEFORE_RESTART.length / 2);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeRollingUpgrade.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeRollingUpgrade.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeRollingUpgrade.java
index 57fee06..7e56988 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeRollingUpgrade.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeRollingUpgrade.java
@@ -34,13 +34,13 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSOutputStream;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster.Builder;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.TestRollingUpgrade;
-import org.apache.hadoop.hdfs.client.BlockReportOptions;
 import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -294,9 +294,9 @@ public class TestDataNodeRollingUpgrade {
       String testFile2 = "/" + GenericTestUtils.getMethodName() + ".02.dat";
       String testFile3 = "/" + GenericTestUtils.getMethodName() + ".03.dat";
 
-      DFSClient client1 = new DFSClient(NameNode.getAddress(conf), conf);
-      DFSClient client2 = new DFSClient(NameNode.getAddress(conf), conf);
-      DFSClient client3 = new DFSClient(NameNode.getAddress(conf), conf);
+      DFSClient client1 = new DFSClient(DFSUtilClient.getNNAddress(conf), conf);
+      DFSClient client2 = new DFSClient(DFSUtilClient.getNNAddress(conf), conf);
+      DFSClient client3 = new DFSClient(DFSUtilClient.getNNAddress(conf), conf);
 
       DFSOutputStream s1 = (DFSOutputStream) client1.create(testFile1, true);
       DFSOutputStream s2 = (DFSOutputStream) client2.create(testFile2, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
index 39894b5..b963d8f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -1497,7 +1498,7 @@ public class NNThroughputBenchmark implements Tool {
             UserGroupInformation.getCurrentUser());
         clientProto = dfs.getClient().getNamenode();
         dataNodeProto = new DatanodeProtocolClientSideTranslatorPB(
-            NameNode.getAddress(nnUri), config);
+            DFSUtilClient.getNNAddress(nnUri), config);
         refreshUserMappingsProto =
             DFSTestUtil.getRefreshUserMappingsProtocolProxy(config, nnUri);
         getBlockPoolId(dfs);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/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 5735dc2..9f5850a 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
@@ -53,9 +53,9 @@ import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -978,7 +978,7 @@ public class TestINodeFile {
       long parentId = fsdir.getINode("/").getId();
       String testPath = "/.reserved/.inodes/" + dirId + "/..";
 
-      client = new DFSClient(NameNode.getAddress(conf), conf);
+      client = new DFSClient(DFSUtilClient.getNNAddress(conf), conf);
       HdfsFileStatus status = client.getFileInfo(testPath);
       assertTrue(parentId == status.getFileId());
       

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eee9750/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
index 5ddc635..f5a72ca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
@@ -246,7 +247,7 @@ public class TestFailureToReadEdits {
     FileSystem fs0 = null;
     try {
       // Make sure that when the active restarts, it loads all the edits.
-      fs0 = FileSystem.get(NameNode.getUri(nn0.getNameNodeAddress()),
+      fs0 = FileSystem.get(DFSUtilClient.getNNUri(nn0.getNameNodeAddress()),
           conf);
       
       assertTrue(fs0.exists(new Path(TEST_DIR1)));


[22/50] [abbrv] hadoop git commit: HDFS-5802. NameNode does not check for inode type before traversing down a path. (Xiao Chen via Yongjun Zhang)

Posted by zh...@apache.org.
HDFS-5802. NameNode does not check for inode type before traversing down a path. (Xiao Chen via Yongjun Zhang)


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

Branch: refs/heads/HDFS-7285
Commit: 2ff6faf954eb0f1ab2b339d589edb30040087669
Parents: 92c1af1
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Thu Sep 17 22:56:14 2015 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Fri Sep 18 07:17:30 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../server/namenode/FSPermissionChecker.java    | 25 +++++++++++-
 .../apache/hadoop/hdfs/TestDFSPermission.java   | 42 +++++++++++++++++++-
 3 files changed, 67 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ff6faf9/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 4912f50..f9837f3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -932,6 +932,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9022. Move NameNode.getAddress() and NameNode.getUri() to
     hadoop-hdfs-client. (Mingliang Liu via wheat9)
 
+    HDFS-5802. NameNode does not check for inode type before traversing down a
+    path. (Xiao Chen via Yongjun Zhang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ff6faf9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
index 041ce0b..9edcda1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
@@ -192,6 +192,25 @@ class FSPermissionChecker implements AccessControlEnforcer {
         ancestorAccess, parentAccess, access, subAccess, ignoreEmptyDir);
   }
 
+  /**
+   * Check whether exception e is due to an ancestor inode's not being
+   * directory.
+   */
+  private void checkAncestorType(INode[] inodes, int ancestorIndex,
+      AccessControlException e) throws AccessControlException {
+    for (int i = 0; i <= ancestorIndex; i++) {
+      if (inodes[i] == null) {
+        break;
+      }
+      if (!inodes[i].isDirectory()) {
+        throw new AccessControlException(
+            e.getMessage() + " (Ancestor " + inodes[i].getFullPathName()
+                + " is not a directory).");
+      }
+    }
+    throw e;
+  }
+
   @Override
   public void checkPermission(String fsOwner, String supergroup,
       UserGroupInformation callerUgi, INodeAttributes[] inodeAttrs,
@@ -202,7 +221,11 @@ class FSPermissionChecker implements AccessControlEnforcer {
       throws AccessControlException {
     for(; ancestorIndex >= 0 && inodes[ancestorIndex] == null;
         ancestorIndex--);
-    checkTraverse(inodeAttrs, path, ancestorIndex);
+    try {
+      checkTraverse(inodeAttrs, path, ancestorIndex);
+    } catch (AccessControlException e) {
+      checkAncestorType(inodes, ancestorIndex, e);
+    }
 
     final INodeAttributes last = inodeAttrs[inodeAttrs.length - 1];
     if (parentAccess != null && parentAccess.implies(FsAction.WRITE)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ff6faf9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java
index 23ce916..80b2eb4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.DataOutputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
@@ -510,8 +511,45 @@ public class TestDFSPermission {
     }
   }
 
-  /* Check if namenode performs permission checking correctly 
-   * for the given user for operations mkdir, open, setReplication, 
+  @Test
+  public void testPermissionMessageOnNonDirAncestor()
+      throws IOException, InterruptedException {
+    FileSystem rootFs = FileSystem.get(conf);
+    Path p4 = new Path("/p4");
+    rootFs.mkdirs(p4);
+    rootFs.setOwner(p4, USER1_NAME, GROUP1_NAME);
+
+    final Path fpath = new Path("/p4/file");
+    DataOutputStream out = rootFs.create(fpath);
+    out.writeBytes("dhruba: " + fpath);
+    out.close();
+    rootFs.setOwner(fpath, USER1_NAME, GROUP1_NAME);
+    assertTrue(rootFs.exists(fpath));
+
+    fs = USER1.doAs(new PrivilegedExceptionAction<FileSystem>() {
+      @Override
+      public FileSystem run() throws Exception {
+        return FileSystem.get(conf);
+      }
+    });
+
+    final Path nfpath = new Path("/p4/file/nonexisting");
+    assertFalse(rootFs.exists(nfpath));
+
+    try {
+      fs.exists(nfpath);
+      fail("The exists call should have failed.");
+    } catch (AccessControlException e) {
+      assertTrue("Permission denied messages must carry file path",
+          e.getMessage().contains(fpath.getName()));
+      assertTrue("Permission denied messages must specify existing_file is not "
+              + "a directory, when checked on /existing_file/non_existing_name",
+          e.getMessage().contains("is not a directory"));
+    }
+  }
+
+  /* Check if namenode performs permission checking correctly
+   * for the given user for operations mkdir, open, setReplication,
    * getFileInfo, isDirectory, exists, getContentLength, list, rename,
    * and delete */
   private void testPermissionCheckingPerUser(UserGroupInformation ugi,