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 um...@apache.org on 2016/10/17 23:14:20 UTC

[01/50] hadoop git commit: HDFS-10916. Switch from "raw" to "system" xattr namespace for erasure coding policy. (Andrew Wang via lei) [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-10285 a88d66e25 -> 13a805035 (forced update)


HDFS-10916. Switch from "raw" to "system" xattr namespace for erasure coding policy. (Andrew Wang via lei)


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

Branch: refs/heads/HDFS-10285
Commit: 809cfd27a30900d2c0e0e133574de49d0b4538cf
Parents: ecb51b8
Author: Lei Xu <le...@apache.org>
Authored: Tue Oct 11 10:04:46 2016 -0700
Committer: Lei Xu <le...@apache.org>
Committed: Tue Oct 11 10:04:46 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/809cfd27/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
index 3798394..d112a48 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
@@ -369,7 +369,7 @@ public interface HdfsServerConstants {
   String SECURITY_XATTR_UNREADABLE_BY_SUPERUSER =
       "security.hdfs.unreadable.by.superuser";
   String XATTR_ERASURECODING_POLICY =
-      "raw.hdfs.erasurecoding.policy";
+      "system.hdfs.erasurecoding.policy";
 
   long BLOCK_GROUP_INDEX_MASK = 15;
   byte MAX_BLOCKS_IN_GROUP = 16;


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


[09/50] hadoop git commit: HDFS-10965. Add unit test for HDFS command 'dfsadmin -printTopology'. Contributed by Xiaobing Zhou

Posted by um...@apache.org.
HDFS-10965. Add unit test for HDFS command 'dfsadmin -printTopology'. Contributed by Xiaobing Zhou


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

Branch: refs/heads/HDFS-10285
Commit: 7ba7092bbcbbccfa24b672414d315656e600096c
Parents: b84c489
Author: Mingliang Liu <li...@apache.org>
Authored: Tue Oct 11 16:47:39 2016 -0700
Committer: Mingliang Liu <li...@apache.org>
Committed: Tue Oct 11 17:23:54 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/tools/TestDFSAdmin.java  | 50 ++++++++++++++++++++
 1 file changed, 50 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ba7092b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
index 94ecb9e..b49f73d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.After;
 import org.junit.Before;
@@ -364,6 +365,55 @@ public class TestDFSAdmin {
   }
 
   @Test(timeout = 30000)
+  public void testPrintTopology() throws Exception {
+    redirectStream();
+
+    /* init conf */
+    final Configuration dfsConf = new HdfsConfiguration();
+    final File baseDir = new File(
+        PathUtils.getTestDir(getClass()),
+        GenericTestUtils.getMethodName());
+    dfsConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath());
+
+    final int numDn = 4;
+    final String[] racks = {
+        "/d1/r1", "/d1/r2",
+        "/d2/r1", "/d2/r2"};
+
+    /* init cluster using topology */
+    try (MiniDFSCluster miniCluster = new MiniDFSCluster.Builder(dfsConf)
+        .numDataNodes(numDn).racks(racks).build()) {
+
+      miniCluster.waitActive();
+      assertEquals(numDn, miniCluster.getDataNodes().size());
+      final DFSAdmin dfsAdmin = new DFSAdmin(dfsConf);
+
+      resetStream();
+      final int ret = ToolRunner.run(dfsAdmin, new String[] {"-printTopology"});
+
+      /* collect outputs */
+      final List<String> outs = Lists.newArrayList();
+      scanIntoList(out, outs);
+
+      /* verify results */
+      assertEquals(0, ret);
+      assertEquals(
+          "There should be three lines per Datanode: the 1st line is"
+              + " rack info, 2nd node info, 3rd empty line. The total"
+              + " should be as a result of 3 * numDn.",
+          12, outs.size());
+      assertThat(outs.get(0),
+          is(allOf(containsString("Rack:"), containsString("/d1/r1"))));
+      assertThat(outs.get(3),
+          is(allOf(containsString("Rack:"), containsString("/d1/r2"))));
+      assertThat(outs.get(6),
+          is(allOf(containsString("Rack:"), containsString("/d2/r1"))));
+      assertThat(outs.get(9),
+          is(allOf(containsString("Rack:"), containsString("/d2/r2"))));
+    }
+  }
+
+  @Test(timeout = 30000)
   public void testNameNodeGetReconfigurationStatus() throws IOException,
       InterruptedException, TimeoutException {
     ReconfigurationUtil ru = mock(ReconfigurationUtil.class);


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


[07/50] hadoop git commit: HDFS-10903. Replace config key literal strings with config key names II: hadoop hdfs. Contributed by Chen Liang

Posted by um...@apache.org.
HDFS-10903. Replace config key literal strings with config key names II: hadoop hdfs. Contributed by Chen Liang


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

Branch: refs/heads/HDFS-10285
Commit: 3c9a01062e9097c2ed1db75318482543db2e382f
Parents: 61f0490
Author: Mingliang Liu <li...@apache.org>
Authored: Tue Oct 11 16:29:30 2016 -0700
Committer: Mingliang Liu <li...@apache.org>
Committed: Tue Oct 11 16:29:30 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/http/server/FSOperations.java | 9 +++++++--
 .../hadoop/lib/service/hadoop/FileSystemAccessService.java  | 6 ++++--
 .../src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java | 3 +++
 .../hadoop-hdfs/src/main/resources/hdfs-default.xml         | 8 ++++++++
 .../test/java/org/apache/hadoop/hdfs/TestFileAppend4.java   | 3 ++-
 .../hdfs/server/blockmanagement/TestBlockTokenWithDFS.java  | 3 ++-
 6 files changed, 26 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c9a0106/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
index 46948f9..001bc92 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
@@ -48,6 +48,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.HTTPFS_BUFFER_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.HTTP_BUFFER_SIZE_DEFAULT;
+
 /**
  * FileSystem operation executors used by {@link HttpFSServer}.
  */
@@ -462,7 +465,8 @@ public class FSOperations {
         blockSize = fs.getDefaultBlockSize(path);
       }
       FsPermission fsPermission = new FsPermission(permission);
-      int bufferSize = fs.getConf().getInt("httpfs.buffer.size", 4096);
+      int bufferSize = fs.getConf().getInt(HTTPFS_BUFFER_SIZE_KEY,
+          HTTP_BUFFER_SIZE_DEFAULT);
       OutputStream os = fs.create(path, fsPermission, override, bufferSize, replication, blockSize, null);
       IOUtils.copyBytes(is, os, bufferSize, true);
       os.close();
@@ -752,7 +756,8 @@ public class FSOperations {
      */
     @Override
     public InputStream execute(FileSystem fs) throws IOException {
-      int bufferSize = HttpFSServerWebApp.get().getConfig().getInt("httpfs.buffer.size", 4096);
+      int bufferSize = HttpFSServerWebApp.get().getConfig().getInt(
+          HTTPFS_BUFFER_SIZE_KEY, HTTP_BUFFER_SIZE_DEFAULT);
       return fs.open(path, bufferSize);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c9a0106/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java
index 0b767be..61d3b45 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java
@@ -50,6 +50,8 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
+
 @InterfaceAudience.Private
 public class FileSystemAccessService extends BaseService implements FileSystemAccess {
   private static final Logger LOG = LoggerFactory.getLogger(FileSystemAccessService.class);
@@ -159,7 +161,7 @@ public class FileSystemAccessService extends BaseService implements FileSystemAc
         throw new ServiceException(FileSystemAccessException.ERROR.H01, KERBEROS_PRINCIPAL);
       }
       Configuration conf = new Configuration();
-      conf.set("hadoop.security.authentication", "kerberos");
+      conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
       UserGroupInformation.setConfiguration(conf);
       try {
         UserGroupInformation.loginUserFromKeytab(principal, keytab);
@@ -169,7 +171,7 @@ public class FileSystemAccessService extends BaseService implements FileSystemAc
       LOG.info("Using FileSystemAccess Kerberos authentication, principal [{}] keytab [{}]", principal, keytab);
     } else if (security.equals("simple")) {
       Configuration conf = new Configuration();
-      conf.set("hadoop.security.authentication", "simple");
+      conf.set(HADOOP_SECURITY_AUTHENTICATION, "simple");
       UserGroupInformation.setConfiguration(conf);
       LOG.info("Using FileSystemAccess simple/pseudo authentication, principal [{}]", System.getProperty("user.name"));
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c9a0106/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index df45e2a..18209ae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -992,6 +992,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.disk.balancer.plan.threshold.percent";
   public static final int DFS_DISK_BALANCER_PLAN_THRESHOLD_DEFAULT = 10;
 
+  public static final String HTTPFS_BUFFER_SIZE_KEY =
+      "httpfs.buffer.size";
+  public static final int HTTP_BUFFER_SIZE_DEFAULT = 4096;
 
   // dfs.client.retry confs are moved to HdfsClientConfigKeys.Retry 
   @Deprecated

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c9a0106/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 672b597..db4035d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -4273,4 +4273,12 @@
       consecutive warnings within this interval.</description>
   </property>
 
+  <property>
+    <name>httpfs.buffer.size</name>
+    <value>4096</value>
+    <description>
+      The size buffer to be used when creating or opening httpfs filesystem IO stream.
+    </description>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c9a0106/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
index 4147851..ae0f0c2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -89,7 +90,7 @@ public class TestFileAppend4 {
     
     // handle failures in the DFSClient pipeline quickly
     // (for cluster.shutdown(); fs.close() idiom)
-    conf.setInt("ipc.client.connect.max.retries", 1);
+    conf.setInt(IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 1);
   }
   
   /*

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c9a0106/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
index 9374ae8..5a8a39a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -218,7 +219,7 @@ public class TestBlockTokenWithDFS {
     conf.setInt("io.bytes.per.checksum", BLOCK_SIZE);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, numDataNodes);
-    conf.setInt("ipc.client.connect.max.retries", 0);
+    conf.setInt(IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
     // Set short retry timeouts so this test runs faster
     conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10);
     return conf;


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


[44/50] hadoop git commit: HADOOP-13724. Fix a few typos in site markdown documents. Contributed by Ding Fei.

Posted by um...@apache.org.
HADOOP-13724. Fix a few typos in site markdown documents. Contributed by Ding Fei.


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

Branch: refs/heads/HDFS-10285
Commit: 987ee51141a15d3f4d1df4dc792a192b92b87b5f
Parents: 0f4afc8
Author: Andrew Wang <wa...@apache.org>
Authored: Mon Oct 17 13:25:58 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Mon Oct 17 13:25:58 2016 -0700

----------------------------------------------------------------------
 .../src/site/markdown/ClusterSetup.md           |  2 +-
 .../src/site/markdown/Compatibility.md          | 16 +++++------
 .../site/markdown/InterfaceClassification.md    | 28 ++++++++++----------
 .../src/site/markdown/filesystem/filesystem.md  | 17 ++++++------
 .../markdown/filesystem/fsdatainputstream.md    | 16 +++++------
 .../site/markdown/filesystem/introduction.md    | 12 ++++-----
 .../src/site/markdown/filesystem/model.md       |  7 ++---
 .../src/site/markdown/filesystem/notation.md    |  2 +-
 .../src/site/markdown/filesystem/testing.md     |  4 +--
 .../src/site/markdown/HadoopArchives.md.vm      |  2 +-
 10 files changed, 53 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/987ee511/hadoop-common-project/hadoop-common/src/site/markdown/ClusterSetup.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/ClusterSetup.md b/hadoop-common-project/hadoop-common/src/site/markdown/ClusterSetup.md
index f222769..56b43e6 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/ClusterSetup.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/ClusterSetup.md
@@ -35,7 +35,7 @@ Installation
 
 Installing a Hadoop cluster typically involves unpacking the software on all the machines in the cluster or installing it via a packaging system as appropriate for your operating system. It is important to divide up the hardware into functions.
 
-Typically one machine in the cluster is designated as the NameNode and another machine the as ResourceManager, exclusively. These are the masters. Other services (such as Web App Proxy Server and MapReduce Job History server) are usually run either on dedicated hardware or on shared infrastrucutre, depending upon the load.
+Typically one machine in the cluster is designated as the NameNode and another machine as the ResourceManager, exclusively. These are the masters. Other services (such as Web App Proxy Server and MapReduce Job History server) are usually run either on dedicated hardware or on shared infrastructure, depending upon the load.
 
 The rest of the machines in the cluster act as both DataNode and NodeManager. These are the workers.
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/987ee511/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md b/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
index d7827b5..05b18b5 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
@@ -68,7 +68,7 @@ Wire compatibility concerns data being transmitted over the wire between Hadoop
 #### Use Cases
 
 * Client-Server compatibility is required to allow users to continue using the old clients even after upgrading the server (cluster) to a later version (or vice versa). For example, a Hadoop 2.1.0 client talking to a Hadoop 2.3.0 cluster.
-* Client-Server compatibility is also required to allow users to upgrade the client before upgrading the server (cluster). For example, a Hadoop 2.4.0 client talking to a Hadoop 2.3.0 cluster. This allows deployment of client-side bug fixes ahead of full cluster upgrades. Note that new cluster features invoked by new client APIs or shell commands will not be usable. YARN applications that attempt to use new APIs (including new fields in data structures) that have not yet deployed to the cluster can expect link exceptions.
+* Client-Server compatibility is also required to allow users to upgrade the client before upgrading the server (cluster). For example, a Hadoop 2.4.0 client talking to a Hadoop 2.3.0 cluster. This allows deployment of client-side bug fixes ahead of full cluster upgrades. Note that new cluster features invoked by new client APIs or shell commands will not be usable. YARN applications that attempt to use new APIs (including new fields in data structures) that have not yet been deployed to the cluster can expect link exceptions.
 * Client-Server compatibility is also required to allow upgrading individual components without upgrading others. For example, upgrade HDFS from version 2.1.0 to 2.2.0 without upgrading MapReduce.
 * Server-Server compatibility is required to allow mixed versions within an active cluster so the cluster may be upgraded without downtime in a rolling fashion.
 
@@ -76,7 +76,7 @@ Wire compatibility concerns data being transmitted over the wire between Hadoop
 
 * Both Client-Server and Server-Server compatibility is preserved within a major release. (Different policies for different categories are yet to be considered.)
 * Compatibility can be broken only at a major release, though breaking compatibility even at major releases has grave consequences and should be discussed in the Hadoop community.
-* Hadoop protocols are defined in .proto (ProtocolBuffers) files. Client-Server protocols and Server-protocol .proto files are marked as stable. When a .proto file is marked as stable it means that changes should be made in a compatible fashion as described below:
+* Hadoop protocols are defined in .proto (ProtocolBuffers) files. Client-Server protocols and Server-Server protocol .proto files are marked as stable. When a .proto file is marked as stable it means that changes should be made in a compatible fashion as described below:
     * The following changes are compatible and are allowed at any time:
         * Add an optional field, with the expectation that the code deals with the field missing due to communication with an older version of the code.
         * Add a new rpc/method to the service
@@ -101,7 +101,7 @@ Wire compatibility concerns data being transmitted over the wire between Hadoop
 
 ### Java Binary compatibility for end-user applications i.e. Apache Hadoop ABI
 
-As Apache Hadoop revisions are upgraded end-users reasonably expect that their applications should continue to work without any modifications. This is fulfilled as a result of support API compatibility, Semantic compatibility and Wire compatibility.
+As Apache Hadoop revisions are upgraded end-users reasonably expect that their applications should continue to work without any modifications. This is fulfilled as a result of supporting API compatibility, Semantic compatibility and Wire compatibility.
 
 However, Apache Hadoop is a very complex, distributed system and services a very wide variety of use-cases. In particular, Apache Hadoop MapReduce is a very, very wide API; in the sense that end-users may make wide-ranging assumptions such as layout of the local disk when their map/reduce tasks are executing, environment variables for their tasks etc. In such cases, it becomes very hard to fully specify, and support, absolute compatibility.
 
@@ -115,12 +115,12 @@ However, Apache Hadoop is a very complex, distributed system and services a very
 
 * Existing MapReduce, YARN & HDFS applications and frameworks should work unmodified within a major release i.e. Apache Hadoop ABI is supported.
 * A very minor fraction of applications maybe affected by changes to disk layouts etc., the developer community will strive to minimize these changes and will not make them within a minor version. In more egregious cases, we will consider strongly reverting these breaking changes and invalidating offending releases if necessary.
-* In particular for MapReduce applications, the developer community will try our best to support provide binary compatibility across major releases e.g. applications using org.apache.hadoop.mapred.
+* In particular for MapReduce applications, the developer community will try our best to support providing binary compatibility across major releases e.g. applications using org.apache.hadoop.mapred.
 * APIs are supported compatibly across hadoop-1.x and hadoop-2.x. See [Compatibility for MapReduce applications between hadoop-1.x and hadoop-2.x](../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduce_Compatibility_Hadoop1_Hadoop2.html) for more details.
 
 ### REST APIs
 
-REST API compatibility corresponds to both the request (URLs) and responses to each request (content, which may contain other URLs). Hadoop REST APIs are specifically meant for stable use by clients across releases, even major releases. The following are the exposed REST APIs:
+REST API compatibility corresponds to both the requests (URLs) and responses to each request (content, which may contain other URLs). Hadoop REST APIs are specifically meant for stable use by clients across releases, even major ones. The following are the exposed REST APIs:
 
 * [WebHDFS](../hadoop-hdfs/WebHDFS.html) - Stable
 * [ResourceManager](../../hadoop-yarn/hadoop-yarn-site/ResourceManagerRest.html)
@@ -136,7 +136,7 @@ The APIs annotated stable in the text above preserve compatibility across at lea
 
 ### Metrics/JMX
 
-While the Metrics API compatibility is governed by Java API compatibility, the actual metrics exposed by Hadoop need to be compatible for users to be able to automate using them (scripts etc.). Adding additional metrics is compatible. Modifying (eg changing the unit or measurement) or removing existing metrics breaks compatibility. Similarly, changes to JMX MBean object names also break compatibility.
+While the Metrics API compatibility is governed by Java API compatibility, the actual metrics exposed by Hadoop need to be compatible for users to be able to automate using them (scripts etc.). Adding additional metrics is compatible. Modifying (e.g. changing the unit or measurement) or removing existing metrics breaks compatibility. Similarly, changes to JMX MBean object names also break compatibility.
 
 #### Policy
 
@@ -148,7 +148,7 @@ User and system level data (including metadata) is stored in files of different
 
 #### User-level file formats
 
-Changes to formats that end-users use to store their data can prevent them for accessing the data in later releases, and hence it is highly important to keep those file-formats compatible. One can always add a "new" format improving upon an existing format. Examples of these formats include har, war, SequenceFileFormat etc.
+Changes to formats that end-users use to store their data can prevent them from accessing the data in later releases, and hence it is highly important to keep those file-formats compatible. One can always add a "new" format improving upon an existing format. Examples of these formats include har, war, SequenceFileFormat etc.
 
 ##### Policy
 
@@ -185,7 +185,7 @@ Depending on the degree of incompatibility in the changes, the following potenti
 
 ### Command Line Interface (CLI)
 
-The Hadoop command line programs may be use either directly via the system shell or via shell scripts. Changing the path of a command, removing or renaming command line options, the order of arguments, or the command return code and output break compatibility and may adversely affect users.
+The Hadoop command line programs may be used either directly via the system shell or via shell scripts. Changing the path of a command, removing or renaming command line options, the order of arguments, or the command return code and output break compatibility and may adversely affect users.
 
 #### Policy
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/987ee511/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md b/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
index 07abdac..c7309ab 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
@@ -44,7 +44,7 @@ Interfaces have two main attributes: Audience and Stability
 
 Audience denotes the potential consumers of the interface. While many interfaces
 are internal/private to the implementation, other are public/external interfaces
-are meant for wider consumption by applications and/or clients. For example, in
+that are meant for wider consumption by applications and/or clients. For example, in
 posix, libc is an external or public interface, while large parts of the kernel
 are internal or private interfaces. Also, some interfaces are targeted towards
 other specific subsystems.
@@ -52,7 +52,7 @@ other specific subsystems.
 Identifying the audience of an interface helps define the impact of breaking
 it. For instance, it might be okay to break the compatibility of an interface
 whose audience is a small number of specific subsystems. On the other hand, it
-is probably not okay to break a protocol interfaces that millions of Internet
+is probably not okay to break a protocol interface that millions of Internet
 users depend on.
 
 Hadoop uses the following kinds of audience in order of increasing/wider visibility:
@@ -75,7 +75,7 @@ referred to as project-private).
 
 The interface is used by a specified set of projects or systems (typically
 closely related projects). Other projects or systems should not use the
-interface. Changes to the interface will be communicated/ negotiated with the
+interface. Changes to the interface will be communicated/negotiated with the
 specified projects. For example, in the Hadoop project, some interfaces are
 LimitedPrivate{HDFS, MapReduce} in that they are private to the HDFS and
 MapReduce projects.
@@ -92,16 +92,16 @@ the interface are allowed. Hadoop APIs have the following levels of stability.
 #### Stable
 
 Can evolve while retaining compatibility for minor release boundaries; in other
-words, incompatible changes to APIs marked Stable are allowed only at major
+words, incompatible changes to APIs marked as Stable are allowed only at major
 releases (i.e. at m.0).
 
 #### Evolving
 
-Evolving, but incompatible changes are allowed at minor release (i.e. m .x)
+Evolving, but incompatible changes are allowed at minor releases (i.e. m .x)
 
 #### Unstable
 
-Incompatible changes to Unstable APIs are allowed any time. This usually makes
+Incompatible changes to Unstable APIs are allowed at any time. This usually makes
 sense for only private interfaces.
 
 However one may call this out for a supposedly public interface to highlight
@@ -109,11 +109,11 @@ that it should not be used as an interface; for public interfaces, labeling it
 as Not-an-interface is probably more appropriate than "Unstable".
 
 Examples of publicly visible interfaces that are unstable
-(i.e. not-an-interface): GUI, CLIs whose output format will change
+(i.e. not-an-interface): GUI, CLIs whose output format will change.
 
 #### Deprecated
 
-APIs that could potentially removed in the future and should not be used.
+APIs that could potentially be removed in the future and should not be used.
 
 How are the Classifications Recorded?
 -------------------------------------
@@ -155,13 +155,13 @@ FAQ
         * e.g. In HDFS, NN-DN protocol is private but stable and can help
           implement rolling upgrades. It communicates that this interface should
           not be changed in incompatible ways even though it is private.
-        * e.g. In HDFS, FSImage stability can help provide more flexible roll backs.
+        * e.g. In HDFS, FSImage stability provides more flexible rollback.
 
 * What is the harm in applications using a private interface that is stable? How
   is it different than a public stable interface?
     * While a private interface marked as stable is targeted to change only at
       major releases, it may break at other times if the providers of that
-      interface are willing to changes the internal users of that
+      interface are willing to change the internal users of that
       interface. Further, a public stable interface is less likely to break even
       at major releases (even though it is allowed to break compatibility)
       because the impact of the change is larger. If you use a private interface
@@ -182,11 +182,11 @@ FAQ
       away with private then do so; if the interface is really for general use
       for all applications then do so. But remember that making an interface
       public has huge responsibility. Sometimes Limited-private is just right.
-    * A good example of a limited-private interface is BlockLocations, This is
+    * A good example of a limited-private interface is BlockLocations, This is a
       fairly low-level interface that we are willing to expose to MR and perhaps
       HBase. We are likely to change it down the road and at that time we will
-      have get a coordinated effort with the MR team to release matching
-      releases. While MR and HDFS are always released in sync today, they may
+      coordinate release effort with the MR team.
+      While MR and HDFS are always released in sync today, they may
       change down the road.
     * If you have a limited-private interface with many projects listed then you
       are fooling yourself. It is practically public.
@@ -207,7 +207,7 @@ FAQ
       break it at minor releases.
     * One example of a public interface that is unstable is where one is
       providing an implementation of a standards-body based interface that is
-      still under development. For example, many companies, in an attampt to be
+      still under development. For example, many companies, in an attempt to be
       first to market, have provided implementations of a new NFS protocol even
       when the protocol was not fully completed by IETF. The implementor cannot
       evolve the interface in a fashion that causes least distruption because

http://git-wip-us.apache.org/repos/asf/hadoop/blob/987ee511/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
index 2c9dd5d..d927b8b 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
@@ -35,7 +35,7 @@ of the client.
 
 **Implementation Note**: the static `FileSystem get(URI uri, Configuration conf) ` method MAY return
 a pre-existing instance of a filesystem client class&mdash;a class that may also be in use in other threads.
-The implementations of `FileSystem` which ship with Apache Hadoop
+The implementations of `FileSystem` shipped with Apache Hadoop
 *do not make any attempt to synchronize access to the working directory field*.
 
 ## Invariants
@@ -214,7 +214,6 @@ response, then, if a listing `listStatus("/d")` takes place concurrently with th
 
 	[a, part-0000001, ... , part-9999999]
 	[part-0000001, ... , part-9999999, z]
-
 	[a, part-0000001, ... , part-9999999, z]
 	[part-0000001, ... , part-9999999]
 
@@ -282,7 +281,7 @@ value is an instance of the `LocatedFileStatus` subclass of a `FileStatus`,
 and that rather than return an entire list, an iterator is returned.
 
 This is actually a `protected` method, directly invoked by
-`listLocatedStatus(Path path):`. Calls to it may be delegated through
+`listLocatedStatus(Path path)`. Calls to it may be delegated through
 layered filesystems, such as `FilterFileSystem`, so its implementation MUST
 be considered mandatory, even if `listLocatedStatus(Path path)` has been
 implemented in a different manner. There are open JIRAs proposing
@@ -442,7 +441,7 @@ the convention is generally retained.
 
 ###  `long getDefaultBlockSize()`
 
-Get the "default" block size for a filesystem. This often used during
+Get the "default" block size for a filesystem. This is often used during
 split calculations to divide work optimally across a set of worker processes.
 
 #### Preconditions
@@ -604,7 +603,7 @@ This MAY be a bug, as it allows >1 client to create a file with `overwrite==fals
  and potentially confuse file/directory logic
 
 * The Local FileSystem raises a `FileNotFoundException` when trying to create a file over
-a directory, hence it is is listed as an exception that MAY be raised when
+a directory, hence it is listed as an exception that MAY be raised when
 this precondition fails.
 
 * Not covered: symlinks. The resolved path of the symlink is used as the final path argument to the `create()` operation
@@ -898,7 +897,7 @@ Renaming a file where the destination is a directory moves the file as a child
 ##### Renaming a directory onto a directory
 
 If `src` is a directory then all its children will then exist under `dest`, while the path
-`src` and its descendants will no longer not exist. The names of the paths under
+`src` and its descendants will no longer exist. The names of the paths under
 `dest` will match those under `src`, as will the contents:
 
     if isDir(FS, src) isDir(FS, dest) and src != dest :
@@ -928,7 +927,7 @@ The outcome is no change to FileSystem state, with a return value of false.
 *Local Filesystem, S3N*
 
 The outcome is as a normal rename, with the additional (implicit) feature
-that the parent directores of the destination also exist
+that the parent directories of the destination also exist.
 
     exists(FS', parent(dest))
 
@@ -1018,9 +1017,9 @@ HDFS: All source files except the final one MUST be a complete block:
 
 
 HDFS's restrictions may be an implementation detail of how it implements
-`concat` -by changing the inode references to join them together in
+`concat` by changing the inode references to join them together in
 a sequence. As no other filesystem in the Hadoop core codebase
-implements this method, there is no way to distinguish implementation detail.
+implements this method, there is no way to distinguish implementation detail
 from specification.
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/987ee511/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md
index adecd57..32eeb5b 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md
@@ -29,7 +29,7 @@ with extensions that add key assumptions to the system.
 1. The stream being read references a finite array of bytes.
 1. The length of the data does not change during the read process.
 1. The contents of the data does not change during the process.
-1. The source file remains present during the read process
+1. The source file remains present during the read process.
 1. Callers may use `Seekable.seek()` to offsets within the array of bytes, with future
 reads starting at this offset.
 1. The cost of forward and backward seeks is low.
@@ -104,7 +104,7 @@ Return the current position. The outcome when a stream is closed is undefined.
 
 Return the data at the current position.
 
-1. Implementations should fail when a stream is closed
+1. Implementations should fail when a stream is closed.
 1. There is no limit on how long `read()` may take to complete.
 
 #### Preconditions
@@ -124,7 +124,7 @@ Return the data at the current position.
 
 Read `length` bytes of data into the destination buffer, starting at offset
 `offset`. The source of the data is the current position of the stream,
-as implicitly set in `pos`
+as implicitly set in `pos`.
 
 #### Preconditions
 
@@ -166,7 +166,7 @@ the stream.
 
 That is, rather than `l` being simply defined as `min(length, len(data)-length)`,
 it strictly is an integer in the range `1..min(length, len(data)-length)`.
-While the caller may expect for as much as the buffer as possible to be filled
+While the caller may expect as much of the buffer as possible to be filled
 in, it is within the specification for an implementation to always return
 a smaller number, perhaps only ever 1 byte.
 
@@ -192,7 +192,7 @@ Some filesystems do not perform this check, relying on the `read()` contract
 to reject reads on a closed stream (e.g. `RawLocalFileSystem`).
 
 A `seek(0)` MUST always succeed, as  the seek position must be
-positive and less than the length of the Stream's:
+positive and less than the length of the Stream:
 
     s > 0 and ((s==0) or ((s < len(data)))) else raise [EOFException, IOException]
 
@@ -222,7 +222,7 @@ data at offset `offset`.
 
 #### Preconditions
 
-Not all subclasses implement the operation operation, and instead
+Not all subclasses implement this operation, and instead
 either raise an exception or return `False`.
 
     supported(FSDIS, Seekable.seekToNewSource) else raise [UnsupportedOperationException, IOException]
@@ -250,7 +250,7 @@ If the operation is supported and there is a new location for the data:
 
 The new data is the original data (or an updated version of it, as covered
 in the Consistency section below), but the block containing the data at `offset`
-sourced from a different replica.
+is sourced from a different replica.
 
 If there is no other copy, `FSDIS` is  not updated; the response indicates this:
 
@@ -258,7 +258,7 @@ If there is no other copy, `FSDIS` is  not updated; the response indicates this:
 
 Outside of test methods, the primary use of this method is in the {{FSInputChecker}}
 class, which can react to a checksum error in a read by attempting to source
-the data elsewhere. It a new source can be found it attempts to reread and
+the data elsewhere. If a new source can be found it attempts to reread and
 recheck that portion of the file.
 
 ## <a name="PositionedReadable"></a> interface `PositionedReadable`

http://git-wip-us.apache.org/repos/asf/hadoop/blob/987ee511/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/introduction.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/introduction.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/introduction.md
index 22b39d4..22da54c 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/introduction.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/introduction.md
@@ -141,7 +141,7 @@ The failure modes when a user lacks security permissions are not specified.
 
 ### Networking Assumptions
 
-This document assumes this all network operations succeed. All statements
+This document assumes that all network operations succeed. All statements
 can be assumed to be qualified as *"assuming the operation does not fail due
 to a network availability problem"*
 
@@ -303,7 +303,7 @@ does not hold on blob stores]
 1. Directory list operations are fast for directories with few entries, but may
 incur a cost that is `O(entries)`. Hadoop 2 added iterative listing to
 handle the challenge of listing directories with millions of entries without
-buffering -at the cost of consistency.
+buffering at the cost of consistency.
 
 1. A `close()` of an `OutputStream` is fast, irrespective of whether or not
 the file operation has succeeded or not.
@@ -317,8 +317,8 @@ This specification refers to *Object Stores* in places, often using the
 term *Blobstore*. Hadoop does provide FileSystem client classes for some of these
 even though they violate many of the requirements. This is why, although
 Hadoop can read and write data in an object store, the two which Hadoop ships
-with direct support for &mdash;Amazon S3 and OpenStack Swift&mdash cannot
-be used as direct replacement for HDFS.
+with direct support for &mdash; Amazon S3 and OpenStack Swift &mdash; cannot
+be used as direct replacements for HDFS.
 
 *What is an Object Store?*
 
@@ -358,10 +358,10 @@ are current with respect to the files within that directory.
 as are `delete()` operations. Object store FileSystem clients implement these
 as operations on the individual objects whose names match the directory prefix.
 As a result, the changes take place a file at a time, and are not atomic. If
-an operation fails part way through the process, the the state of the object store
+an operation fails part way through the process, then the state of the object store
 reflects the partially completed operation.  Note also that client code
 assumes that these operations are `O(1)` &mdash;in an object store they are
-more likely to be be `O(child-entries)`.
+more likely to be `O(child-entries)`.
 
 1. **Durability**. Hadoop assumes that `OutputStream` implementations write data
 to their (persistent) storage on a `flush()` operation. Object store implementations

http://git-wip-us.apache.org/repos/asf/hadoop/blob/987ee511/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/model.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/model.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/model.md
index e04a640..e121c92 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/model.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/model.md
@@ -18,7 +18,7 @@
 
 ## Paths and Path Elements
 
-A Path is a list of Path elements which represents a path to a file, directory of symbolic link
+A Path is a list of Path elements which represents a path to a file, directory or symbolic link
 
 Path elements are non-empty strings. The exact set of valid strings MAY
 be specific to a particular FileSystem implementation.
@@ -179,7 +179,7 @@ path begins with the path P -that is their parent is P or an ancestor is P
 
 ### File references
 
-A path MAY refer to a file; that it it has data in the filesystem; its path is a key in the data dictionary
+A path MAY refer to a file that has data in the filesystem; its path is a key in the data dictionary
 
     def isFile(FS, p) =  p in FS.Files
 
@@ -206,7 +206,8 @@ process working with the filesystem:
 
 The function `getHomeDirectory` returns the home directory for the Filesystem and the current user account.
 For some FileSystems, the path is `["/","users", System.getProperty("user-name")]`. However,
-for HDFS,
+for HDFS, the username is derived from the credentials used to authenticate the client with HDFS.
+This may differ from the local user account name.
 
 
 ### Exclusivity

http://git-wip-us.apache.org/repos/asf/hadoop/blob/987ee511/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/notation.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/notation.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/notation.md
index aa310f8..472bb5d 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/notation.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/notation.md
@@ -130,7 +130,7 @@ Strings are lists of characters represented in double quotes. e.g. `"abc"`
 
 All system state declarations are immutable.
 
-The suffix "'" (single quote) is used as the convention to indicate the state of the system after a operation:
+The suffix "'" (single quote) is used as the convention to indicate the state of the system after an operation:
 
     L' = L + ['d','e']
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/987ee511/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/testing.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/testing.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/testing.md
index 6619332..6823e0c 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/testing.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/testing.md
@@ -28,7 +28,7 @@ remote server providing the filesystem.
 
 These filesystem bindings must be defined in an XML configuration file, usually
 `hadoop-common-project/hadoop-common/src/test/resources/contract-test-options.xml`.
-This file is excluded should not be checked in.
+This file is excluded and should not be checked in.
 
 ### ftp://
 
@@ -122,7 +122,7 @@ new contract class, then creating a new non-abstract test class for every test
 suite that you wish to test.
 
 1. Do not try and add these tests into Hadoop itself. They won't be added to
-the soutce tree. The tests must live with your own filesystem source.
+the source tree. The tests must live with your own filesystem source.
 1. Create a package in your own test source tree (usually) under `contract`,
 for the files and tests.
 1. Subclass `AbstractFSContract` for your own contract implementation.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/987ee511/hadoop-tools/hadoop-archives/src/site/markdown/HadoopArchives.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-archives/src/site/markdown/HadoopArchives.md.vm b/hadoop-tools/hadoop-archives/src/site/markdown/HadoopArchives.md.vm
index 8bbb1ea..9d83ed9 100644
--- a/hadoop-tools/hadoop-archives/src/site/markdown/HadoopArchives.md.vm
+++ b/hadoop-tools/hadoop-archives/src/site/markdown/HadoopArchives.md.vm
@@ -157,6 +157,6 @@ Hadoop Archives and MapReduce
   Using Hadoop Archives in MapReduce is as easy as specifying a different input
   filesystem than the default file system. If you have a hadoop archive stored
   in HDFS in /user/zoo/foo.har then for using this archive for MapReduce input,
-  all you need to specify the input directory as har:///user/zoo/foo.har. Since
+  all you need is to specify the input directory as har:///user/zoo/foo.har. Since
   Hadoop Archives is exposed as a file system MapReduce will be able to use all
   the logical input files in Hadoop Archives as input.


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


[12/50] hadoop git commit: HDFS-10789. Route webhdfs through the RPC call queue. Contributed by Daryn Sharp and Rushabh S Shah.

Posted by um...@apache.org.
HDFS-10789. Route webhdfs through the RPC call queue. Contributed by Daryn Sharp and Rushabh S Shah.


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

Branch: refs/heads/HDFS-10285
Commit: 85cd06f6636f295ad1f3bf2a90063f4714c9cca7
Parents: 6476934
Author: Kihwal Lee <ki...@apache.org>
Authored: Wed Oct 12 15:11:42 2016 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Wed Oct 12 15:11:42 2016 -0500

----------------------------------------------------------------------
 .../org/apache/hadoop/ipc/ExternalCall.java     |   9 +-
 .../java/org/apache/hadoop/ipc/TestRPC.java     |   6 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   3 +
 .../hdfs/server/namenode/FSNamesystem.java      |  15 +-
 .../hadoop/hdfs/server/namenode/NameNode.java   |  12 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |   6 +-
 .../web/resources/NamenodeWebHdfsMethods.java   | 150 +++++++++++--------
 .../src/main/resources/hdfs-default.xml         |   7 +
 .../server/namenode/TestNamenodeRetryCache.java |  25 +++-
 .../web/resources/TestWebHdfsDataLocality.java  |  25 +++-
 10 files changed, 160 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/85cd06f6/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ExternalCall.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ExternalCall.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ExternalCall.java
index 9b4cbcf..5566136 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ExternalCall.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ExternalCall.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.ipc;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.hadoop.ipc.Server.Call;
@@ -37,14 +38,10 @@ public abstract class ExternalCall<T> extends Call {
 
   public abstract UserGroupInformation getRemoteUser();
 
-  public final T get() throws IOException, InterruptedException {
+  public final T get() throws InterruptedException, ExecutionException {
     waitForCompletion();
     if (error != null) {
-      if (error instanceof IOException) {
-        throw (IOException)error;
-      } else {
-        throw new IOException(error);
-      }
+      throw new ExecutionException(error);
     }
     return result;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85cd06f6/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
index 92d9183..72b603a 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
@@ -72,6 +72,7 @@ import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
@@ -989,8 +990,9 @@ public class TestRPC extends TestRpcBase {
       try {
         exceptionCall.get();
         fail("didn't throw");
-      } catch (IOException ioe) {
-        assertEquals(expectedIOE.getMessage(), ioe.getMessage());
+      } catch (ExecutionException ee) {
+        assertTrue((ee.getCause()) instanceof IOException);
+        assertEquals(expectedIOE.getMessage(), ee.getCause().getMessage());
       }
     } finally {
       server.stop();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85cd06f6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 18209ae..10c0ad6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -70,6 +70,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.webhdfs.ugi.expire.after.access";
   public static final int     DFS_WEBHDFS_UGI_EXPIRE_AFTER_ACCESS_DEFAULT =
       10*60*1000; //10 minutes
+  public static final String DFS_WEBHDFS_USE_IPC_CALLQ =
+      "dfs.webhdfs.use.ipc.callq";
+  public static final boolean DFS_WEBHDFS_USE_IPC_CALLQ_DEFAULT = true;
 
   // HA related configuration
   public static final String  DFS_DATANODE_RESTART_REPLICA_EXPIRY_KEY = "dfs.datanode.restart.replica.expiration";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85cd06f6/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 b9b02ef..8c59186 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
@@ -242,7 +242,6 @@ import org.apache.hadoop.hdfs.server.namenode.top.TopAuditLogger;
 import org.apache.hadoop.hdfs.server.namenode.top.TopConf;
 import org.apache.hadoop.hdfs.server.namenode.top.metrics.TopMetrics;
 import org.apache.hadoop.hdfs.server.namenode.top.window.RollingWindowManager;
-import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -338,7 +337,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   private void logAuditEvent(boolean succeeded, String cmd, String src,
       String dst, HdfsFileStatus stat) throws IOException {
     if (isAuditEnabled() && isExternalInvocation()) {
-      logAuditEvent(succeeded, getRemoteUser(), getRemoteIp(),
+      logAuditEvent(succeeded, Server.getRemoteUser(), Server.getRemoteIp(),
                     cmd, src, dst, stat);
     }
   }
@@ -5262,17 +5261,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * RPC call context even if the client exits.
    */
   boolean isExternalInvocation() {
-    return Server.isRpcInvocation() || NamenodeWebHdfsMethods.isWebHdfsInvocation();
+    return Server.isRpcInvocation();
   }
 
-  private static InetAddress getRemoteIp() {
-    InetAddress ip = Server.getRemoteIp();
-    if (ip != null) {
-      return ip;
-    }
-    return NamenodeWebHdfsMethods.getRemoteIp();
-  }
-  
   // optimize ugi lookup for RPC operations to avoid a trip through
   // UGI.getCurrentUser which is synch'ed
   private static UserGroupInformation getRemoteUser() throws IOException {
@@ -6918,7 +6909,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           sb.append(trackingId);
         }
         sb.append("\t").append("proto=");
-        sb.append(NamenodeWebHdfsMethods.isWebHdfsInvocation() ? "webhdfs" : "rpc");
+        sb.append(Server.getProtocol());
         if (isCallerContextEnabled &&
             callerContext != null &&
             callerContext.isContextValid()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85cd06f6/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 ae7a937..afedbb9 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
@@ -64,7 +64,9 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.ipc.ExternalCall;
 import org.apache.hadoop.ipc.RefreshCallQueueProtocol;
+import org.apache.hadoop.ipc.RetriableException;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@@ -407,7 +409,15 @@ public class NameNode extends ReconfigurableBase implements
   public NamenodeProtocols getRpcServer() {
     return rpcServer;
   }
-  
+
+  public void queueExternalCall(ExternalCall<?> extCall)
+      throws IOException, InterruptedException {
+    if (rpcServer == null) {
+      throw new RetriableException("Namenode is in startup mode");
+    }
+    rpcServer.getClientRpcServer().queueCall(extCall);
+  }
+
   public static void initMetrics(Configuration conf, NamenodeRole role) {
     metrics = NameNodeMetrics.create(conf, role);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85cd06f6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 57f7cb1..a97a307 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -139,7 +139,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
-import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
@@ -1686,10 +1685,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   private static String getClientMachine() {
-    String clientMachine = NamenodeWebHdfsMethods.getRemoteAddress();
-    if (clientMachine == null) { //not a web client
-      clientMachine = Server.getRemoteAddress();
-    }
+    String clientMachine = Server.getRemoteAddress();
     if (clientMachine == null) { //not a RPC client
       clientMachine = "";
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85cd06f6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
index 3ab0c67..4887e35 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
@@ -25,10 +25,13 @@ import java.io.PrintWriter;
 import java.net.InetAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.net.UnknownHostException;
+import java.security.Principal;
 import java.security.PrivilegedExceptionAction;
 import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.List;
+import java.util.concurrent.ExecutionException;
 
 import javax.servlet.ServletContext;
 import javax.servlet.http.HttpServletRequest;
@@ -60,6 +63,7 @@ import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsCreateModes;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -81,8 +85,8 @@ import org.apache.hadoop.hdfs.web.WebHdfsConstants;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.resources.*;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ipc.ExternalCall;
 import org.apache.hadoop.ipc.RetriableException;
-import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.security.Credentials;
@@ -103,39 +107,39 @@ public class NamenodeWebHdfsMethods {
   public static final Log LOG = LogFactory.getLog(NamenodeWebHdfsMethods.class);
 
   private static final UriFsPathParam ROOT = new UriFsPathParam("");
-  
-  private static final ThreadLocal<String> REMOTE_ADDRESS = new ThreadLocal<String>(); 
-
-  /** @return the remote client address. */
-  public static String getRemoteAddress() {
-    return REMOTE_ADDRESS.get();
-  }
-
-  public static InetAddress getRemoteIp() {
-    try {
-      return InetAddress.getByName(getRemoteAddress());
-    } catch (Exception e) {
-      return null;
-    }
-  }
 
-  /**
-   * Returns true if a WebHdfs request is in progress.  Akin to
-   * {@link Server#isRpcInvocation()}.
-   */
-  public static boolean isWebHdfsInvocation() {
-    return getRemoteAddress() != null;
-  }
+  private volatile Boolean useIpcCallq;
+  private String scheme;
+  private Principal userPrincipal;
+  private String remoteAddr;
 
   private @Context ServletContext context;
-  private @Context HttpServletRequest request;
   private @Context HttpServletResponse response;
 
+  public NamenodeWebHdfsMethods(@Context HttpServletRequest request) {
+    // the request object is a proxy to thread-locals so we have to extract
+    // what we want from it since the external call will be processed in a
+    // different thread.
+    scheme = request.getScheme();
+    userPrincipal = request.getUserPrincipal();
+    // get the remote address, if coming in via a trusted proxy server then
+    // the address with be that of the proxied client
+    remoteAddr = JspHelper.getRemoteAddr(request);
+  }
+
   private void init(final UserGroupInformation ugi,
       final DelegationParam delegation,
       final UserParam username, final DoAsParam doAsUser,
       final UriFsPathParam path, final HttpOpParam<?> op,
       final Param<?, ?>... parameters) {
+    if (useIpcCallq == null) {
+      Configuration conf =
+          (Configuration)context.getAttribute(JspHelper.CURRENT_CONF);
+      useIpcCallq = conf.getBoolean(
+          DFSConfigKeys.DFS_WEBHDFS_USE_IPC_CALLQ,
+          DFSConfigKeys.DFS_WEBHDFS_USE_IPC_CALLQ_DEFAULT);
+    }
+
     if (LOG.isTraceEnabled()) {
       LOG.trace("HTTP " + op.getValue().getType() + ": " + op + ", " + path
           + ", ugi=" + ugi + ", " + username + ", " + doAsUser
@@ -144,16 +148,8 @@ public class NamenodeWebHdfsMethods {
 
     //clear content type
     response.setContentType(null);
-    
-    // set the remote address, if coming in via a trust proxy server then
-    // the address with be that of the proxied client
-    REMOTE_ADDRESS.set(JspHelper.getRemoteAddr(request));
   }
 
-  private void reset() {
-    REMOTE_ADDRESS.set(null);
-  }
-  
   private static NamenodeProtocols getRPCServer(NameNode namenode)
       throws IOException {
      final NamenodeProtocols np = namenode.getRpcServer();
@@ -162,11 +158,63 @@ public class NamenodeWebHdfsMethods {
      }
      return np;
   }
-  
+
+  private <T> T doAs(final UserGroupInformation ugi,
+      final PrivilegedExceptionAction<T> action)
+          throws IOException, InterruptedException {
+    return useIpcCallq ? doAsExternalCall(ugi, action) : ugi.doAs(action);
+  }
+
+  private <T> T doAsExternalCall(final UserGroupInformation ugi,
+      final PrivilegedExceptionAction<T> action)
+          throws IOException, InterruptedException {
+    // set the remote address, if coming in via a trust proxy server then
+    // the address with be that of the proxied client
+    ExternalCall<T> call = new ExternalCall<T>(action){
+      @Override
+      public UserGroupInformation getRemoteUser() {
+        return ugi;
+      }
+      @Override
+      public String getProtocol() {
+        return "webhdfs";
+      }
+      @Override
+      public String getHostAddress() {
+        return remoteAddr;
+      }
+      @Override
+      public InetAddress getHostInetAddress() {
+        try {
+          return InetAddress.getByName(getHostAddress());
+        } catch (UnknownHostException e) {
+          return null;
+        }
+      }
+    };
+    final NameNode namenode = (NameNode)context.getAttribute("name.node");
+    namenode.queueExternalCall(call);
+    T result = null;
+    try {
+      result = call.get();
+    } catch (ExecutionException ee) {
+      Throwable t = ee.getCause();
+      if (t instanceof RuntimeException) {
+        throw (RuntimeException)t;
+      } else if (t instanceof IOException) {
+        throw (IOException)t;
+      } else {
+        throw new IOException(t);
+      }
+    }
+    return result;
+  }
+
   @VisibleForTesting
   static DatanodeInfo chooseDatanode(final NameNode namenode,
       final String path, final HttpOpParam.Op op, final long openOffset,
-      final long blocksize, final String excludeDatanodes) throws IOException {
+      final long blocksize, final String excludeDatanodes,
+      final String remoteAddr) throws IOException {
     FSNamesystem fsn = namenode.getNamesystem();
     if (fsn == null) {
       throw new IOException("Namesystem has not been intialized yet.");
@@ -190,7 +238,7 @@ public class NamenodeWebHdfsMethods {
     if (op == PutOpParam.Op.CREATE) {
       //choose a datanode near to client 
       final DatanodeDescriptor clientNode = bm.getDatanodeManager(
-          ).getDatanodeByHost(getRemoteAddress());
+          ).getDatanodeByHost(remoteAddr);
       if (clientNode != null) {
         final DatanodeStorageInfo[] storages = bm.chooseTarget4WebHDFS(
             path, clientNode, excludes, blocksize);
@@ -253,7 +301,8 @@ public class NamenodeWebHdfsMethods {
       return null;
     }
     final Token<? extends TokenIdentifier> t = c.getAllTokens().iterator().next();
-    Text kind = request.getScheme().equals("http") ? WebHdfsConstants.WEBHDFS_TOKEN_KIND
+    Text kind = scheme.equals("http")
+        ? WebHdfsConstants.WEBHDFS_TOKEN_KIND
         : WebHdfsConstants.SWEBHDFS_TOKEN_KIND;
     t.setKind(kind);
     return t;
@@ -267,7 +316,7 @@ public class NamenodeWebHdfsMethods {
       final Param<?, ?>... parameters) throws URISyntaxException, IOException {
     final DatanodeInfo dn;
     dn = chooseDatanode(namenode, path, op, openOffset, blocksize,
-        excludeDatanodes);
+        excludeDatanodes, remoteAddr);
     if (dn == null) {
       throw new IOException("Failed to find datanode, suggest to check cluster"
           + " health. excludeDatanodes=" + excludeDatanodes);
@@ -283,7 +332,7 @@ public class NamenodeWebHdfsMethods {
     } else {
       //generate a token
       final Token<? extends TokenIdentifier> t = generateDelegationToken(
-          namenode, ugi, request.getUserPrincipal().getName());
+          namenode, ugi, userPrincipal.getName());
       delegationQuery = "&" + new DelegationParam(t.encodeToUrlString());
     }
     final String query = op.toQueryString() + delegationQuery
@@ -291,7 +340,6 @@ public class NamenodeWebHdfsMethods {
         + Param.toSortedString("&", parameters);
     final String uripath = WebHdfsFileSystem.PATH_PREFIX + path;
 
-    final String scheme = request.getScheme();
     int port = "http".equals(scheme) ? dn.getInfoPort() : dn
         .getInfoSecurePort();
     final URI uri = new URI(scheme, null, dn.getHostName(), port, uripath,
@@ -446,10 +494,9 @@ public class NamenodeWebHdfsMethods {
         xattrSetFlag, snapshotName, oldSnapshotName, excludeDatanodes,
         createFlagParam, noredirect);
 
-    return ugi.doAs(new PrivilegedExceptionAction<Response>() {
+    return doAs(ugi, new PrivilegedExceptionAction<Response>() {
       @Override
       public Response run() throws IOException, URISyntaxException {
-        try {
           return put(ugi, delegation, username, doAsUser,
               path.getAbsolutePath(), op, destination, owner, group,
               permission, unmaskedPermission, overwrite, bufferSize,
@@ -458,9 +505,6 @@ public class NamenodeWebHdfsMethods {
               aclPermission, xattrName, xattrValue, xattrSetFlag,
               snapshotName, oldSnapshotName, excludeDatanodes,
               createFlagParam, noredirect);
-        } finally {
-          reset();
-        }
       }
     });
   }
@@ -703,16 +747,12 @@ public class NamenodeWebHdfsMethods {
     init(ugi, delegation, username, doAsUser, path, op, concatSrcs, bufferSize,
         excludeDatanodes, newLength);
 
-    return ugi.doAs(new PrivilegedExceptionAction<Response>() {
+    return doAs(ugi, new PrivilegedExceptionAction<Response>() {
       @Override
       public Response run() throws IOException, URISyntaxException {
-        try {
           return post(ugi, delegation, username, doAsUser,
               path.getAbsolutePath(), op, concatSrcs, bufferSize,
               excludeDatanodes, newLength, noredirect);
-        } finally {
-          reset();
-        }
       }
     });
   }
@@ -858,17 +898,13 @@ public class NamenodeWebHdfsMethods {
         renewer, bufferSize, xattrEncoding, excludeDatanodes, fsAction,
         tokenKind, tokenService, startAfter);
 
-    return ugi.doAs(new PrivilegedExceptionAction<Response>() {
+    return doAs(ugi, new PrivilegedExceptionAction<Response>() {
       @Override
       public Response run() throws IOException, URISyntaxException {
-        try {
           return get(ugi, delegation, username, doAsUser,
               path.getAbsolutePath(), op, offset, length, renewer, bufferSize,
               xattrNames, xattrEncoding, excludeDatanodes, fsAction, tokenKind,
               tokenService, noredirect, startAfter);
-        } finally {
-          reset();
-        }
       }
     });
   }
@@ -1138,15 +1174,11 @@ public class NamenodeWebHdfsMethods {
 
     init(ugi, delegation, username, doAsUser, path, op, recursive, snapshotName);
 
-    return ugi.doAs(new PrivilegedExceptionAction<Response>() {
+    return doAs(ugi, new PrivilegedExceptionAction<Response>() {
       @Override
       public Response run() throws IOException {
-        try {
           return delete(ugi, delegation, username, doAsUser,
               path.getAbsolutePath(), op, recursive, snapshotName);
-        } finally {
-          reset();
-        }
       }
     });
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85cd06f6/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index db4035d..84b51f6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -4281,4 +4281,11 @@
     </description>
   </property>
 
+  <property>
+    <name>dfs.webhdfs.use.ipc.callq</name>
+    <value>true</value>
+    <description>Enables routing of webhdfs calls through rpc
+      call queue</description>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85cd06f6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
index 26efce5..d7a2c81 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.ipc.RpcConstants;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.LightWeightCache;
 import org.junit.After;
@@ -111,19 +112,33 @@ public class TestNamenodeRetryCache {
     }
   }
   
+  static class DummyCall extends Server.Call {
+    private UserGroupInformation ugi;
+
+    DummyCall(int callId, byte[] clientId) {
+      super(callId, 1, null, null, RpcKind.RPC_PROTOCOL_BUFFER, clientId);
+      try {
+        ugi = UserGroupInformation.getCurrentUser();
+      } catch (IOException ioe) {
+      }
+    }
+    @Override
+    public UserGroupInformation getRemoteUser() {
+      return ugi;
+    }
+  }
   /** Set the current Server RPC call */
   public static void newCall() {
-    Server.Call call = new Server.Call(++callId, 1, null, null,
-        RpcKind.RPC_PROTOCOL_BUFFER, CLIENT_ID);
+    Server.Call call = new DummyCall(++callId, CLIENT_ID);
     Server.getCurCall().set(call);
   }
   
   public static void resetCall() {
-    Server.Call call = new Server.Call(RpcConstants.INVALID_CALL_ID, 1, null,
-        null, RpcKind.RPC_PROTOCOL_BUFFER, RpcConstants.DUMMY_CLIENT_ID);
+    Server.Call call = new DummyCall(RpcConstants.INVALID_CALL_ID,
+        RpcConstants.DUMMY_CLIENT_ID);
     Server.getCurCall().set(call);
   }
-  
+
   private void concatSetup(String file1, String file2) throws Exception {
     DFSTestUtil.createFile(filesystem, new Path(file1), BlockSize, (short)1, 0L);
     DFSTestUtil.createFile(filesystem, new Path(file2), BlockSize, (short)1, 0L);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85cd06f6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/web/resources/TestWebHdfsDataLocality.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/web/resources/TestWebHdfsDataLocality.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/web/resources/TestWebHdfsDataLocality.java
index 15e1c04..604bf79 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/web/resources/TestWebHdfsDataLocality.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/web/resources/TestWebHdfsDataLocality.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode.web.resources;
 import static org.mockito.Mockito.*;
 
 import java.io.IOException;
+import java.net.InetAddress;
 import java.util.Arrays;
 import java.util.List;
 
@@ -62,6 +63,9 @@ public class TestWebHdfsDataLocality {
   private static final String RACK1 = "/rack1";
   private static final String RACK2 = "/rack2";
 
+  private static final String LOCALHOST =
+      InetAddress.getLoopbackAddress().getHostName();
+
   @Rule
   public final ExpectedException exception = ExpectedException.none();
 
@@ -96,7 +100,8 @@ public class TestWebHdfsDataLocality {
 
           //The chosen datanode must be the same as the client address
           final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode(
-              namenode, f, PutOpParam.Op.CREATE, -1L, blocksize, null);
+              namenode, f, PutOpParam.Op.CREATE, -1L, blocksize, null,
+              LOCALHOST);
           Assert.assertEquals(ipAddr, chosen.getIpAddr());
         }
       }
@@ -121,19 +126,22 @@ public class TestWebHdfsDataLocality {
 
       { //test GETFILECHECKSUM
         final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode(
-            namenode, f, GetOpParam.Op.GETFILECHECKSUM, -1L, blocksize, null);
+            namenode, f, GetOpParam.Op.GETFILECHECKSUM, -1L, blocksize, null,
+            LOCALHOST);
         Assert.assertEquals(expected, chosen);
       }
   
       { //test OPEN
         final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode(
-            namenode, f, GetOpParam.Op.OPEN, 0, blocksize, null);
+            namenode, f, GetOpParam.Op.OPEN, 0, blocksize, null,
+            LOCALHOST);
         Assert.assertEquals(expected, chosen);
       }
 
       { //test APPEND
         final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode(
-            namenode, f, PostOpParam.Op.APPEND, -1L, blocksize, null);
+            namenode, f, PostOpParam.Op.APPEND, -1L, blocksize, null,
+            LOCALHOST);
         Assert.assertEquals(expected, chosen);
       }
     } finally {
@@ -189,7 +197,7 @@ public class TestWebHdfsDataLocality {
         { // test GETFILECHECKSUM
           final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode(
               namenode, f, GetOpParam.Op.GETFILECHECKSUM, -1L, blocksize,
-              sb.toString());
+              sb.toString(), LOCALHOST);
           for (int j = 0; j <= i; j++) {
             Assert.assertNotEquals(locations[j].getHostName(),
                 chosen.getHostName());
@@ -198,7 +206,8 @@ public class TestWebHdfsDataLocality {
 
         { // test OPEN
           final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode(
-              namenode, f, GetOpParam.Op.OPEN, 0, blocksize, sb.toString());
+              namenode, f, GetOpParam.Op.OPEN, 0, blocksize, sb.toString(),
+              LOCALHOST);
           for (int j = 0; j <= i; j++) {
             Assert.assertNotEquals(locations[j].getHostName(),
                 chosen.getHostName());
@@ -208,7 +217,7 @@ public class TestWebHdfsDataLocality {
         { // test APPEND
           final DatanodeInfo chosen = NamenodeWebHdfsMethods
               .chooseDatanode(namenode, f, PostOpParam.Op.APPEND, -1L,
-                  blocksize, sb.toString());
+                  blocksize, sb.toString(), LOCALHOST);
           for (int j = 0; j <= i; j++) {
             Assert.assertNotEquals(locations[j].getHostName(),
                 chosen.getHostName());
@@ -229,6 +238,6 @@ public class TestWebHdfsDataLocality {
     exception.expect(IOException.class);
     exception.expectMessage("Namesystem has not been intialized yet.");
     NamenodeWebHdfsMethods.chooseDatanode(nn, "/path", PutOpParam.Op.CREATE, 0,
-        DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT, null);
+        DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT, null, LOCALHOST);
   }
 }


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


[18/50] hadoop git commit: HADOOP-13565. KerberosAuthenticationHandler#authenticate should not rebuild SPN based on client request. Contributed by Xiaoyu Yao.

Posted by um...@apache.org.
HADOOP-13565. KerberosAuthenticationHandler#authenticate should not rebuild SPN based on client request. Contributed by Xiaoyu Yao.


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

Branch: refs/heads/HDFS-10285
Commit: 9097e2efe4c92d83c8fab88dc11be84505a6cab5
Parents: b371c56
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Thu Oct 13 10:52:13 2016 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Thu Oct 13 10:52:28 2016 -0700

----------------------------------------------------------------------
 .../authentication/server/KerberosAuthenticationHandler.java  | 7 +------
 1 file changed, 1 insertion(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9097e2ef/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java
index c6d1881..07c2a31 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java
@@ -343,8 +343,6 @@ public class KerberosAuthenticationHandler implements AuthenticationHandler {
       authorization = authorization.substring(KerberosAuthenticator.NEGOTIATE.length()).trim();
       final Base64 base64 = new Base64(0);
       final byte[] clientToken = base64.decode(authorization);
-      final String serverName = InetAddress.getByName(request.getServerName())
-                                           .getCanonicalHostName();
       try {
         token = Subject.doAs(serverSubject, new PrivilegedExceptionAction<AuthenticationToken>() {
 
@@ -354,10 +352,7 @@ public class KerberosAuthenticationHandler implements AuthenticationHandler {
             GSSContext gssContext = null;
             GSSCredential gssCreds = null;
             try {
-              gssCreds = gssManager.createCredential(
-                  gssManager.createName(
-                      KerberosUtil.getServicePrincipal("HTTP", serverName),
-                      KerberosUtil.getOidInstance("NT_GSS_KRB5_PRINCIPAL")),
+              gssCreds = gssManager.createCredential(null,
                   GSSCredential.INDEFINITE_LIFETIME,
                   new Oid[]{
                     KerberosUtil.getOidInstance("GSS_SPNEGO_MECH_OID"),


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


[05/50] hadoop git commit: HDFS-10991. Export hdfsTruncateFile symbol in libhdfs. Contributed by Surendra Singh Lilhore.

Posted by um...@apache.org.
HDFS-10991. Export hdfsTruncateFile symbol in libhdfs. Contributed by Surendra Singh Lilhore.


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

Branch: refs/heads/HDFS-10285
Commit: dacd3ec66b111be24131957c986f0c748cf9ea26
Parents: 8a09bf7
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Oct 11 15:07:14 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue Oct 11 15:07:14 2016 -0700

----------------------------------------------------------------------
 .../src/main/native/libhdfs/include/hdfs/hdfs.h                     | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dacd3ec6/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/include/hdfs/hdfs.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/include/hdfs/hdfs.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/include/hdfs/hdfs.h
index c856928..83c1c59 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/include/hdfs/hdfs.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/include/hdfs/hdfs.h
@@ -493,6 +493,7 @@ extern  "C" {
      *         complete before proceeding with further file updates.
      *         -1 on error.
      */
+    LIBHDFS_EXTERNAL
     int hdfsTruncateFile(hdfsFS fs, const char* path, tOffset newlength);
 
     /**


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


[43/50] hadoop git commit: HADOOP-13722. Code cleanup -- ViewFileSystem and InodeTree. Contributed by Manoj Govindassamy.

Posted by um...@apache.org.
HADOOP-13722. Code cleanup -- ViewFileSystem and InodeTree. Contributed by Manoj Govindassamy.


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

Branch: refs/heads/HDFS-10285
Commit: 0f4afc81009129bbee89d5b6cf22c8dda612d223
Parents: 412c4c9
Author: Andrew Wang <wa...@apache.org>
Authored: Mon Oct 17 13:15:11 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Mon Oct 17 13:15:11 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/viewfs/InodeTree.java  | 206 +++++++++----------
 .../apache/hadoop/fs/viewfs/ViewFileSystem.java |  91 ++++----
 .../hadoop/fs/viewfs/TestViewFsConfig.java      |  42 ++--
 3 files changed, 155 insertions(+), 184 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0f4afc81/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java
index 8c42cdf..a485a3b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <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.
@@ -36,47 +36,45 @@ import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 
-
 /**
  * InodeTree implements a mount-table as a tree of inodes.
  * It is used to implement ViewFs and ViewFileSystem.
  * In order to use it the caller must subclass it and implement
  * the abstract methods {@link #getTargetFileSystem(INodeDir)}, etc.
- * 
+ *
  * The mountable is initialized from the config variables as 
  * specified in {@link ViewFs}
  *
  * @param <T> is AbstractFileSystem or FileSystem
- * 
- * The three main methods are
- * {@link #InodeTreel(Configuration)} // constructor
+ *
+ * The two main methods are
  * {@link #InodeTree(Configuration, String)} // constructor
  * {@link #resolve(String, boolean)} 
  */
 
 @InterfaceAudience.Private
-@InterfaceStability.Unstable 
+@InterfaceStability.Unstable
 abstract class InodeTree<T> {
-  static enum ResultKind {isInternalDir, isExternalDir;};
+  enum ResultKind {
+    INTERNAL_DIR,
+    EXTERNAL_DIR
+  }
+
   static final Path SlashPath = new Path("/");
-  
-  final INodeDir<T> root; // the root of the mount table
-  
-  final String homedirPrefix; // the homedir config value for this mount table
-  
-  List<MountPoint<T>> mountPoints = new ArrayList<MountPoint<T>>();
-  
-  
+  private final INodeDir<T> root;     // the root of the mount table
+  private final String homedirPrefix; // the homedir for this mount table
+  private List<MountPoint<T>> mountPoints = new ArrayList<MountPoint<T>>();
+
   static class MountPoint<T> {
     String src;
     INodeLink<T> target;
+
     MountPoint(String srcPath, INodeLink<T> mountLink) {
       src = srcPath;
       target = mountLink;
     }
-
   }
-  
+
   /**
    * Breaks file path into component names.
    * @param path
@@ -84,18 +82,19 @@ abstract class InodeTree<T> {
    */
   static String[] breakIntoPathComponents(final String path) {
     return path == null ? null : path.split(Path.SEPARATOR);
-  } 
-  
+  }
+
   /**
    * Internal class for inode tree
    * @param <T>
    */
   abstract static class INode<T> {
     final String fullPath; // the full path to the root
+
     public INode(String pathToNode, UserGroupInformation aUgi) {
       fullPath = pathToNode;
     }
-  };
+  }
 
   /**
    * Internal class to represent an internal dir of the mount table
@@ -105,37 +104,28 @@ abstract class InodeTree<T> {
     final Map<String,INode<T>> children = new HashMap<String,INode<T>>();
     T InodeDirFs =  null; // file system of this internal directory of mountT
     boolean isRoot = false;
-    
+
     INodeDir(final String pathToNode, final UserGroupInformation aUgi) {
       super(pathToNode, aUgi);
     }
 
-    INode<T> resolve(final String pathComponent) throws FileNotFoundException {
-      final INode<T> result = resolveInternal(pathComponent);
-      if (result == null) {
-        throw new FileNotFoundException();
-      }
-      return result;
-    }
-    
     INode<T> resolveInternal(final String pathComponent) {
       return children.get(pathComponent);
     }
-    
+
     INodeDir<T> addDir(final String pathComponent,
-        final UserGroupInformation aUgi)
-      throws FileAlreadyExistsException {
+        final UserGroupInformation aUgi) throws FileAlreadyExistsException {
       if (children.containsKey(pathComponent)) {
         throw new FileAlreadyExistsException();
       }
-      final INodeDir<T> newDir = new INodeDir<T>(fullPath+ (isRoot ? "" : "/") + 
-          pathComponent, aUgi);
+      final INodeDir<T> newDir = new INodeDir<T>(fullPath +
+          (isRoot ? "" : "/") + pathComponent, aUgi);
       children.put(pathComponent, newDir);
       return newDir;
     }
-    
+
     void addLink(final String pathComponent, final INodeLink<T> link)
-      throws FileAlreadyExistsException {
+        throws FileAlreadyExistsException {
       if (children.containsKey(pathComponent)) {
         throw new FileAlreadyExistsException();
       }
@@ -144,14 +134,14 @@ abstract class InodeTree<T> {
   }
 
   /**
-   * In internal class to represent a mount link
+   * An internal class to represent a mount link.
    * A mount link can be single dir link or a merge dir link.
 
    * A merge dir link is  a merge (junction) of links to dirs:
-   * example : <merge of 2 dirs
+   * example : merge of 2 dirs
    *     /users -> hdfs:nn1//users
    *     /users -> hdfs:nn2//users
-   * 
+   *
    * For a merge, each target is checked to be dir when created but if target
    * is changed later it is then ignored (a dir with null entries)
    */
@@ -159,9 +149,9 @@ abstract class InodeTree<T> {
     final boolean isMergeLink; // true if MergeLink
     final URI[] targetDirLinkList;
     final T targetFileSystem;   // file system object created from the link.
-    
+
     /**
-     * Construct a mergeLink
+     * Construct a mergeLink.
      */
     INodeLink(final String pathToNode, final UserGroupInformation aUgi,
         final T targetMergeFs, final URI[] aTargetDirLinkList) {
@@ -170,9 +160,9 @@ abstract class InodeTree<T> {
       targetDirLinkList = aTargetDirLinkList;
       isMergeLink = true;
     }
-    
+
     /**
-     * Construct a simple link (i.e. not a mergeLink)
+     * Construct a simple link (i.e. not a mergeLink).
      */
     INodeLink(final String pathToNode, final UserGroupInformation aUgi,
         final T targetFs, final URI aTargetDirLink) {
@@ -182,38 +172,36 @@ abstract class InodeTree<T> {
       targetDirLinkList[0] = aTargetDirLink;
       isMergeLink = false;
     }
-    
+
     /**
-     * Get the target of the link
-     * If a merge link then it returned as "," separated URI list.
+     * Get the target of the link. If a merge link then it returned
+     * as "," separated URI list.
      */
     Path getTargetLink() {
-      // is merge link - use "," as separator between the merged URIs
-      //String result = targetDirLinkList[0].toString();
       StringBuilder result = new StringBuilder(targetDirLinkList[0].toString());
-      for (int i=1; i < targetDirLinkList.length; ++i) { 
+      // If merge link, use "," as separator between the merged URIs
+      for (int i = 1; i < targetDirLinkList.length; ++i) {
         result.append(',').append(targetDirLinkList[i].toString());
       }
       return new Path(result.toString());
     }
   }
 
-
   private void createLink(final String src, final String target,
       final boolean isLinkMerge, final UserGroupInformation aUgi)
       throws URISyntaxException, IOException,
-    FileAlreadyExistsException, UnsupportedFileSystemException {
+      FileAlreadyExistsException, UnsupportedFileSystemException {
     // Validate that src is valid absolute path
-    final Path srcPath = new Path(src); 
+    final Path srcPath = new Path(src);
     if (!srcPath.isAbsoluteAndSchemeAuthorityNull()) {
-      throw new IOException("ViewFs:Non absolute mount name in config:" + src);
+      throw new IOException("ViewFs: Non absolute mount name in config:" + src);
     }
- 
+
     final String[] srcPaths = breakIntoPathComponents(src);
     INodeDir<T> curInode = root;
     int i;
     // Ignore first initial slash, process all except last component
-    for (i = 1; i < srcPaths.length-1; i++) {
+    for (i = 1; i < srcPaths.length - 1; i++) {
       final String iPath = srcPaths[i];
       INode<T> nextInode = curInode.resolveInternal(iPath);
       if (nextInode == null) {
@@ -226,11 +214,11 @@ abstract class InodeTree<T> {
         throw new FileAlreadyExistsException("Path " + nextInode.fullPath +
             " already exists as link");
       } else {
-        assert(nextInode instanceof INodeDir);
+        assert (nextInode instanceof INodeDir);
         curInode = (INodeDir<T>) nextInode;
       }
     }
-    
+
     // Now process the last component
     // Add the link in 2 cases: does not exist or a link exists
     String iPath = srcPaths[i];// last component
@@ -241,9 +229,9 @@ abstract class InodeTree<T> {
         strB.append('/').append(srcPaths[j]);
       }
       throw new FileAlreadyExistsException("Path " + strB +
-            " already exists as dir; cannot create link here");
+          " already exists as dir; cannot create link here");
     }
-    
+
     final INodeLink<T> newLink;
     final String fullPath = curInode.fullPath + (curInode == root ? "" : "/")
         + iPath;
@@ -263,25 +251,21 @@ abstract class InodeTree<T> {
     curInode.addLink(iPath, newLink);
     mountPoints.add(new MountPoint<T>(src, newLink));
   }
-  
-  /**
-   * Below the "public" methods of InodeTree
-   */
-  
+
   /**
    * The user of this class must subclass and implement the following
    * 3 abstract methods.
-   * @throws IOException 
+   * @throws IOException
    */
   protected abstract T getTargetFileSystem(final URI uri)
-    throws UnsupportedFileSystemException, URISyntaxException, IOException;
-  
+      throws UnsupportedFileSystemException, URISyntaxException, IOException;
+
   protected abstract T getTargetFileSystem(final INodeDir<T> dir)
-    throws URISyntaxException;
-  
+      throws URISyntaxException;
+
   protected abstract T getTargetFileSystem(final URI[] mergeFsURIList)
-  throws UnsupportedFileSystemException, URISyntaxException;
-  
+      throws UnsupportedFileSystemException, URISyntaxException;
+
   /**
    * Create Inode Tree from the specified mount-table specified in Config
    * @param config - the mount table keys are prefixed with 
@@ -294,7 +278,7 @@ abstract class InodeTree<T> {
    */
   protected InodeTree(final Configuration config, final String viewName)
       throws UnsupportedFileSystemException, URISyntaxException,
-    FileAlreadyExistsException, IOException { 
+      FileAlreadyExistsException, IOException {
     String vName = viewName;
     if (vName == null) {
       vName = Constants.CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE;
@@ -303,9 +287,9 @@ abstract class InodeTree<T> {
     root = new INodeDir<T>("/", UserGroupInformation.getCurrentUser());
     root.InodeDirFs = getTargetFileSystem(root);
     root.isRoot = true;
-    
-    final String mtPrefix = Constants.CONFIG_VIEWFS_PREFIX + "." + 
-                            vName + ".";
+
+    final String mtPrefix = Constants.CONFIG_VIEWFS_PREFIX + "." +
+        vName + ".";
     final String linkPrefix = Constants.CONFIG_VIEWFS_LINK + ".";
     final String linkMergePrefix = Constants.CONFIG_VIEWFS_LINK_MERGE + ".";
     boolean gotMountTableEntry = false;
@@ -325,18 +309,17 @@ abstract class InodeTree<T> {
           // ignore - we set home dir from config
           continue;
         } else {
-          throw new IOException(
-          "ViewFs: Cannot initialize: Invalid entry in Mount table in config: "+ 
-          src);
+          throw new IOException("ViewFs: Cannot initialize: Invalid entry in " +
+              "Mount table in config: " + src);
         }
         final String target = si.getValue(); // link or merge link
-        createLink(src, target, isMergeLink, ugi); 
+        createLink(src, target, isMergeLink, ugi);
       }
     }
     if (!gotMountTableEntry) {
       throw new IOException(
           "ViewFs: Cannot initialize: Empty Mount table in config for " +
-             "viewfs://" + vName + "/");
+              "viewfs://" + vName + "/");
     }
   }
 
@@ -344,7 +327,7 @@ abstract class InodeTree<T> {
    * Resolve returns ResolveResult.
    * The caller can continue the resolution of the remainingPath
    * in the targetFileSystem.
-   * 
+   *
    * If the input pathname leads to link to another file system then
    * the targetFileSystem is the one denoted by the link (except it is
    * file system chrooted to link target.
@@ -356,7 +339,7 @@ abstract class InodeTree<T> {
     final T targetFileSystem;
     final String resolvedPath;
     final Path remainingPath;   // to resolve in the target FileSystem
-    
+
     ResolveResult(final ResultKind k, final T targetFs, final String resolveP,
         final Path remainingP) {
       kind = k;
@@ -364,31 +347,30 @@ abstract class InodeTree<T> {
       resolvedPath = resolveP;
       remainingPath = remainingP;
     }
-    
-    // isInternalDir of path resolution completed within the mount table 
+
+    // Internal dir path resolution completed within the mount table
     boolean isInternalDir() {
-      return (kind == ResultKind.isInternalDir);
+      return (kind == ResultKind.INTERNAL_DIR);
     }
   }
-  
+
   /**
    * Resolve the pathname p relative to root InodeDir
    * @param p - inout path
-   * @param resolveLastComponent 
+   * @param resolveLastComponent
    * @return ResolveResult which allows further resolution of the remaining path
    * @throws FileNotFoundException
    */
   ResolveResult<T> resolve(final String p, final boolean resolveLastComponent)
-    throws FileNotFoundException {
-    // TO DO: - more efficient to not split the path, but simply compare
-    String[] path = breakIntoPathComponents(p); 
+      throws FileNotFoundException {
+    String[] path = breakIntoPathComponents(p);
     if (path.length <= 1) { // special case for when path is "/"
-      ResolveResult<T> res = 
-        new ResolveResult<T>(ResultKind.isInternalDir, 
+      ResolveResult<T> res =
+          new ResolveResult<T>(ResultKind.INTERNAL_DIR,
               root.InodeDirFs, root.fullPath, SlashPath);
       return res;
     }
-    
+
     INodeDir<T> curInode = root;
     int i;
     // ignore first slash
@@ -396,27 +378,27 @@ abstract class InodeTree<T> {
       INode<T> nextInode = curInode.resolveInternal(path[i]);
       if (nextInode == null) {
         StringBuilder failedAt = new StringBuilder(path[0]);
-        for ( int j = 1; j <=i; ++j) {
+        for (int j = 1; j <= i; ++j) {
           failedAt.append('/').append(path[j]);
         }
-        throw (new FileNotFoundException(failedAt.toString()));      
+        throw (new FileNotFoundException(failedAt.toString()));
       }
 
       if (nextInode instanceof INodeLink) {
         final INodeLink<T> link = (INodeLink<T>) nextInode;
         final Path remainingPath;
-        if (i >= path.length-1) {
+        if (i >= path.length - 1) {
           remainingPath = SlashPath;
         } else {
-          StringBuilder remainingPathStr = new StringBuilder("/" + path[i+1]);
-          for (int j = i+2; j< path.length; ++j) {
+          StringBuilder remainingPathStr = new StringBuilder("/" + path[i + 1]);
+          for (int j = i + 2; j < path.length; ++j) {
             remainingPathStr.append('/').append(path[j]);
           }
           remainingPath = new Path(remainingPathStr.toString());
         }
-        final ResolveResult<T> res = 
-          new ResolveResult<T>(ResultKind.isExternalDir,
-              link.targetFileSystem, nextInode.fullPath, remainingPath);
+        final ResolveResult<T> res =
+            new ResolveResult<T>(ResultKind.EXTERNAL_DIR,
+                link.targetFileSystem, nextInode.fullPath, remainingPath);
         return res;
       } else if (nextInode instanceof INodeDir) {
         curInode = (INodeDir<T>) nextInode;
@@ -433,23 +415,23 @@ abstract class InodeTree<T> {
       // that follows will do a children.get(remaningPath) and will have to
       // strip-out the initial /
       StringBuilder remainingPathStr = new StringBuilder("/" + path[i]);
-      for (int j = i+1; j< path.length; ++j) {
+      for (int j = i + 1; j < path.length; ++j) {
         remainingPathStr.append('/').append(path[j]);
       }
       remainingPath = new Path(remainingPathStr.toString());
     }
-    final ResolveResult<T> res = 
-       new ResolveResult<T>(ResultKind.isInternalDir,
-           curInode.InodeDirFs, curInode.fullPath, remainingPath); 
+    final ResolveResult<T> res =
+        new ResolveResult<T>(ResultKind.INTERNAL_DIR,
+            curInode.InodeDirFs, curInode.fullPath, remainingPath);
     return res;
   }
-  
-  List<MountPoint<T>> getMountPoints() { 
+
+  List<MountPoint<T>> getMountPoints() {
     return mountPoints;
   }
-  
+
   /**
-   * 
+   *
    * @return home dir value from mount table; null if no config value
    * was found.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0f4afc81/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
index f6947ff..f2a91d1 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
@@ -87,10 +87,19 @@ public class ViewFileSystem extends FileSystem {
       final Path p) {
     return readOnlyMountTable(operation, p.toString());
   }
-  
+
   static public class MountPoint {
-    private Path src;       // the src of the mount
-    private URI[] targets; //  target of the mount; Multiple targets imply mergeMount
+    /**
+     *  The source of the mount.
+     */
+    private Path src;
+
+    /**
+     * One or more targets of the mount.
+     * Multiple targets imply MergeMount.
+     */
+    private URI[] targets;
+
     MountPoint(Path srcPath, URI[] targetURIs) {
       src = srcPath;
       targets = targetURIs;
@@ -142,19 +151,18 @@ public class ViewFileSystem extends FileSystem {
 
   /**
    * Return the protocol scheme for the FileSystem.
-   * <p/>
    *
    * @return <code>viewfs</code>
    */
   @Override
   public String getScheme() {
-    return "viewfs";
+    return FsConstants.VIEWFS_SCHEME;
   }
 
   /**
    * Called after a new FileSystem instance is constructed.
    * @param theUri a uri whose authority section names the host, port, etc. for
-   *          this FileSystem
+   *        this FileSystem
    * @param conf the configuration
    */
   @Override
@@ -197,8 +205,7 @@ public class ViewFileSystem extends FileSystem {
     }
 
   }
-  
-  
+
   /**
    * Convenience Constructor for apps to call directly
    * @param theUri which must be that of ViewFileSystem
@@ -206,7 +213,7 @@ public class ViewFileSystem extends FileSystem {
    * @throws IOException
    */
   ViewFileSystem(final URI theUri, final Configuration conf)
-    throws IOException {
+      throws IOException {
     this();
     initialize(theUri, conf);
   }
@@ -226,8 +233,7 @@ public class ViewFileSystem extends FileSystem {
   }
   
   @Override
-  public Path resolvePath(final Path f)
-      throws IOException {
+  public Path resolvePath(final Path f) throws IOException {
     final InodeTree.ResolveResult<FileSystem> res;
       res = fsState.resolve(getUriPath(f), true);
     if (res.isInternalDir()) {
@@ -271,8 +277,8 @@ public class ViewFileSystem extends FileSystem {
   
   @Override
   public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
-      EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize,
-      Progressable progress) throws IOException {
+      EnumSet<CreateFlag> flags, int bufferSize, short replication,
+      long blockSize, Progressable progress) throws IOException {
     InodeTree.ResolveResult<FileSystem> res;
     try {
       res = fsState.resolve(getUriPath(f), false);
@@ -280,8 +286,8 @@ public class ViewFileSystem extends FileSystem {
         throw readOnlyMountTable("create", f);
     }
     assert(res.remainingPath != null);
-    return res.targetFileSystem.createNonRecursive(res.remainingPath, permission,
-        flags, bufferSize, replication, blockSize, progress);
+    return res.targetFileSystem.createNonRecursive(res.remainingPath,
+        permission, flags, bufferSize, replication, blockSize, progress);
   }
   
   @Override
@@ -302,10 +308,9 @@ public class ViewFileSystem extends FileSystem {
   
   @Override
   public boolean delete(final Path f, final boolean recursive)
-      throws AccessControlException, FileNotFoundException,
-      IOException {
+      throws AccessControlException, FileNotFoundException, IOException {
     InodeTree.ResolveResult<FileSystem> res = 
-      fsState.resolve(getUriPath(f), true);
+        fsState.resolve(getUriPath(f), true);
     // If internal dir or target is a mount link (ie remainingPath is Slash)
     if (res.isInternalDir() || res.remainingPath == InodeTree.SlashPath) {
       throw readOnlyMountTable("delete", f);
@@ -316,9 +321,8 @@ public class ViewFileSystem extends FileSystem {
   @Override
   @SuppressWarnings("deprecation")
   public boolean delete(final Path f)
-      throws AccessControlException, FileNotFoundException,
-      IOException {
-      return delete(f, true);
+      throws AccessControlException, FileNotFoundException, IOException {
+    return delete(f, true);
   }
   
   @Override
@@ -339,7 +343,6 @@ public class ViewFileSystem extends FileSystem {
     return res.targetFileSystem.getFileChecksum(res.remainingPath);
   }
 
-
   private static FileStatus fixFileStatus(FileStatus orig,
       Path qualified) throws IOException {
     // FileStatus#getPath is a fully qualified path relative to the root of
@@ -367,7 +370,6 @@ public class ViewFileSystem extends FileSystem {
         : new ViewFsFileStatus(orig, qualified);
   }
 
-
   @Override
   public FileStatus getFileStatus(final Path f) throws AccessControlException,
       FileNotFoundException, IOException {
@@ -407,10 +409,10 @@ public class ViewFileSystem extends FileSystem {
   @Override
   public RemoteIterator<LocatedFileStatus>listLocatedStatus(final Path f,
       final PathFilter filter) throws FileNotFoundException, IOException {
-    final InodeTree.ResolveResult<FileSystem> res = fsState
-        .resolve(getUriPath(f), true);
-    final RemoteIterator<LocatedFileStatus> statusIter = res.targetFileSystem
-        .listLocatedStatus(res.remainingPath);
+    final InodeTree.ResolveResult<FileSystem> res =
+        fsState.resolve(getUriPath(f), true);
+    final RemoteIterator<LocatedFileStatus> statusIter =
+        res.targetFileSystem.listLocatedStatus(res.remainingPath);
 
     if (res.isInternalDir()) {
       return statusIter;
@@ -449,8 +451,7 @@ public class ViewFileSystem extends FileSystem {
 
   @Override
   public FSDataInputStream open(final Path f, final int bufferSize)
-      throws AccessControlException, FileNotFoundException,
-      IOException {
+      throws AccessControlException, FileNotFoundException, IOException {
     InodeTree.ResolveResult<FileSystem> res = 
         fsState.resolve(getUriPath(f), true);
     return res.targetFileSystem.open(res.remainingPath, bufferSize);
@@ -507,8 +508,7 @@ public class ViewFileSystem extends FileSystem {
   @Override
   public void setOwner(final Path f, final String username,
       final String groupname) throws AccessControlException,
-      FileNotFoundException,
-      IOException {
+      FileNotFoundException, IOException {
     InodeTree.ResolveResult<FileSystem> res = 
       fsState.resolve(getUriPath(f), true);
     res.targetFileSystem.setOwner(res.remainingPath, username, groupname); 
@@ -516,8 +516,7 @@ public class ViewFileSystem extends FileSystem {
 
   @Override
   public void setPermission(final Path f, final FsPermission permission)
-      throws AccessControlException, FileNotFoundException,
-      IOException {
+      throws AccessControlException, FileNotFoundException, IOException {
     InodeTree.ResolveResult<FileSystem> res = 
       fsState.resolve(getUriPath(f), true);
     res.targetFileSystem.setPermission(res.remainingPath, permission); 
@@ -525,8 +524,7 @@ public class ViewFileSystem extends FileSystem {
 
   @Override
   public boolean setReplication(final Path f, final short replication)
-      throws AccessControlException, FileNotFoundException,
-      IOException {
+      throws AccessControlException, FileNotFoundException, IOException {
     InodeTree.ResolveResult<FileSystem> res = 
       fsState.resolve(getUriPath(f), true);
     return res.targetFileSystem.setReplication(res.remainingPath, replication);
@@ -534,8 +532,7 @@ public class ViewFileSystem extends FileSystem {
 
   @Override
   public void setTimes(final Path f, final long mtime, final long atime)
-      throws AccessControlException, FileNotFoundException,
-      IOException {
+      throws AccessControlException, FileNotFoundException, IOException {
     InodeTree.ResolveResult<FileSystem> res = 
       fsState.resolve(getUriPath(f), true);
     res.targetFileSystem.setTimes(res.remainingPath, mtime, atime); 
@@ -793,8 +790,8 @@ public class ViewFileSystem extends FileSystem {
     return allPolicies;
   }
 
-  /*
-   * An instance of this class represents an internal dir of the viewFs 
+  /**
+   * An instance of this class represents an internal dir of the viewFs
    * that is internal dir of the mount table.
    * It is a read only mount tables and create, mkdir or delete operations
    * are not allowed.
@@ -826,8 +823,8 @@ public class ViewFileSystem extends FileSystem {
 
     static private void checkPathIsSlash(final Path f) throws IOException {
       if (f != InodeTree.SlashPath) {
-        throw new IOException (
-        "Internal implementation error: expected file name to be /" );
+        throw new IOException(
+            "Internal implementation error: expected file name to be /");
       }
     }
     
@@ -838,14 +835,14 @@ public class ViewFileSystem extends FileSystem {
 
     @Override
     public Path getWorkingDirectory() {
-      throw new RuntimeException (
-      "Internal impl error: getWorkingDir should not have been called" );
+      throw new RuntimeException(
+          "Internal impl error: getWorkingDir should not have been called");
     }
 
     @Override
     public void setWorkingDirectory(final Path new_dir) {
-      throw new RuntimeException (
-      "Internal impl error: getWorkingDir should not have been called" ); 
+      throw new RuntimeException(
+          "Internal impl error: getWorkingDir should not have been called");
     }
 
     @Override
@@ -878,7 +875,7 @@ public class ViewFileSystem extends FileSystem {
 
     @Override
     public BlockLocation[] getFileBlockLocations(final FileStatus fs,
-        final long start, final long len) throws 
+        final long start, final long len) throws
         FileNotFoundException, IOException {
       checkPathIsSlash(fs.getPath());
       throw new FileNotFoundException("Path points to dir not a file");
@@ -1055,7 +1052,7 @@ public class ViewFileSystem extends FileSystem {
 
     @Override
     public void setXAttr(Path path, String name, byte[] value,
-                         EnumSet<XAttrSetFlag> flag) throws IOException {
+        EnumSet<XAttrSetFlag> flag) throws IOException {
       checkPathIsSlash(path);
       throw readOnlyMountTable("setXAttr", path);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0f4afc81/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
index 75b329c..895ae0c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <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.
@@ -22,49 +22,41 @@ import java.net.URI;
 import java.net.URISyntaxException;
 
 import org.apache.hadoop.conf.Configuration;
-
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
-import org.apache.hadoop.fs.viewfs.ConfigUtil;
-import org.apache.hadoop.fs.viewfs.InodeTree;
 import org.junit.Test;
 
-
 public class TestViewFsConfig {
-  
-  
-  @Test(expected=FileAlreadyExistsException.class)
+
+  @Test(expected = FileAlreadyExistsException.class)
   public void testInvalidConfig() throws IOException, URISyntaxException {
     Configuration conf = new Configuration();
     ConfigUtil.addLink(conf, "/internalDir/linkToDir2",
         new Path("file:///dir2").toUri());
     ConfigUtil.addLink(conf, "/internalDir/linkToDir2/linkToDir3",
         new Path("file:///dir3").toUri());
-    
-    class Foo { };
-    
-     new InodeTree<Foo>(conf, null) {
+
+    class Foo {
+    }
+
+    new InodeTree<Foo>(conf, null) {
 
       @Override
-      protected
-      Foo getTargetFileSystem(final URI uri)
-        throws URISyntaxException, UnsupportedFileSystemException {
-          return null;
+      protected Foo getTargetFileSystem(final URI uri)
+          throws URISyntaxException, UnsupportedFileSystemException {
+        return null;
       }
 
       @Override
-      protected
-      Foo getTargetFileSystem(
-          org.apache.hadoop.fs.viewfs.InodeTree.INodeDir<Foo>
-                                          dir)
-        throws URISyntaxException {
+      protected Foo getTargetFileSystem(
+          org.apache.hadoop.fs.viewfs.InodeTree.INodeDir<Foo> dir)
+          throws URISyntaxException {
         return null;
       }
 
       @Override
-      protected
-      Foo getTargetFileSystem(URI[] mergeFsURIList)
+      protected Foo getTargetFileSystem(URI[] mergeFsURIList)
           throws URISyntaxException, UnsupportedFileSystemException {
         return null;
       }


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


[37/50] hadoop git commit: HDFS-10301. Remove FBR tracking state to fix false zombie storage detection for interleaving block reports. Contributed by Vinitha Gankidi.

Posted by um...@apache.org.
HDFS-10301. Remove FBR tracking state to fix false zombie storage detection for interleaving block reports. Contributed by Vinitha Gankidi.

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

Branch: refs/heads/HDFS-10285
Commit: 391ce535a739dc92cb90017d759217265a4fd969
Parents: 30bb197
Author: Vinitha Reddy Gankidi <vi...@linkedin.com>
Authored: Fri Oct 14 10:37:44 2016 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Fri Oct 14 18:13:54 2016 -0700

----------------------------------------------------------------------
 .../server/blockmanagement/BlockManager.java    | 75 ++++++--------------
 .../blockmanagement/DatanodeDescriptor.java     | 48 -------------
 .../blockmanagement/DatanodeStorageInfo.java    | 11 ---
 .../hdfs/server/namenode/NameNodeRpcServer.java |  4 +-
 .../blockmanagement/TestBlockManager.java       | 19 +++--
 .../TestNameNodePrunesMissingStorages.java      | 70 +++++++++++++++---
 .../server/datanode/BlockReportTestBase.java    | 50 +++++++++++++
 .../TestAddOverReplicatedStripedBlocks.java     |  4 ++
 8 files changed, 147 insertions(+), 134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/391ce535/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 7949439..7b13add 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -1347,6 +1347,8 @@ public class BlockManager implements BlockStatsMXBean {
       }
     }
     checkSafeMode();
+    LOG.info("Removed blocks associated with storage {} from DataNode {}",
+        storageInfo, node);
   }
 
   /**
@@ -2191,7 +2193,7 @@ public class BlockManager implements BlockStatsMXBean {
   public boolean processReport(final DatanodeID nodeID,
       final DatanodeStorage storage,
       final BlockListAsLongs newReport,
-      BlockReportContext context, boolean lastStorageInRpc) throws IOException {
+      BlockReportContext context) throws IOException {
     namesystem.writeLock();
     final long startTime = Time.monotonicNow(); //after acquiring write lock
     final long endTime;
@@ -2245,32 +2247,6 @@ public class BlockManager implements BlockStatsMXBean {
       }
       
       storageInfo.receivedBlockReport();
-      if (context != null) {
-        storageInfo.setLastBlockReportId(context.getReportId());
-        if (lastStorageInRpc) {
-          int rpcsSeen = node.updateBlockReportContext(context);
-          if (rpcsSeen >= context.getTotalRpcs()) {
-            long leaseId = blockReportLeaseManager.removeLease(node);
-            BlockManagerFaultInjector.getInstance().
-                removeBlockReportLease(node, leaseId);
-            List<DatanodeStorageInfo> zombies = node.removeZombieStorages();
-            if (zombies.isEmpty()) {
-              LOG.debug("processReport 0x{}: no zombie storages found.",
-                  Long.toHexString(context.getReportId()));
-            } else {
-              for (DatanodeStorageInfo zombie : zombies) {
-                removeZombieReplicas(context, zombie);
-              }
-            }
-            node.clearBlockReportContext();
-          } else {
-            LOG.debug("processReport 0x{}: {} more RPCs remaining in this " +
-                    "report.", Long.toHexString(context.getReportId()),
-                (context.getTotalRpcs() - rpcsSeen)
-            );
-          }
-        }
-      }
     } finally {
       endTime = Time.monotonicNow();
       namesystem.writeUnlock();
@@ -2295,36 +2271,25 @@ public class BlockManager implements BlockStatsMXBean {
     return !node.hasStaleStorages();
   }
 
-  private void removeZombieReplicas(BlockReportContext context,
-      DatanodeStorageInfo zombie) {
-    LOG.warn("processReport 0x{}: removing zombie storage {}, which no " +
-            "longer exists on the DataNode.",
-        Long.toHexString(context.getReportId()), zombie.getStorageID());
-    assert(namesystem.hasWriteLock());
-    Iterator<BlockInfo> iter = zombie.getBlockIterator();
-    int prevBlocks = zombie.numBlocks();
-    while (iter.hasNext()) {
-      BlockInfo block = iter.next();
-      // We assume that a block can be on only one storage in a DataNode.
-      // That's why we pass in the DatanodeDescriptor rather than the
-      // DatanodeStorageInfo.
-      // TODO: remove this assumption in case we want to put a block on
-      // more than one storage on a datanode (and because it's a difficult
-      // assumption to really enforce)
-      // DatanodeStorageInfo must be removed using the iterator to avoid
-      // ConcurrentModificationException in the underlying storage
-      iter.remove();
-      removeStoredBlock(block, zombie.getDatanodeDescriptor());
-      Block b = getBlockOnStorage(block, zombie);
-      if (b != null) {
-        invalidateBlocks.remove(zombie.getDatanodeDescriptor(), b);
+  public void removeBRLeaseIfNeeded(final DatanodeID nodeID,
+      final BlockReportContext context) throws IOException {
+    namesystem.writeLock();
+    DatanodeDescriptor node;
+    try {
+      node = datanodeManager.getDatanode(nodeID);
+      if (context != null) {
+        if (context.getTotalRpcs() == context.getCurRpc() + 1) {
+          long leaseId = this.getBlockReportLeaseManager().removeLease(node);
+          BlockManagerFaultInjector.getInstance().
+              removeBlockReportLease(node, leaseId);
+        }
+        LOG.debug("Processing RPC with index {} out of total {} RPCs in "
+                + "processReport 0x{}", context.getCurRpc(),
+            context.getTotalRpcs(), Long.toHexString(context.getReportId()));
       }
+    } finally {
+      namesystem.writeUnlock();
     }
-    assert(zombie.numBlocks() == 0);
-    LOG.warn("processReport 0x{}: removed {} replicas from storage {}, " +
-            "which no longer exists on the DataNode.",
-        Long.toHexString(context.getReportId()), prevBlocks,
-        zombie.getStorageID());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/391ce535/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index c74d7c5..6d163ec 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -31,7 +30,6 @@ import java.util.Queue;
 import java.util.Set;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableList;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -43,7 +41,6 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
-import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
@@ -68,8 +65,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
       LoggerFactory.getLogger(DatanodeDescriptor.class);
   public static final DatanodeDescriptor[] EMPTY_ARRAY = {};
   private static final int BLOCKS_SCHEDULED_ROLL_INTERVAL = 600*1000; //10min
-  private static final List<DatanodeStorageInfo> EMPTY_STORAGE_INFO_LIST =
-      ImmutableList.of();
 
   /** Block and targets pair */
   @InterfaceAudience.Private
@@ -154,10 +149,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
   public final DecommissioningStatus decommissioningStatus =
       new DecommissioningStatus();
 
-  private long curBlockReportId = 0;
-
-  private BitSet curBlockReportRpcsSeen = null;
-
   private final Map<String, DatanodeStorageInfo> storageMap =
       new HashMap<>();
 
@@ -257,20 +248,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     updateHeartbeatState(StorageReport.EMPTY_ARRAY, 0L, 0L, 0, 0, null);
   }
 
-  public int updateBlockReportContext(BlockReportContext context) {
-    if (curBlockReportId != context.getReportId()) {
-      curBlockReportId = context.getReportId();
-      curBlockReportRpcsSeen = new BitSet(context.getTotalRpcs());
-    }
-    curBlockReportRpcsSeen.set(context.getCurRpc());
-    return curBlockReportRpcsSeen.cardinality();
-  }
-
-  public void clearBlockReportContext() {
-    curBlockReportId = 0;
-    curBlockReportRpcsSeen = null;
-  }
-
   public CachedBlocksList getPendingCached() {
     return pendingCached;
   }
@@ -334,31 +311,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
 
-  List<DatanodeStorageInfo> removeZombieStorages() {
-    List<DatanodeStorageInfo> zombies = null;
-    synchronized (storageMap) {
-      Iterator<Map.Entry<String, DatanodeStorageInfo>> iter =
-          storageMap.entrySet().iterator();
-      while (iter.hasNext()) {
-        Map.Entry<String, DatanodeStorageInfo> entry = iter.next();
-        DatanodeStorageInfo storageInfo = entry.getValue();
-        if (storageInfo.getLastBlockReportId() != curBlockReportId) {
-          LOG.info("{} had lastBlockReportId 0x{} but curBlockReportId = 0x{}",
-              storageInfo.getStorageID(),
-              Long.toHexString(storageInfo.getLastBlockReportId()),
-              Long.toHexString(curBlockReportId));
-          iter.remove();
-          if (zombies == null) {
-            zombies = new LinkedList<>();
-          }
-          zombies.add(storageInfo);
-        }
-        storageInfo.setLastBlockReportId(0);
-      }
-    }
-    return zombies == null ? EMPTY_STORAGE_INFO_LIST : zombies;
-  }
-
   public void resetBlocks() {
     setCapacity(0);
     setRemaining(0);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/391ce535/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
index d98a2c1..b4c8aaa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
@@ -99,9 +99,6 @@ public class DatanodeStorageInfo {
 
   private final FoldedTreeSet<BlockInfo> blocks = new FoldedTreeSet<>();
 
-  // The ID of the last full block report which updated this storage.
-  private long lastBlockReportId = 0;
-
   /** The number of block reports received */
   private int blockReportCount = 0;
 
@@ -166,14 +163,6 @@ public class DatanodeStorageInfo {
     this.blockPoolUsed = blockPoolUsed;
   }
 
-  long getLastBlockReportId() {
-    return lastBlockReportId;
-  }
-
-  void setLastBlockReportId(long lastBlockReportId) {
-    this.lastBlockReportId = lastBlockReportId;
-  }
-
   State getState() {
     return this.state;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/391ce535/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index a97a307..7894163 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -1441,11 +1441,13 @@ public class NameNodeRpcServer implements NamenodeProtocols {
         @Override
         public Boolean call() throws IOException {
           return bm.processReport(nodeReg, reports[index].getStorage(),
-              blocks, context, (index == reports.length - 1));
+              blocks, context);
         }
       });
       metrics.incrStorageBlockReportOps();
     }
+    bm.removeBRLeaseIfNeeded(nodeReg, context);
+
     BlockManagerFaultInjector.getInstance().
         incomingBlockReportRpc(nodeReg, context);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/391ce535/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
index 942569a..2c7c720 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
@@ -716,12 +716,12 @@ public class TestBlockManager {
     reset(node);
     
     bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
-        BlockListAsLongs.EMPTY, null, false);
+        BlockListAsLongs.EMPTY, null);
     assertEquals(1, ds.getBlockReportCount());
     // send block report again, should NOT be processed
     reset(node);
     bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
-        BlockListAsLongs.EMPTY, null, false);
+        BlockListAsLongs.EMPTY, null);
     assertEquals(1, ds.getBlockReportCount());
 
     // re-register as if node restarted, should update existing node
@@ -732,7 +732,7 @@ public class TestBlockManager {
     // send block report, should be processed after restart
     reset(node);
     bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
-                     BlockListAsLongs.EMPTY, null, false);
+                     BlockListAsLongs.EMPTY, null);
     // Reinitialize as registration with empty storage list pruned
     // node.storageMap.
     ds = node.getStorageInfos()[0];
@@ -761,7 +761,7 @@ public class TestBlockManager {
     reset(node);
     doReturn(1).when(node).numBlocks();
     bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
-        BlockListAsLongs.EMPTY, null, false);
+        BlockListAsLongs.EMPTY, null);
     assertEquals(1, ds.getBlockReportCount());
   }
 
@@ -835,7 +835,7 @@ public class TestBlockManager {
     assertEquals(0, ds.getBlockReportCount());
     bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
         builder.build(),
-        new BlockReportContext(1, 0, System.nanoTime(), 0, true), false);
+        new BlockReportContext(1, 0, System.nanoTime(), 0, true));
     assertEquals(1, ds.getBlockReportCount());
 
     // verify the storage info is correct
@@ -874,8 +874,7 @@ public class TestBlockManager {
     assertEquals(0, ds.getBlockReportCount());
     bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
                      generateReport(blocks),
-                     new BlockReportContext(1, 0, System.nanoTime(), 0, false),
-                     false);
+                     new BlockReportContext(1, 0, System.nanoTime(), 0, false));
     assertEquals(1, ds.getBlockReportCount());
     // verify the storage info is correct
     for (BlockInfo block : blocks) {
@@ -885,8 +884,7 @@ public class TestBlockManager {
     // Send unsorted report
     bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
                      generateReport(blocks),
-                     new BlockReportContext(1, 0, System.nanoTime(), 0, false),
-                     false);
+                     new BlockReportContext(1, 0, System.nanoTime(), 0, false));
     assertEquals(2, ds.getBlockReportCount());
     // verify the storage info is correct
     for (BlockInfo block : blocks) {
@@ -897,8 +895,7 @@ public class TestBlockManager {
     Collections.sort(blocks);
     bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
                      generateReport(blocks),
-                     new BlockReportContext(1, 0, System.nanoTime(), 0, true),
-                     false);
+                     new BlockReportContext(1, 0, System.nanoTime(), 0, true));
     assertEquals(3, ds.getBlockReportCount());
     // verify the storage info is correct
     for (BlockInfo block : blocks) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/391ce535/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
index 6efc53a..274627f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
@@ -19,24 +19,23 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import com.google.common.base.Supplier;
+import java.util.ArrayList;
+import java.util.Collection;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.fs.FSDataOutputStream;
 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.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi.FsVolumeReferences;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -47,7 +46,6 @@ import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Test;
 
-import java.io.BufferedOutputStream;
 import java.io.BufferedReader;
 import java.io.BufferedWriter;
 import java.io.File;
@@ -56,13 +54,11 @@ import java.io.FileReader;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
-import java.util.Arrays;
-import java.util.EnumSet;
 import java.util.Iterator;
-import java.util.List;
 import java.util.UUID;
 
 import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertNotNull;
@@ -160,6 +156,8 @@ public class TestNameNodePrunesMissingStorages {
   public void testRemovingStorageDoesNotProduceZombies() throws Exception {
     Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
+        1000);
     final int NUM_STORAGES_PER_DN = 2;
     final MiniDFSCluster cluster = new MiniDFSCluster
         .Builder(conf).numDataNodes(3)
@@ -262,7 +260,7 @@ public class TestNameNodePrunesMissingStorages {
           assertEquals(NUM_STORAGES_PER_DN - 1, infos.length);
           return true;
         }
-      }, 10, 30000);
+      }, 1000, 30000);
     } finally {
       if (cluster != null) {
         cluster.shutdown();
@@ -371,4 +369,60 @@ public class TestNameNodePrunesMissingStorages {
       cluster.shutdown();
     }
   }
+
+  @Test(timeout=300000)
+  public void testNameNodePrunesUnreportedStorages() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    // Create a cluster with one datanode with two storages
+    MiniDFSCluster cluster = new MiniDFSCluster
+        .Builder(conf).numDataNodes(1)
+        .storagesPerDatanode(2)
+        .build();
+    // Create two files to ensure each storage has a block
+    DFSTestUtil.createFile(cluster.getFileSystem(), new Path("file1"),
+        102400, 102400, 102400, (short)1,
+        0x1BAD5EE);
+    DFSTestUtil.createFile(cluster.getFileSystem(), new Path("file2"),
+        102400, 102400, 102400, (short)1,
+        0x1BAD5EED);
+    // Get the datanode storages and data directories
+    DataNode dn = cluster.getDataNodes().get(0);
+    BlockManager bm = cluster.getNameNode().getNamesystem().getBlockManager();
+    DatanodeDescriptor dnDescriptor = bm.getDatanodeManager().
+        getDatanode(cluster.getDataNodes().get(0).getDatanodeUuid());
+    DatanodeStorageInfo[] dnStoragesInfosBeforeRestart =
+        dnDescriptor.getStorageInfos();
+    Collection<String> oldDirs =  new ArrayList<String>(dn.getConf().
+        getTrimmedStringCollection(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY));
+    // Keep the first data directory and remove the second.
+    String newDirs = oldDirs.iterator().next();
+    conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, newDirs);
+    // Restart the datanode with the new conf
+    cluster.stopDataNode(0);
+    cluster.startDataNodes(conf, 1, false, null, null);
+    dn = cluster.getDataNodes().get(0);
+    cluster.waitActive();
+    // Assert that the dnDescriptor has both the storages after restart
+    assertArrayEquals(dnStoragesInfosBeforeRestart,
+        dnDescriptor.getStorageInfos());
+    // Assert that the removed storage is marked as FAILED
+    // when DN heartbeats to the NN
+    int numFailedStoragesWithBlocks = 0;
+    DatanodeStorageInfo failedStorageInfo = null;
+    for (DatanodeStorageInfo dnStorageInfo: dnDescriptor.getStorageInfos()) {
+      if (dnStorageInfo.areBlocksOnFailedStorage()) {
+        numFailedStoragesWithBlocks++;
+        failedStorageInfo = dnStorageInfo;
+      }
+    }
+    assertEquals(1, numFailedStoragesWithBlocks);
+    // Heartbeat manager removes the blocks associated with this failed storage
+    bm.getDatanodeManager().getHeartbeatManager().heartbeatCheck();
+    assertTrue(!failedStorageInfo.areBlocksOnFailedStorage());
+    // pruneStorageMap removes the unreported storage
+    cluster.triggerHeartbeats();
+    // Assert that the unreported storage is pruned
+    assertEquals(DataNode.getStorageLocations(dn.getConf()).size(),
+        dnDescriptor.getStorageInfos().length);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/391ce535/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
index 53b9263..6810a0b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
@@ -29,7 +29,12 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
@@ -50,7 +55,10 @@ import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
@@ -649,6 +657,48 @@ public abstract class BlockReportTestBase {
     DFSTestUtil.readFile(fs, filePath);
   }
 
+  // See HDFS-10301
+  @Test(timeout = 300000)
+  public void testInterleavedBlockReports()
+      throws IOException, ExecutionException, InterruptedException {
+    int numConcurrentBlockReports = 3;
+    DataNode dn = cluster.getDataNodes().get(DN_N0);
+    final String poolId = cluster.getNamesystem().getBlockPoolId();
+    LOG.info("Block pool id: " + poolId);
+    final DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
+    final StorageBlockReport[] reports =
+        getBlockReports(dn, poolId, true, true);
+
+    // Get the list of storage ids associated with the datanode
+    // before the test
+    BlockManager bm = cluster.getNameNode().getNamesystem().getBlockManager();
+    final DatanodeDescriptor dnDescriptor =
+        bm.getDatanodeManager().getDatanode(dn.getDatanodeId());
+    DatanodeStorageInfo[] storageInfos = dnDescriptor.getStorageInfos();
+
+    // Send the block report concurrently using
+    // numThreads=numConcurrentBlockReports
+    ExecutorService executorService =
+        Executors.newFixedThreadPool(numConcurrentBlockReports);
+    List<Future<Void>> futureList = new ArrayList<>(numConcurrentBlockReports);
+    for (int i = 0; i < numConcurrentBlockReports; i++) {
+      futureList.add(executorService.submit(new Callable<Void>() {
+        @Override
+        public Void call() throws IOException {
+          sendBlockReports(dnR, poolId, reports);
+          return null;
+        }
+      }));
+    }
+    for (Future<Void> future : futureList) {
+      future.get();
+    }
+    executorService.shutdown();
+
+    // Verify that the storages match before and after the test
+    Assert.assertArrayEquals(storageInfos, dnDescriptor.getStorageInfos());
+  }
+
   private void waitForTempReplica(Block bl, int DN_N1) throws IOException {
     final boolean tooLongWait = false;
     final int TIMEOUT = 40000;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/391ce535/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java
index 7b281a6..13dcccf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
@@ -236,6 +237,9 @@ public class TestAddOverReplicatedStripedBlocks {
     }
   }
 
+  // This test is going to be rewritten in HDFS-10854. Ignoring this test
+  // temporarily as it fails with the fix for HDFS-10301.
+  @Ignore
   @Test
   public void testProcessOverReplicatedAndMissingStripedBlock()
       throws Exception {


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


[41/50] hadoop git commit: HADOOP-13661. Upgrade HTrace version. Contributed by Sean Mackrory.

Posted by um...@apache.org.
HADOOP-13661. Upgrade HTrace version. Contributed by Sean Mackrory.


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

Branch: refs/heads/HDFS-10285
Commit: ed9fcbec544df149d08d9ac31989a7291eff6507
Parents: 1f304b0
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Mon Oct 17 05:04:49 2016 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Mon Oct 17 05:04:49 2016 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/src/site/markdown/Tracing.md | 2 +-
 hadoop-project/pom.xml                                           | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed9fcbec/hadoop-common-project/hadoop-common/src/site/markdown/Tracing.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Tracing.md b/hadoop-common-project/hadoop-common/src/site/markdown/Tracing.md
index cbdee8a..9b7084d 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Tracing.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Tracing.md
@@ -48,7 +48,7 @@ LocalFileSpanReceiver is included in the htrace-core4 jar which is bundled
 with Hadoop.)
 
 ```
-    $ cp htrace-htraced/target/htrace-htraced-4.0.1-incubating.jar $HADOOP_HOME/share/hadoop/common/lib/
+    $ cp htrace-htraced/target/htrace-htraced-4.1.0-incubating.jar $HADOOP_HOME/share/hadoop/common/lib/
 ```
 
 ### Dynamic update of tracing configuration

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed9fcbec/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 82adebf..5826cf6 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -882,7 +882,7 @@
       <dependency>
         <groupId>org.apache.htrace</groupId>
         <artifactId>htrace-core4</artifactId>
-        <version>4.0.1-incubating</version>
+        <version>4.1.0-incubating</version>
       </dependency>
       <dependency>
         <groupId>org.jdom</groupId>


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


[48/50] hadoop git commit: HDFS-10794. [SPS]: Provide storage policy satisfy worker at DN for co-ordinating the block storage movement work. Contributed by Rakesh R

Posted by um...@apache.org.
HDFS-10794. [SPS]: Provide storage policy satisfy worker at DN for co-ordinating the block storage movement work. Contributed by Rakesh R


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

Branch: refs/heads/HDFS-10285
Commit: 8aa5e5ac1ab380298ed0f2d3aabafd3020bae530
Parents: f5d9235
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Sep 14 17:02:11 2016 +0800
Committer: Uma Maheswara Rao G <um...@intel.com>
Committed: Mon Oct 17 14:47:39 2016 -0700

----------------------------------------------------------------------
 .../datanode/StoragePolicySatisfyWorker.java    | 258 +++++++++++++++++++
 .../protocol/BlockStorageMovementCommand.java   | 101 ++++++++
 .../TestStoragePolicySatisfyWorker.java         | 159 ++++++++++++
 3 files changed, 518 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aa5e5ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
new file mode 100644
index 0000000..6df4e81
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
@@ -0,0 +1,258 @@
+/**
+ * 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;
+
+import static org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed;
+
+import java.io.BufferedInputStream;
+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.Socket;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
+import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.Daemon;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * StoragePolicySatisfyWorker handles the storage policy satisfier 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
+ * BlockStorageMovement commands.
+ */
+@InterfaceAudience.Private
+public class StoragePolicySatisfyWorker {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(StoragePolicySatisfyWorker.class);
+
+  private final DataNode datanode;
+  private final int ioFileBufferSize;
+
+  private final int moverThreads;
+  private final ExecutorService moveExecutor;
+  private final CompletionService<Void> moverExecutorCompletionService;
+  private final List<Future<Void>> moverTaskFutures;
+
+  public StoragePolicySatisfyWorker(Configuration conf, DataNode datanode) {
+    this.datanode = datanode;
+    this.ioFileBufferSize = DFSUtilClient.getIoFileBufferSize(conf);
+
+    moverThreads = conf.getInt(DFSConfigKeys.DFS_MOVER_MOVERTHREADS_KEY,
+        DFSConfigKeys.DFS_MOVER_MOVERTHREADS_DEFAULT);
+    moveExecutor = initializeBlockMoverThreadPool(moverThreads);
+    moverExecutorCompletionService = new ExecutorCompletionService<>(
+        moveExecutor);
+    moverTaskFutures = new ArrayList<>();
+    // TODO: Needs to manage the number of concurrent moves per DataNode.
+  }
+
+  private ThreadPoolExecutor initializeBlockMoverThreadPool(int num) {
+    LOG.debug("Block mover to satisfy storage policy; pool threads={}", num);
+
+    ThreadPoolExecutor moverThreadPool = 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("BlockMoverTask-" + threadIndex.getAndIncrement());
+            return t;
+          }
+        }, new ThreadPoolExecutor.CallerRunsPolicy() {
+          @Override
+          public void rejectedExecution(Runnable runnable,
+              ThreadPoolExecutor e) {
+            LOG.info("Execution for block movement to satisfy storage policy"
+                + " got rejected, Executing in current thread");
+            // will run in the current thread.
+            super.rejectedExecution(runnable, e);
+          }
+        });
+
+    moverThreadPool.allowCoreThreadTimeOut(true);
+    return moverThreadPool;
+  }
+
+  public void processBlockMovingTasks(long trackID,
+      List<BlockMovingInfo> blockMovingInfos) {
+    Future<Void> moveCallable = null;
+    for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
+      assert blkMovingInfo
+          .getSources().length == blkMovingInfo.getTargets().length;
+
+      for (int i = 0; i < blkMovingInfo.getSources().length; i++) {
+        BlockMovingTask blockMovingTask =
+            new BlockMovingTask(blkMovingInfo.getBlock(),
+            blkMovingInfo.getSources()[i],
+            blkMovingInfo.getTargets()[i],
+            blkMovingInfo.getTargetStorageTypes()[i]);
+        moveCallable = moverExecutorCompletionService
+            .submit(blockMovingTask);
+        moverTaskFutures.add(moveCallable);
+      }
+    }
+
+    // TODO: Presently this function act as a blocking call, this has to be
+    // refined by moving the tracking logic to another tracker thread.
+    for (int i = 0; i < moverTaskFutures.size(); i++) {
+      try {
+        moveCallable = moverExecutorCompletionService.take();
+        moveCallable.get();
+      } catch (InterruptedException | ExecutionException e) {
+        // TODO: Failure retries and report back the error to NameNode.
+        LOG.error("Exception while moving block replica to target storage type",
+            e);
+      }
+    }
+  }
+
+  /**
+   * This class encapsulates the process of moving the block replica to the
+   * given target.
+   */
+  private class BlockMovingTask implements Callable<Void> {
+    private final ExtendedBlock block;
+    private final DatanodeInfo source;
+    private final DatanodeInfo target;
+    private final StorageType targetStorageType;
+
+    BlockMovingTask(ExtendedBlock block, DatanodeInfo source,
+        DatanodeInfo target, StorageType targetStorageType) {
+      this.block = block;
+      this.source = source;
+      this.target = target;
+      this.targetStorageType = targetStorageType;
+    }
+
+    @Override
+    public Void call() {
+      moveBlock();
+      return null;
+    }
+
+    private void moveBlock() {
+      LOG.info("Start moving block {}", block);
+
+      LOG.debug("Start moving block:{} from src:{} to destin:{} to satisfy "
+          + "storageType:{}", block, source, target, targetStorageType);
+      Socket sock = null;
+      DataOutputStream out = null;
+      DataInputStream in = null;
+      try {
+        DNConf dnConf = datanode.getDnConf();
+        String dnAddr = target.getXferAddr(dnConf.getConnectToDnViaHostname());
+        sock = datanode.newSocket();
+        NetUtils.connect(sock, NetUtils.createSocketAddr(dnAddr),
+            dnConf.getSocketTimeout());
+        sock.setSoTimeout(2 * dnConf.getSocketTimeout());
+        LOG.debug("Connecting to datanode {}", dnAddr);
+
+        OutputStream unbufOut = sock.getOutputStream();
+        InputStream unbufIn = sock.getInputStream();
+
+        Token<BlockTokenIdentifier> accessToken = datanode.getBlockAccessToken(
+            block, EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE));
+
+        DataEncryptionKeyFactory keyFactory = datanode
+            .getDataEncryptionKeyFactoryForBlock(block);
+        IOStreamPair saslStreams = datanode.getSaslClient().socketSend(sock,
+            unbufOut, unbufIn, keyFactory, accessToken, target);
+        unbufOut = saslStreams.out;
+        unbufIn = saslStreams.in;
+        out = new DataOutputStream(
+            new BufferedOutputStream(unbufOut, ioFileBufferSize));
+        in = new DataInputStream(
+            new BufferedInputStream(unbufIn, ioFileBufferSize));
+        sendRequest(out, block, accessToken, source, targetStorageType);
+        receiveResponse(in);
+
+        LOG.debug(
+            "Successfully moved block:{} from src:{} to destin:{} for"
+                + " satisfying storageType:{}",
+            block, source, target, targetStorageType);
+      } catch (IOException e) {
+        // TODO: handle failure retries
+        LOG.warn(
+            "Failed to move block:{} from src:{} to destin:{} to satisfy "
+                + "storageType:{}",
+            block, source, target, targetStorageType, e);
+      } finally {
+        IOUtils.closeStream(out);
+        IOUtils.closeStream(in);
+        IOUtils.closeSocket(sock);
+      }
+    }
+
+    /** Send a reportedBlock replace request to the output stream. */
+    private void sendRequest(DataOutputStream out, ExtendedBlock eb,
+        Token<BlockTokenIdentifier> accessToken, DatanodeInfo srcDn,
+        StorageType destinStorageType) throws IOException {
+      new Sender(out).replaceBlock(eb, destinStorageType, accessToken,
+          srcDn.getDatanodeUuid(), srcDn);
+    }
+
+    /** Receive a reportedBlock copy response from the input stream. */
+    private void receiveResponse(DataInputStream in) throws IOException {
+      BlockOpResponseProto response = BlockOpResponseProto
+          .parseFrom(vintPrefixed(in));
+      while (response.getStatus() == Status.IN_PROGRESS) {
+        // read intermediate responses
+        response = BlockOpResponseProto.parseFrom(vintPrefixed(in));
+      }
+      String logInfo = "reportedBlock move is failed";
+      DataTransferProtoUtil.checkBlockOpStatus(response, logInfo);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aa5e5ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
new file mode 100644
index 0000000..42ba265
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
@@ -0,0 +1,101 @@
+/**
+ * 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.protocol;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+
+/**
+ * A BlockStorageMovementCommand is an instruction to a DataNode to move the
+ * given set of blocks to specified target DataNodes to fulfill the block
+ * storage policy.
+ *
+ * Upon receiving this command, this DataNode coordinates all the block movement
+ * by passing the details to
+ * {@link org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker}
+ * service. After the block movement this DataNode sends response back to the
+ * NameNode about the movement status.
+ */
+public class BlockStorageMovementCommand extends DatanodeCommand {
+
+  // TODO: constructor needs to be refined based on the block movement data
+  // structure.
+  BlockStorageMovementCommand(int action) {
+    super(action);
+  }
+
+  /**
+   * Stores block to storage info that can be used for block movement.
+   */
+  public static class BlockMovingInfo {
+    private ExtendedBlock blk;
+    private DatanodeInfo[] sourceNodes;
+    private StorageType[] sourceStorageTypes;
+    private DatanodeInfo[] targetNodes;
+    private StorageType[] targetStorageTypes;
+
+    public BlockMovingInfo(ExtendedBlock block,
+        DatanodeInfo[] sourceDnInfos, DatanodeInfo[] targetDnInfos,
+        StorageType[] srcStorageTypes, StorageType[] targetStorageTypes) {
+      this.blk = block;
+      this.sourceNodes = sourceDnInfos;
+      this.targetNodes = targetDnInfos;
+      this.sourceStorageTypes = srcStorageTypes;
+      this.targetStorageTypes = targetStorageTypes;
+    }
+
+    public void addBlock(ExtendedBlock block) {
+      this.blk = block;
+    }
+
+    public ExtendedBlock getBlock() {
+      return this.blk;
+    }
+
+    public DatanodeInfo[] getSources() {
+      return sourceNodes;
+    }
+
+    public DatanodeInfo[] getTargets() {
+      return targetNodes;
+    }
+
+    public StorageType[] getTargetStorageTypes() {
+      return targetStorageTypes;
+    }
+
+    public StorageType[] getSourceStorageTypes() {
+      return sourceStorageTypes;
+    }
+
+    @Override
+    public String toString() {
+      return new StringBuilder().append("BlockMovingInfo(\n  ")
+          .append("Moving block: ").append(blk).append(" From: ")
+          .append(Arrays.asList(sourceNodes)).append(" To: [")
+          .append(Arrays.asList(targetNodes)).append(")\n")
+          .append(" sourceStorageTypes: ")
+          .append(Arrays.toString(sourceStorageTypes))
+          .append(" targetStorageTypes: ")
+          .append(Arrays.toString(targetStorageTypes)).toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aa5e5ac/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStoragePolicySatisfyWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStoragePolicySatisfyWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStoragePolicySatisfyWorker.java
new file mode 100644
index 0000000..692847d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStoragePolicySatisfyWorker.java
@@ -0,0 +1,159 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Supplier;
+
+/**
+ * This class tests the behavior of moving block replica to the given storage
+ * type to fulfill the storage policy requirement.
+ */
+public class TestStoragePolicySatisfyWorker {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestStoragePolicySatisfyWorker.class);
+
+  private static final int DEFAULT_BLOCK_SIZE = 100;
+
+  private static void initConf(Configuration conf) {
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
+    conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BLOCK_SIZE);
+    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L);
+    conf.setLong(DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 2000L);
+  }
+
+  /**
+   * Tests to verify that the block replica is moving to ARCHIVE storage type to
+   * fulfill the storage policy requirement.
+   */
+  @Test(timeout = 120000)
+  public void testMoveSingleBlockToAnotherDatanode() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+    initConf(conf);
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(4)
+        .storageTypes(
+            new StorageType[][] {{StorageType.DISK, StorageType.ARCHIVE},
+                {StorageType.DISK, StorageType.ARCHIVE},
+                {StorageType.DISK, StorageType.ARCHIVE},
+                {StorageType.DISK, StorageType.ARCHIVE}})
+        .build();
+    try {
+      cluster.waitActive();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      final String file = "/testMoveSingleBlockToAnotherDatanode";
+      // write to DISK
+      final FSDataOutputStream out = dfs.create(new Path(file), (short) 2);
+      out.writeChars("testMoveSingleBlockToAnotherDatanode");
+      out.close();
+
+      // verify before movement
+      LocatedBlock lb = dfs.getClient().getLocatedBlocks(file, 0).get(0);
+      StorageType[] storageTypes = lb.getStorageTypes();
+      for (StorageType storageType : storageTypes) {
+        Assert.assertTrue(StorageType.DISK == storageType);
+      }
+      // move to ARCHIVE
+      dfs.setStoragePolicy(new Path(file), "COLD");
+
+      lb = dfs.getClient().getLocatedBlocks(file, 0).get(0);
+      DataNode src = cluster.getDataNodes().get(3);
+      DatanodeInfo targetDnInfo = DFSTestUtil
+          .getLocalDatanodeInfo(src.getXferPort());
+
+      // TODO: Need to revisit this when NN is implemented to be able to send
+      // block moving commands.
+      StoragePolicySatisfyWorker worker = new StoragePolicySatisfyWorker(conf,
+          src);
+      List<BlockMovingInfo> blockMovingInfos = new ArrayList<>();
+      BlockMovingInfo blockMovingInfo = prepareBlockMovingInfo(
+          lb.getBlock(), lb.getLocations()[0], targetDnInfo,
+          lb.getStorageTypes()[0], StorageType.ARCHIVE);
+      blockMovingInfos.add(blockMovingInfo);
+      INode inode = cluster.getNamesystem().getFSDirectory().getINode(file);
+      worker.processBlockMovingTasks(inode.getId(),
+          blockMovingInfos);
+      cluster.triggerHeartbeats();
+
+      // Wait till NameNode notified about the block location details
+      waitForLocatedBlockWithArchiveStorageType(dfs, file, 1, 30000);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  private void waitForLocatedBlockWithArchiveStorageType(
+      final DistributedFileSystem dfs, final String file,
+      int expectedArchiveCount, int timeout) throws Exception {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        LocatedBlock lb = null;
+        try {
+          lb = dfs.getClient().getLocatedBlocks(file, 0).get(0);
+        } catch (IOException e) {
+          LOG.error("Exception while getting located blocks", e);
+          return false;
+        }
+
+        int archiveCount = 0;
+        for (StorageType storageType : lb.getStorageTypes()) {
+          if (StorageType.ARCHIVE == storageType) {
+            archiveCount++;
+          }
+        }
+        LOG.info("Archive replica count, expected={} and actual={}",
+            expectedArchiveCount, archiveCount);
+        return expectedArchiveCount == archiveCount;
+      }
+    }, 100, timeout);
+  }
+
+  BlockMovingInfo prepareBlockMovingInfo(ExtendedBlock block,
+      DatanodeInfo src, DatanodeInfo destin, StorageType storageType,
+      StorageType targetStorageType) {
+    return new BlockMovingInfo(block, new DatanodeInfo[] {src},
+        new DatanodeInfo[] {destin}, new StorageType[] {storageType},
+        new StorageType[] {targetStorageType});
+  }
+}


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


[40/50] hadoop git commit: YARN-5699. Retrospect yarn entity fields which are publishing in events info fields. Contributed by Rohith Sharma K S.

Posted by um...@apache.org.
YARN-5699. Retrospect yarn entity fields which are publishing in events info fields. 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/1f304b0c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1f304b0c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1f304b0c

Branch: refs/heads/HDFS-10285
Commit: 1f304b0c7f261369dd68839507bb609a949965ad
Parents: 5f4ae85
Author: Sangjin Lee <sj...@apache.org>
Authored: Sat Oct 15 13:54:40 2016 -0700
Committer: Sangjin Lee <sj...@apache.org>
Committed: Sat Oct 15 13:54:40 2016 -0700

----------------------------------------------------------------------
 ...pplicationHistoryManagerOnTimelineStore.java |  69 ++++++-------
 ...pplicationHistoryManagerOnTimelineStore.java |  38 +++----
 .../metrics/AppAttemptMetricsConstants.java     |  16 +--
 .../metrics/ContainerMetricsConstants.java      |  21 ++--
 .../timelineservice/NMTimelinePublisher.java    |  34 ++++---
 .../metrics/TimelineServiceV1Publisher.java     |  44 ++++----
 .../metrics/TimelineServiceV2Publisher.java     | 101 +++++++++----------
 .../metrics/TestSystemMetricsPublisher.java     |  40 ++++----
 8 files changed, 186 insertions(+), 177 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f304b0c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
index feeafdd..6e6576a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
@@ -463,21 +463,21 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
           if (eventInfo == null) {
             continue;
           }
-          if (eventInfo.containsKey(AppAttemptMetricsConstants.HOST_EVENT_INFO)) {
+          if (eventInfo.containsKey(AppAttemptMetricsConstants.HOST_INFO)) {
             host =
-                eventInfo.get(AppAttemptMetricsConstants.HOST_EVENT_INFO)
+                eventInfo.get(AppAttemptMetricsConstants.HOST_INFO)
                     .toString();
           }
           if (eventInfo
-              .containsKey(AppAttemptMetricsConstants.RPC_PORT_EVENT_INFO)) {
+              .containsKey(AppAttemptMetricsConstants.RPC_PORT_INFO)) {
             rpcPort = (Integer) eventInfo.get(
-                    AppAttemptMetricsConstants.RPC_PORT_EVENT_INFO);
+                    AppAttemptMetricsConstants.RPC_PORT_INFO);
           }
           if (eventInfo
-              .containsKey(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO)) {
+              .containsKey(AppAttemptMetricsConstants.MASTER_CONTAINER_INFO)) {
             amContainerId =
                 ContainerId.fromString(eventInfo.get(
-                    AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO)
+                    AppAttemptMetricsConstants.MASTER_CONTAINER_INFO)
                     .toString());
           }
         } else if (event.getEventType().equals(
@@ -487,39 +487,40 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
             continue;
           }
           if (eventInfo
-              .containsKey(AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO)) {
+              .containsKey(AppAttemptMetricsConstants.TRACKING_URL_INFO)) {
             trackingUrl =
                 eventInfo.get(
-                    AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO)
+                    AppAttemptMetricsConstants.TRACKING_URL_INFO)
                     .toString();
           }
           if (eventInfo
-              .containsKey(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO)) {
+              .containsKey(
+                  AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_INFO)) {
             originalTrackingUrl =
                 eventInfo
                     .get(
-                        AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO)
+                        AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_INFO)
                     .toString();
           }
           if (eventInfo
-              .containsKey(AppAttemptMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO)) {
+              .containsKey(AppAttemptMetricsConstants.DIAGNOSTICS_INFO)) {
             diagnosticsInfo =
                 eventInfo.get(
-                    AppAttemptMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO)
+                    AppAttemptMetricsConstants.DIAGNOSTICS_INFO)
                     .toString();
           }
           if (eventInfo
-              .containsKey(AppAttemptMetricsConstants.STATE_EVENT_INFO)) {
+              .containsKey(AppAttemptMetricsConstants.STATE_INFO)) {
             state =
                 YarnApplicationAttemptState.valueOf(eventInfo.get(
-                    AppAttemptMetricsConstants.STATE_EVENT_INFO)
+                    AppAttemptMetricsConstants.STATE_INFO)
                     .toString());
           }
           if (eventInfo
-              .containsKey(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO)) {
+              .containsKey(AppAttemptMetricsConstants.MASTER_CONTAINER_INFO)) {
             amContainerId =
                 ContainerId.fromString(eventInfo.get(
-                    AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO)
+                    AppAttemptMetricsConstants.MASTER_CONTAINER_INFO)
                     .toString());
           }
         }
@@ -547,37 +548,37 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
     Map<String, Object> entityInfo = entity.getOtherInfo();
     if (entityInfo != null) {
       if (entityInfo
-          .containsKey(ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO)) {
+          .containsKey(ContainerMetricsConstants.ALLOCATED_MEMORY_INFO)) {
         allocatedMem = (Integer) entityInfo.get(
-                ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO);
+                ContainerMetricsConstants.ALLOCATED_MEMORY_INFO);
       }
       if (entityInfo
-          .containsKey(ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO)) {
+          .containsKey(ContainerMetricsConstants.ALLOCATED_VCORE_INFO)) {
         allocatedVcore = (Integer) entityInfo.get(
-                ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO);
+                ContainerMetricsConstants.ALLOCATED_VCORE_INFO);
       }
       if (entityInfo
-          .containsKey(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO)) {
+          .containsKey(ContainerMetricsConstants.ALLOCATED_HOST_INFO)) {
         allocatedHost =
             entityInfo
-                .get(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO)
+                .get(ContainerMetricsConstants.ALLOCATED_HOST_INFO)
                 .toString();
       }
       if (entityInfo
-          .containsKey(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO)) {
+          .containsKey(ContainerMetricsConstants.ALLOCATED_PORT_INFO)) {
         allocatedPort = (Integer) entityInfo.get(
-                ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO);
+                ContainerMetricsConstants.ALLOCATED_PORT_INFO);
       }
       if (entityInfo
-          .containsKey(ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO)) {
+          .containsKey(ContainerMetricsConstants.ALLOCATED_PRIORITY_INFO)) {
         allocatedPriority = (Integer) entityInfo.get(
-                ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO);
+                ContainerMetricsConstants.ALLOCATED_PRIORITY_INFO);
       }
       if (entityInfo.containsKey(
-          ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO)) {
+          ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_INFO)) {
         nodeHttpAddress =
             (String) entityInfo
-              .get(ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO);
+              .get(ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_INFO);
       }
     }
     List<TimelineEvent> events = entity.getEvents();
@@ -594,22 +595,22 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
             continue;
           }
           if (eventInfo
-              .containsKey(ContainerMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO)) {
+              .containsKey(ContainerMetricsConstants.DIAGNOSTICS_INFO)) {
             diagnosticsInfo =
                 eventInfo.get(
-                    ContainerMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO)
+                    ContainerMetricsConstants.DIAGNOSTICS_INFO)
                     .toString();
           }
           if (eventInfo
-              .containsKey(ContainerMetricsConstants.EXIT_STATUS_EVENT_INFO)) {
+              .containsKey(ContainerMetricsConstants.EXIT_STATUS_INFO)) {
             exitStatus = (Integer) eventInfo.get(
-                    ContainerMetricsConstants.EXIT_STATUS_EVENT_INFO);
+                    ContainerMetricsConstants.EXIT_STATUS_INFO);
           }
           if (eventInfo
-              .containsKey(ContainerMetricsConstants.STATE_EVENT_INFO)) {
+              .containsKey(ContainerMetricsConstants.STATE_INFO)) {
             state =
                 ContainerState.valueOf(eventInfo.get(
-                    ContainerMetricsConstants.STATE_EVENT_INFO).toString());
+                    ContainerMetricsConstants.STATE_INFO).toString());
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f304b0c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java
index dd1a453..526cb8f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java
@@ -593,13 +593,13 @@ public class TestApplicationHistoryManagerOnTimelineStore {
     tEvent.setEventType(AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE);
     tEvent.setTimestamp(Integer.MAX_VALUE + 1L);
     Map<String, Object> eventInfo = new HashMap<String, Object>();
-    eventInfo.put(AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO,
+    eventInfo.put(AppAttemptMetricsConstants.TRACKING_URL_INFO,
         "test tracking url");
-    eventInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO,
+    eventInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_INFO,
         "test original tracking url");
-    eventInfo.put(AppAttemptMetricsConstants.HOST_EVENT_INFO, "test host");
-    eventInfo.put(AppAttemptMetricsConstants.RPC_PORT_EVENT_INFO, 100);
-    eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO,
+    eventInfo.put(AppAttemptMetricsConstants.HOST_INFO, "test host");
+    eventInfo.put(AppAttemptMetricsConstants.RPC_PORT_INFO, 100);
+    eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_INFO,
         ContainerId.newContainerId(appAttemptId, 1));
     tEvent.setEventInfo(eventInfo);
     entity.addEvent(tEvent);
@@ -607,15 +607,15 @@ public class TestApplicationHistoryManagerOnTimelineStore {
     tEvent.setEventType(AppAttemptMetricsConstants.FINISHED_EVENT_TYPE);
     tEvent.setTimestamp(Integer.MAX_VALUE + 2L);
     eventInfo = new HashMap<String, Object>();
-    eventInfo.put(AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO,
+    eventInfo.put(AppAttemptMetricsConstants.TRACKING_URL_INFO,
         "test tracking url");
-    eventInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO,
+    eventInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_INFO,
         "test original tracking url");
-    eventInfo.put(AppAttemptMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
+    eventInfo.put(AppAttemptMetricsConstants.DIAGNOSTICS_INFO,
         "test diagnostics info");
-    eventInfo.put(AppAttemptMetricsConstants.FINAL_STATUS_EVENT_INFO,
+    eventInfo.put(AppAttemptMetricsConstants.FINAL_STATUS_INFO,
         FinalApplicationStatus.UNDEFINED.toString());
-    eventInfo.put(AppAttemptMetricsConstants.STATE_EVENT_INFO,
+    eventInfo.put(AppAttemptMetricsConstants.STATE_INFO,
         YarnApplicationAttemptState.FINISHED.toString());
     tEvent.setEventInfo(eventInfo);
     entity.addEvent(tEvent);
@@ -632,15 +632,15 @@ public class TestApplicationHistoryManagerOnTimelineStore {
     entity.addPrimaryFilter(
         TimelineStore.SystemFilter.ENTITY_OWNER.toString(), "yarn");
     Map<String, Object> entityInfo = new HashMap<String, Object>();
-    entityInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO, -1);
-    entityInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO, -1);
-    entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO,
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_INFO, -1);
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_INFO, -1);
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_INFO,
         "test host");
-    entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO, 100);
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_INFO, 100);
     entityInfo
-        .put(ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO, -1);
+        .put(ContainerMetricsConstants.ALLOCATED_PRIORITY_INFO, -1);
     entityInfo.put(ContainerMetricsConstants
-        .ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO, "http://test:1234");
+        .ALLOCATED_HOST_HTTP_ADDRESS_INFO, "http://test:1234");
     entity.setOtherInfo(entityInfo);
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setEventType(ContainerMetricsConstants.CREATED_EVENT_TYPE);
@@ -651,10 +651,10 @@ public class TestApplicationHistoryManagerOnTimelineStore {
     tEvent.setEventType(ContainerMetricsConstants.FINISHED_EVENT_TYPE);
     tEvent.setTimestamp(Integer.MAX_VALUE + 2L);
     Map<String, Object> eventInfo = new HashMap<String, Object>();
-    eventInfo.put(ContainerMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
+    eventInfo.put(ContainerMetricsConstants.DIAGNOSTICS_INFO,
         "test diagnostics info");
-    eventInfo.put(ContainerMetricsConstants.EXIT_STATUS_EVENT_INFO, -1);
-    eventInfo.put(ContainerMetricsConstants.STATE_EVENT_INFO,
+    eventInfo.put(ContainerMetricsConstants.EXIT_STATUS_INFO, -1);
+    eventInfo.put(ContainerMetricsConstants.STATE_INFO,
         ContainerState.COMPLETE.toString());
     tEvent.setEventInfo(eventInfo);
     entity.addEvent(tEvent);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f304b0c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/AppAttemptMetricsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/AppAttemptMetricsConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/AppAttemptMetricsConstants.java
index a7809cf..2f61f43 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/AppAttemptMetricsConstants.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/AppAttemptMetricsConstants.java
@@ -37,28 +37,28 @@ public class AppAttemptMetricsConstants {
   public static final String PARENT_PRIMARY_FILTER =
       "YARN_APPLICATION_ATTEMPT_PARENT";
       
-  public static final String TRACKING_URL_EVENT_INFO =
+  public static final String TRACKING_URL_INFO =
       "YARN_APPLICATION_ATTEMPT_TRACKING_URL";
 
-  public static final String ORIGINAL_TRACKING_URL_EVENT_INFO =
+  public static final String ORIGINAL_TRACKING_URL_INFO =
       "YARN_APPLICATION_ATTEMPT_ORIGINAL_TRACKING_URL";
 
-  public static final String HOST_EVENT_INFO =
+  public static final String HOST_INFO =
       "YARN_APPLICATION_ATTEMPT_HOST";
 
-  public static final String RPC_PORT_EVENT_INFO =
+  public static final String RPC_PORT_INFO =
       "YARN_APPLICATION_ATTEMPT_RPC_PORT";
 
-  public static final String MASTER_CONTAINER_EVENT_INFO =
+  public static final String MASTER_CONTAINER_INFO =
       "YARN_APPLICATION_ATTEMPT_MASTER_CONTAINER";
 
-  public static final String DIAGNOSTICS_INFO_EVENT_INFO =
+  public static final String DIAGNOSTICS_INFO =
       "YARN_APPLICATION_ATTEMPT_DIAGNOSTICS_INFO";
 
-  public static final String FINAL_STATUS_EVENT_INFO =
+  public static final String FINAL_STATUS_INFO =
       "YARN_APPLICATION_ATTEMPT_FINAL_STATUS";
 
-  public static final String STATE_EVENT_INFO =
+  public static final String STATE_INFO =
       "YARN_APPLICATION_ATTEMPT_STATE";
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f304b0c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ContainerMetricsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ContainerMetricsConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ContainerMetricsConstants.java
index eadb5b7..9cf2b0a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ContainerMetricsConstants.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ContainerMetricsConstants.java
@@ -41,33 +41,36 @@ public class ContainerMetricsConstants {
   public static final String FINISHED_IN_RM_EVENT_TYPE =
       "YARN_RM_CONTAINER_FINISHED";
 
+  public static final String CONTAINER_FINISHED_TIME =
+      "YARN_CONTAINER_FINISHED_TIME";
+
   public static final String PARENT_PRIMARIY_FILTER = "YARN_CONTAINER_PARENT";
 
-  public static final String ALLOCATED_MEMORY_ENTITY_INFO =
+  public static final String ALLOCATED_MEMORY_INFO =
       "YARN_CONTAINER_ALLOCATED_MEMORY";
 
-  public static final String ALLOCATED_VCORE_ENTITY_INFO =
+  public static final String ALLOCATED_VCORE_INFO =
       "YARN_CONTAINER_ALLOCATED_VCORE";
 
-  public static final String ALLOCATED_HOST_ENTITY_INFO =
+  public static final String ALLOCATED_HOST_INFO =
       "YARN_CONTAINER_ALLOCATED_HOST";
 
-  public static final String ALLOCATED_PORT_ENTITY_INFO =
+  public static final String ALLOCATED_PORT_INFO =
       "YARN_CONTAINER_ALLOCATED_PORT";
 
-  public static final String ALLOCATED_PRIORITY_ENTITY_INFO =
+  public static final String ALLOCATED_PRIORITY_INFO =
       "YARN_CONTAINER_ALLOCATED_PRIORITY";
 
-  public static final String DIAGNOSTICS_INFO_EVENT_INFO =
+  public static final String DIAGNOSTICS_INFO =
       "YARN_CONTAINER_DIAGNOSTICS_INFO";
 
-  public static final String EXIT_STATUS_EVENT_INFO =
+  public static final String EXIT_STATUS_INFO =
       "YARN_CONTAINER_EXIT_STATUS";
 
-  public static final String STATE_EVENT_INFO =
+  public static final String STATE_INFO =
       "YARN_CONTAINER_STATE";
 
-  public static final String ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO =
+  public static final String ALLOCATED_HOST_HTTP_ADDRESS_INFO =
       "YARN_CONTAINER_ALLOCATED_HOST_HTTP_ADDRESS";
 
   // Event of this type will be emitted by NM.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f304b0c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.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/timelineservice/NMTimelinePublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java
index 8e68889..d54189f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -98,7 +99,6 @@ public class NMTimelinePublisher extends CompositeService {
     // context will be updated after containerManagerImpl is started
     // hence NMMetricsPublisher is added subservice of containerManagerImpl
     this.nodeId = context.getNodeId();
-    this.httpAddress = nodeId.getHost() + ":" + context.getHttpPort();
   }
 
   @VisibleForTesting
@@ -167,18 +167,18 @@ public class NMTimelinePublisher extends CompositeService {
     Resource resource = container.getResource();
 
     Map<String, Object> entityInfo = new HashMap<String, Object>();
-    entityInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO,
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_INFO,
         resource.getMemorySize());
-    entityInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO,
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_INFO,
         resource.getVirtualCores());
-    entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO,
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_INFO,
         nodeId.getHost());
-    entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO,
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_INFO,
         nodeId.getPort());
-    entityInfo.put(ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO,
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_PRIORITY_INFO,
         container.getPriority().toString());
     entityInfo.put(
-        ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO,
+        ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_INFO,
         httpAddress);
     entity.setInfo(entityInfo);
 
@@ -198,19 +198,20 @@ public class NMTimelinePublisher extends CompositeService {
     ContainerId containerId = containerStatus.getContainerId();
     TimelineEntity entity = createContainerEntity(containerId);
 
-    Map<String, Object> eventInfo = new HashMap<String, Object>();
-    eventInfo.put(ContainerMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
+    Map<String, Object> entityInfo = new HashMap<String, Object>();
+    entityInfo.put(ContainerMetricsConstants.DIAGNOSTICS_INFO,
         containerStatus.getDiagnostics());
-    eventInfo.put(ContainerMetricsConstants.EXIT_STATUS_EVENT_INFO,
+    entityInfo.put(ContainerMetricsConstants.EXIT_STATUS_INFO,
         containerStatus.getExitStatus());
-    eventInfo.put(ContainerMetricsConstants.STATE_EVENT_INFO, containerStatus
-        .getState().toString());
+    entityInfo.put(ContainerMetricsConstants.STATE_INFO,
+        ContainerState.COMPLETE.toString());
+    entityInfo.put(ContainerMetricsConstants.CONTAINER_FINISHED_TIME,
+        timeStamp);
+    entity.setInfo(entityInfo);
 
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setId(ContainerMetricsConstants.FINISHED_EVENT_TYPE);
     tEvent.setTimestamp(timeStamp);
-    tEvent.setInfo(eventInfo);
-
     entity.addEvent(tEvent);
 
     dispatcher.getEventHandler().handle(new TimelinePublishEvent(entity,
@@ -304,6 +305,11 @@ public class NMTimelinePublisher extends CompositeService {
 
   public void publishContainerEvent(ContainerEvent event) {
     // publish only when the desired event is received
+    if (this.httpAddress == null) {
+      // update httpAddress for first time. When this service started,
+      // web server will not be started.
+      this.httpAddress = nodeId.getHost() + ":" + context.getHttpPort();
+    }
     switch (event.getType()) {
     case INIT_CONTAINER:
       publishContainerCreatedEvent(event);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f304b0c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV1Publisher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV1Publisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV1Publisher.java
index ffbc747..cbf6a73 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV1Publisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV1Publisher.java
@@ -215,16 +215,16 @@ public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher {
     tEvent.setEventType(AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE);
     tEvent.setTimestamp(registeredTime);
     Map<String, Object> eventInfo = new HashMap<String, Object>();
-    eventInfo.put(AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO,
+    eventInfo.put(AppAttemptMetricsConstants.TRACKING_URL_INFO,
         appAttempt.getTrackingUrl());
-    eventInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO,
+    eventInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_INFO,
         appAttempt.getOriginalTrackingUrl());
-    eventInfo.put(AppAttemptMetricsConstants.HOST_EVENT_INFO,
+    eventInfo.put(AppAttemptMetricsConstants.HOST_INFO,
         appAttempt.getHost());
-    eventInfo.put(AppAttemptMetricsConstants.RPC_PORT_EVENT_INFO,
+    eventInfo.put(AppAttemptMetricsConstants.RPC_PORT_INFO,
         appAttempt.getRpcPort());
     if (appAttempt.getMasterContainer() != null) {
-      eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO,
+      eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_INFO,
           appAttempt.getMasterContainer().getId().toString());
     }
     tEvent.setEventInfo(eventInfo);
@@ -246,18 +246,18 @@ public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher {
     tEvent.setEventType(AppAttemptMetricsConstants.FINISHED_EVENT_TYPE);
     tEvent.setTimestamp(finishedTime);
     Map<String, Object> eventInfo = new HashMap<String, Object>();
-    eventInfo.put(AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO,
+    eventInfo.put(AppAttemptMetricsConstants.TRACKING_URL_INFO,
         appAttempt.getTrackingUrl());
-    eventInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO,
+    eventInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_INFO,
         appAttempt.getOriginalTrackingUrl());
-    eventInfo.put(AppAttemptMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
+    eventInfo.put(AppAttemptMetricsConstants.DIAGNOSTICS_INFO,
         appAttempt.getDiagnostics());
-    eventInfo.put(AppAttemptMetricsConstants.FINAL_STATUS_EVENT_INFO,
+    eventInfo.put(AppAttemptMetricsConstants.FINAL_STATUS_INFO,
         app.getFinalApplicationStatus().toString());
-    eventInfo.put(AppAttemptMetricsConstants.STATE_EVENT_INFO, RMServerUtils
+    eventInfo.put(AppAttemptMetricsConstants.STATE_INFO, RMServerUtils
         .createApplicationAttemptState(appAttemtpState).toString());
     if (appAttempt.getMasterContainer() != null) {
-      eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO,
+      eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_INFO,
           appAttempt.getMasterContainer().getId().toString());
     }
     tEvent.setEventInfo(eventInfo);
@@ -273,18 +273,18 @@ public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher {
   public void containerCreated(RMContainer container, long createdTime) {
     TimelineEntity entity = createContainerEntity(container.getContainerId());
     Map<String, Object> entityInfo = new HashMap<String, Object>();
-    entityInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO,
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_INFO,
         container.getAllocatedResource().getMemorySize());
-    entityInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO,
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_INFO,
         container.getAllocatedResource().getVirtualCores());
-    entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO,
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_INFO,
         container.getAllocatedNode().getHost());
-    entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO,
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_INFO,
         container.getAllocatedNode().getPort());
-    entityInfo.put(ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO,
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_PRIORITY_INFO,
         container.getAllocatedPriority().getPriority());
     entityInfo.put(
-        ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO,
+        ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_INFO,
         container.getNodeHttpAddress());
     entity.setOtherInfo(entityInfo);
 
@@ -307,16 +307,16 @@ public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher {
     tEvent.setEventType(ContainerMetricsConstants.FINISHED_EVENT_TYPE);
     tEvent.setTimestamp(finishedTime);
     Map<String, Object> eventInfo = new HashMap<String, Object>();
-    eventInfo.put(ContainerMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
+    eventInfo.put(ContainerMetricsConstants.DIAGNOSTICS_INFO,
         container.getDiagnosticsInfo());
-    eventInfo.put(ContainerMetricsConstants.EXIT_STATUS_EVENT_INFO,
+    eventInfo.put(ContainerMetricsConstants.EXIT_STATUS_INFO,
         container.getContainerExitStatus());
-    eventInfo.put(ContainerMetricsConstants.STATE_EVENT_INFO,
+    eventInfo.put(ContainerMetricsConstants.STATE_INFO,
         container.getContainerState().toString());
     Map<String, Object> entityInfo = new HashMap<String, Object>();
-    entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO,
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_INFO,
         container.getAllocatedNode().getHost());
-    entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO,
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_INFO,
         container.getAllocatedNode().getPort());
     entity.setOtherInfo(entityInfo);
     tEvent.setEventInfo(eventInfo);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f304b0c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
index 1485b91..dbdc1a8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
@@ -157,22 +157,22 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
     tEvent.setTimestamp(finishedTime);
-    Map<String, Object> eventInfo = new HashMap<String, Object>();
-    eventInfo.put(ApplicationMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
+    entity.addEvent(tEvent);
+
+    Map<String, Object> entityInfo = new HashMap<String, Object>();
+    entityInfo.put(ApplicationMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
         app.getDiagnostics().toString());
-    eventInfo.put(ApplicationMetricsConstants.FINAL_STATUS_EVENT_INFO,
+    entityInfo.put(ApplicationMetricsConstants.FINAL_STATUS_EVENT_INFO,
         app.getFinalApplicationStatus().toString());
-    eventInfo.put(ApplicationMetricsConstants.STATE_EVENT_INFO,
+    entityInfo.put(ApplicationMetricsConstants.STATE_EVENT_INFO,
         RMServerUtils.createApplicationState(state).toString());
     ApplicationAttemptId appAttemptId = app.getCurrentAppAttempt() == null
         ? null : app.getCurrentAppAttempt().getAppAttemptId();
     if (appAttemptId != null) {
-      eventInfo.put(ApplicationMetricsConstants.LATEST_APP_ATTEMPT_EVENT_INFO,
+      entityInfo.put(ApplicationMetricsConstants.LATEST_APP_ATTEMPT_EVENT_INFO,
           appAttemptId.toString());
     }
-    tEvent.setInfo(eventInfo);
-
-    entity.addEvent(tEvent);
+    entity.setInfo(entityInfo);
 
     getDispatcher().getEventHandler().handle(new TimelineV2PublishEvent(
         SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId()));
@@ -193,6 +193,11 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
     tEvent.setInfo(eventInfo);
     entity.addEvent(tEvent);
 
+    // publish in entity info also to query using filters
+    Map<String, Object> entityInfo = new HashMap<String, Object>();
+    entityInfo.put(ApplicationMetricsConstants.STATE_EVENT_INFO, appState);
+    entity.setInfo(entityInfo);
+
     getDispatcher().getEventHandler().handle(new TimelineV2PublishEvent(
         SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId()));
   }
@@ -250,21 +255,23 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setId(AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE);
     tEvent.setTimestamp(registeredTime);
-    Map<String, Object> eventInfo = new HashMap<String, Object>();
-    eventInfo.put(AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO,
+    entity.addEvent(tEvent);
+
+    Map<String, Object> entityInfo = new HashMap<String, Object>();
+    entityInfo.put(AppAttemptMetricsConstants.TRACKING_URL_INFO,
         appAttempt.getTrackingUrl());
-    eventInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO,
+    entityInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_INFO,
         appAttempt.getOriginalTrackingUrl());
-    eventInfo.put(AppAttemptMetricsConstants.HOST_EVENT_INFO,
+    entityInfo.put(AppAttemptMetricsConstants.HOST_INFO,
         appAttempt.getHost());
-    eventInfo.put(AppAttemptMetricsConstants.RPC_PORT_EVENT_INFO,
+    entityInfo.put(AppAttemptMetricsConstants.RPC_PORT_INFO,
         appAttempt.getRpcPort());
     if (appAttempt.getMasterContainer() != null) {
-      eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO,
+      entityInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_INFO,
           appAttempt.getMasterContainer().getId().toString());
     }
-    tEvent.setInfo(eventInfo);
-    entity.addEvent(tEvent);
+    entity.setInfo(entityInfo);
+
     getDispatcher().getEventHandler().handle(
         new TimelineV2PublishEvent(SystemMetricsEventType.PUBLISH_ENTITY,
             entity, appAttempt.getAppAttemptId().getApplicationId()));
@@ -281,26 +288,20 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setId(AppAttemptMetricsConstants.FINISHED_EVENT_TYPE);
     tEvent.setTimestamp(finishedTime);
-    Map<String, Object> eventInfo = new HashMap<String, Object>();
-    eventInfo.put(AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO,
-        appAttempt.getTrackingUrl());
-    eventInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO,
-        appAttempt.getOriginalTrackingUrl());
-    eventInfo.put(AppAttemptMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
+    entity.addEvent(tEvent);
+
+    Map<String, Object> entityInfo = new HashMap<String, Object>();
+    entityInfo.put(AppAttemptMetricsConstants.DIAGNOSTICS_INFO,
         appAttempt.getDiagnostics());
     // app will get the final status from app attempt, or create one
     // based on app state if it doesn't exist
-    eventInfo.put(AppAttemptMetricsConstants.FINAL_STATUS_EVENT_INFO,
+    entityInfo.put(AppAttemptMetricsConstants.FINAL_STATUS_INFO,
         app.getFinalApplicationStatus().toString());
-    eventInfo.put(AppAttemptMetricsConstants.STATE_EVENT_INFO, RMServerUtils
+    entityInfo.put(AppAttemptMetricsConstants.STATE_INFO, RMServerUtils
         .createApplicationAttemptState(appAttemtpState).toString());
-    if (appAttempt.getMasterContainer() != null) {
-      eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO,
-          appAttempt.getMasterContainer().getId().toString());
-    }
-    tEvent.setInfo(eventInfo);
+    entity.setInfo(entityInfo);
+
 
-    entity.addEvent(tEvent);
     getDispatcher().getEventHandler().handle(
         new TimelineV2PublishEvent(SystemMetricsEventType.PUBLISH_ENTITY,
             entity, appAttempt.getAppAttemptId().getApplicationId()));
@@ -325,25 +326,26 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
       TimelineEvent tEvent = new TimelineEvent();
       tEvent.setId(ContainerMetricsConstants.CREATED_IN_RM_EVENT_TYPE);
       tEvent.setTimestamp(createdTime);
+      entity.addEvent(tEvent);
+
       // updated as event info instead of entity info, as entity info is updated
       // by NM
-      Map<String, Object> eventInfo = new HashMap<String, Object>();
-      eventInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO,
+      Map<String, Object> entityInfo = new HashMap<String, Object>();
+      entityInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_INFO,
           container.getAllocatedResource().getMemorySize());
-      eventInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO,
+      entityInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_INFO,
           container.getAllocatedResource().getVirtualCores());
-      eventInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO,
+      entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_INFO,
           container.getAllocatedNode().getHost());
-      eventInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO,
+      entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_INFO,
           container.getAllocatedNode().getPort());
-      eventInfo.put(ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO,
+      entityInfo.put(ContainerMetricsConstants.ALLOCATED_PRIORITY_INFO,
           container.getAllocatedPriority().getPriority());
-      eventInfo.put(
-          ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO,
+      entityInfo.put(
+          ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_INFO,
           container.getNodeHttpAddress());
-      tEvent.setInfo(eventInfo);
+      entity.setInfo(entityInfo);
 
-      entity.addEvent(tEvent);
       getDispatcher().getEventHandler().handle(new TimelineV2PublishEvent(
           SystemMetricsEventType.PUBLISH_ENTITY, entity, container
               .getContainerId().getApplicationAttemptId().getApplicationId()));
@@ -359,22 +361,19 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
       TimelineEvent tEvent = new TimelineEvent();
       tEvent.setId(ContainerMetricsConstants.FINISHED_IN_RM_EVENT_TYPE);
       tEvent.setTimestamp(finishedTime);
-      Map<String, Object> eventInfo = new HashMap<String, Object>();
-      eventInfo.put(ContainerMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
+      entity.addEvent(tEvent);
+
+      Map<String, Object> entityInfo = new HashMap<String, Object>();
+      entityInfo.put(ContainerMetricsConstants.DIAGNOSTICS_INFO,
           container.getDiagnosticsInfo());
-      eventInfo.put(ContainerMetricsConstants.EXIT_STATUS_EVENT_INFO,
+      entityInfo.put(ContainerMetricsConstants.EXIT_STATUS_INFO,
           container.getContainerExitStatus());
-      eventInfo.put(ContainerMetricsConstants.STATE_EVENT_INFO,
+      entityInfo.put(ContainerMetricsConstants.STATE_INFO,
           container.getContainerState().toString());
-      Map<String, Object> entityInfo = new HashMap<String, Object>();
-      entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO,
-          container.getAllocatedNode().getHost());
-      entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO,
-          container.getAllocatedNode().getPort());
+      entityInfo.put(ContainerMetricsConstants.CONTAINER_FINISHED_TIME,
+          finishedTime);
       entity.setInfo(entityInfo);
-      tEvent.setInfo(eventInfo);
 
-      entity.addEvent(tEvent);
       getDispatcher().getEventHandler().handle(new TimelineV2PublishEvent(
           SystemMetricsEventType.PUBLISH_ENTITY, entity, container
               .getContainerId().getApplicationAttemptId().getApplicationId()));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f304b0c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java
index 386932d..b38091e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java
@@ -341,34 +341,34 @@ public class TestSystemMetricsPublisher {
         hasRegisteredEvent = true;
         Assert.assertEquals(appAttempt.getHost(),
             event.getEventInfo()
-                .get(AppAttemptMetricsConstants.HOST_EVENT_INFO));
+                .get(AppAttemptMetricsConstants.HOST_INFO));
         Assert
             .assertEquals(appAttempt.getRpcPort(),
                 event.getEventInfo().get(
-                    AppAttemptMetricsConstants.RPC_PORT_EVENT_INFO));
+                    AppAttemptMetricsConstants.RPC_PORT_INFO));
         Assert.assertEquals(
             appAttempt.getMasterContainer().getId().toString(),
             event.getEventInfo().get(
-                AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO));
+                AppAttemptMetricsConstants.MASTER_CONTAINER_INFO));
       } else if (event.getEventType().equals(
           AppAttemptMetricsConstants.FINISHED_EVENT_TYPE)) {
         hasFinishedEvent = true;
         Assert.assertEquals(appAttempt.getDiagnostics(), event.getEventInfo()
-            .get(AppAttemptMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO));
+            .get(AppAttemptMetricsConstants.DIAGNOSTICS_INFO));
         Assert.assertEquals(appAttempt.getTrackingUrl(), event.getEventInfo()
-            .get(AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO));
+            .get(AppAttemptMetricsConstants.TRACKING_URL_INFO));
         Assert.assertEquals(
             appAttempt.getOriginalTrackingUrl(),
             event.getEventInfo().get(
-                AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO));
+                AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_INFO));
         Assert.assertEquals(
             FinalApplicationStatus.UNDEFINED.toString(),
             event.getEventInfo().get(
-                AppAttemptMetricsConstants.FINAL_STATUS_EVENT_INFO));
+                AppAttemptMetricsConstants.FINAL_STATUS_INFO));
         Assert.assertEquals(
             YarnApplicationAttemptState.FINISHED.toString(),
             event.getEventInfo().get(
-                AppAttemptMetricsConstants.STATE_EVENT_INFO));
+                AppAttemptMetricsConstants.STATE_INFO));
       }
     }
     Assert.assertTrue(hasRegisteredEvent && hasFinishedEvent);
@@ -391,17 +391,17 @@ public class TestSystemMetricsPublisher {
     Assert.assertNotNull(entity.getOtherInfo());
     Assert.assertEquals(2, entity.getOtherInfo().size());
     Assert.assertNotNull(entity.getOtherInfo().get(
-        ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO));
+        ContainerMetricsConstants.ALLOCATED_HOST_INFO));
     Assert.assertNotNull(entity.getOtherInfo().get(
-        ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO));
+        ContainerMetricsConstants.ALLOCATED_PORT_INFO));
     Assert.assertEquals(
         container.getAllocatedNode().getHost(),
         entity.getOtherInfo().get(
-            ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO));
+            ContainerMetricsConstants.ALLOCATED_HOST_INFO));
     Assert.assertEquals(
         container.getAllocatedNode().getPort(),
         entity.getOtherInfo().get(
-            ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO));
+            ContainerMetricsConstants.ALLOCATED_PORT_INFO));
   }
 
   @Test(timeout = 10000)
@@ -432,25 +432,25 @@ public class TestSystemMetricsPublisher {
     Assert.assertEquals(
         container.getAllocatedNode().getHost(),
         entity.getOtherInfo().get(
-            ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO));
+            ContainerMetricsConstants.ALLOCATED_HOST_INFO));
     Assert.assertEquals(
         container.getAllocatedNode().getPort(),
         entity.getOtherInfo().get(
-            ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO));
+            ContainerMetricsConstants.ALLOCATED_PORT_INFO));
     Assert.assertEquals(container.getAllocatedResource().getMemorySize(),
         // KeyValueBasedTimelineStore could cast long to integer, need make sure
         // variables for compare have same type.
         ((Integer) entity.getOtherInfo().get(
-            ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO))
+            ContainerMetricsConstants.ALLOCATED_MEMORY_INFO))
             .longValue());
     Assert.assertEquals(
         container.getAllocatedResource().getVirtualCores(),
         entity.getOtherInfo().get(
-            ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO));
+            ContainerMetricsConstants.ALLOCATED_VCORE_INFO));
     Assert.assertEquals(
         container.getAllocatedPriority().getPriority(),
         entity.getOtherInfo().get(
-            ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO));
+            ContainerMetricsConstants.ALLOCATED_PRIORITY_INFO));
     boolean hasCreatedEvent = false;
     boolean hasFinishedEvent = false;
     for (TimelineEvent event : entity.getEvents()) {
@@ -465,13 +465,13 @@ public class TestSystemMetricsPublisher {
         Assert.assertEquals(
             container.getDiagnosticsInfo(),
             event.getEventInfo().get(
-                ContainerMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO));
+                ContainerMetricsConstants.DIAGNOSTICS_INFO));
         Assert.assertEquals(
             container.getContainerExitStatus(),
             event.getEventInfo().get(
-                ContainerMetricsConstants.EXIT_STATUS_EVENT_INFO));
+                ContainerMetricsConstants.EXIT_STATUS_INFO));
         Assert.assertEquals(container.getContainerState().toString(), event
-            .getEventInfo().get(ContainerMetricsConstants.STATE_EVENT_INFO));
+            .getEventInfo().get(ContainerMetricsConstants.STATE_INFO));
       }
     }
     Assert.assertTrue(hasCreatedEvent && hasFinishedEvent);


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


[22/50] hadoop git commit: Revert "HDFS-10990. TestPendingInvalidateBlock should wait for IBRs. Contributed by Yiqun Lin."

Posted by um...@apache.org.
Revert "HDFS-10990. TestPendingInvalidateBlock should wait for IBRs. Contributed by Yiqun Lin."

This reverts commit fdce515091f0a61ffd6c9ae464a68447dedf1124.


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

Branch: refs/heads/HDFS-10285
Commit: 8c721aa00a47a976959e3861ddd742f09db432fc
Parents: 332a61f
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Oct 13 13:23:12 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Oct 13 13:23:28 2016 -0700

----------------------------------------------------------------------
 .../blockmanagement/TestPendingInvalidateBlock.java    | 13 ++++---------
 1 file changed, 4 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c721aa0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
index 19f3178..696b2aa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
@@ -86,8 +86,6 @@ public class TestPendingInvalidateBlock {
   public void testPendingDeletion() throws Exception {
     final Path foo = new Path("/foo");
     DFSTestUtil.createFile(dfs, foo, BLOCKSIZE, REPLICATION, 0);
-    DFSTestUtil.waitForReplication(dfs, foo, REPLICATION, 10000);
-
     // restart NN
     cluster.restartNameNode(true);
     InvalidateBlocks invalidateBlocks =
@@ -100,7 +98,6 @@ public class TestPendingInvalidateBlock {
         "invalidateBlocks", mockIb);
     dfs.delete(foo, true);
 
-    waitForNumPendingDeletionBlocks(REPLICATION);
     Assert.assertEquals(0, cluster.getNamesystem().getBlocksTotal());
     Assert.assertEquals(REPLICATION, cluster.getNamesystem()
         .getPendingDeletionBlocks());
@@ -108,7 +105,7 @@ public class TestPendingInvalidateBlock {
         dfs.getPendingDeletionBlocksCount());
     Mockito.doReturn(0L).when(mockIb).getInvalidationDelay();
 
-    waitForNumPendingDeletionBlocks(0);
+    waitForBlocksToDelete();
     Assert.assertEquals(0, cluster.getNamesystem().getBlocksTotal());
     Assert.assertEquals(0, cluster.getNamesystem().getPendingDeletionBlocks());
     Assert.assertEquals(0, dfs.getPendingDeletionBlocksCount());
@@ -185,7 +182,7 @@ public class TestPendingInvalidateBlock {
     Assert.assertEquals(4, cluster.getNamesystem().getPendingDeletionBlocks());
 
     cluster.restartNameNode(true);
-    waitForNumPendingDeletionBlocks(0);
+    waitForBlocksToDelete();
     Assert.assertEquals(3, cluster.getNamesystem().getBlocksTotal());
     Assert.assertEquals(0, cluster.getNamesystem().getPendingDeletionBlocks());
   }
@@ -202,8 +199,7 @@ public class TestPendingInvalidateBlock {
     return cluster.getNamesystem().getUnderReplicatedBlocks();
   }
 
-  private void waitForNumPendingDeletionBlocks(int numBlocks)
-      throws Exception {
+  private void waitForBlocksToDelete() throws Exception {
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
 
       @Override
@@ -211,8 +207,7 @@ public class TestPendingInvalidateBlock {
         try {
           cluster.triggerBlockReports();
 
-          if (cluster.getNamesystem().getPendingDeletionBlocks()
-              == numBlocks) {
+          if (cluster.getNamesystem().getPendingDeletionBlocks() == 0) {
             return true;
           }
         } catch (Exception e) {


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


[11/50] hadoop git commit: YARN-5677. RM should transition to standby when connection is lost for an extended period. (Daniel Templeton via kasha)

Posted by um...@apache.org.
YARN-5677. RM should transition to standby when connection is lost for an extended period. (Daniel Templeton via kasha)


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

Branch: refs/heads/HDFS-10285
Commit: 6476934ae5de1be7988ab198b673d82fe0f006e3
Parents: 6378845
Author: Karthik Kambatla <ka...@cloudera.com>
Authored: Tue Oct 11 22:07:10 2016 -0700
Committer: Karthik Kambatla <ka...@cloudera.com>
Committed: Tue Oct 11 22:07:10 2016 -0700

----------------------------------------------------------------------
 .../resourcemanager/EmbeddedElectorService.java |  59 +++++-
 .../resourcemanager/TestRMEmbeddedElector.java  | 191 +++++++++++++++++++
 2 files changed, 244 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6476934a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/EmbeddedElectorService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/EmbeddedElectorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/EmbeddedElectorService.java
index 72327e8..88d2e10 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/EmbeddedElectorService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/EmbeddedElectorService.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -39,6 +40,8 @@ import org.apache.zookeeper.data.ACL;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.Timer;
+import java.util.TimerTask;
 
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
@@ -54,6 +57,10 @@ public class EmbeddedElectorService extends AbstractService
 
   private byte[] localActiveNodeInfo;
   private ActiveStandbyElector elector;
+  private long zkSessionTimeout;
+  private Timer zkDisconnectTimer;
+  @VisibleForTesting
+  final Object zkDisconnectLock = new Object();
 
   EmbeddedElectorService(RMContext rmContext) {
     super(EmbeddedElectorService.class.getName());
@@ -80,7 +87,7 @@ public class EmbeddedElectorService extends AbstractService
         YarnConfiguration.DEFAULT_AUTO_FAILOVER_ZK_BASE_PATH);
     String electionZNode = zkBasePath + "/" + clusterId;
 
-    long zkSessionTimeout = conf.getLong(YarnConfiguration.RM_ZK_TIMEOUT_MS,
+    zkSessionTimeout = conf.getLong(YarnConfiguration.RM_ZK_TIMEOUT_MS,
         YarnConfiguration.DEFAULT_RM_ZK_TIMEOUT_MS);
 
     List<ACL> zkAcls = RMZKUtils.getZKAcls(conf);
@@ -123,6 +130,8 @@ public class EmbeddedElectorService extends AbstractService
 
   @Override
   public void becomeActive() throws ServiceFailedException {
+    cancelDisconnectTimer();
+
     try {
       rmContext.getRMAdminService().transitionToActive(req);
     } catch (Exception e) {
@@ -132,6 +141,8 @@ public class EmbeddedElectorService extends AbstractService
 
   @Override
   public void becomeStandby() {
+    cancelDisconnectTimer();
+
     try {
       rmContext.getRMAdminService().transitionToStandby(req);
     } catch (Exception e) {
@@ -139,13 +150,49 @@ public class EmbeddedElectorService extends AbstractService
     }
   }
 
+  /**
+   * Stop the disconnect timer.  Any running tasks will be allowed to complete.
+   */
+  private void cancelDisconnectTimer() {
+    synchronized (zkDisconnectLock) {
+      if (zkDisconnectTimer != null) {
+        zkDisconnectTimer.cancel();
+        zkDisconnectTimer = null;
+      }
+    }
+  }
+
+  /**
+   * When the ZK client loses contact with ZK, this method will be called to
+   * allow the RM to react. Because the loss of connection can be noticed
+   * before the session timeout happens, it is undesirable to transition
+   * immediately. Instead the method starts a timer that will wait
+   * {@link YarnConfiguration#RM_ZK_TIMEOUT_MS} milliseconds before
+   * initiating the transition into standby state.
+   */
   @Override
   public void enterNeutralMode() {
-    /**
-     * Possibly due to transient connection issues. Do nothing.
-     * TODO: Might want to keep track of how long in this state and transition
-     * to standby.
-     */
+    LOG.warn("Lost contact with Zookeeper. Transitioning to standby in "
+        + zkSessionTimeout + " ms if connection is not reestablished.");
+
+    // If we've just become disconnected, start a timer.  When the time's up,
+    // we'll transition to standby.
+    synchronized (zkDisconnectLock) {
+      if (zkDisconnectTimer == null) {
+        zkDisconnectTimer = new Timer("Zookeeper disconnect timer");
+        zkDisconnectTimer.schedule(new TimerTask() {
+          @Override
+          public void run() {
+            synchronized (zkDisconnectLock) {
+              // Only run if the timer hasn't been cancelled
+              if (zkDisconnectTimer != null) {
+                becomeStandby();
+              }
+            }
+          }
+        }, zkSessionTimeout);
+      }
+    }
   }
 
   @SuppressWarnings(value = "unchecked")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6476934a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java
index 20b1c0e..bfd0b4e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java
@@ -28,6 +28,14 @@ import org.junit.Test;
 
 import java.io.IOException;
 import java.util.concurrent.atomic.AtomicBoolean;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.atLeast;
+import static org.mockito.Mockito.atMost;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 public class TestRMEmbeddedElector extends ClientBaseWithFixes {
   private static final Log LOG =
@@ -41,6 +49,14 @@ public class TestRMEmbeddedElector extends ClientBaseWithFixes {
   private Configuration conf;
   private AtomicBoolean callbackCalled;
 
+  private enum SyncTestType {
+    ACTIVE,
+    STANDBY,
+    NEUTRAL,
+    ACTIVE_TIMING,
+    STANDBY_TIMING
+  }
+
   @Before
   public void setup() throws IOException {
     conf = new YarnConfiguration();
@@ -79,6 +95,181 @@ public class TestRMEmbeddedElector extends ClientBaseWithFixes {
     LOG.info("Stopped RM");
   }
 
+  /**
+   * Test that neutral mode plays well with all other transitions.
+   *
+   * @throws IOException if there's an issue transitioning
+   * @throws InterruptedException if interrupted
+   */
+  @Test
+  public void testCallbackSynchronization()
+      throws IOException, InterruptedException {
+    testCallbackSynchronization(SyncTestType.ACTIVE);
+    testCallbackSynchronization(SyncTestType.STANDBY);
+    testCallbackSynchronization(SyncTestType.NEUTRAL);
+    testCallbackSynchronization(SyncTestType.ACTIVE_TIMING);
+    testCallbackSynchronization(SyncTestType.STANDBY_TIMING);
+  }
+
+  /**
+   * Helper method to test that neutral mode plays well with other transitions.
+   *
+   * @param type the type of test to run
+   * @throws IOException if there's an issue transitioning
+   * @throws InterruptedException if interrupted
+   */
+  private void testCallbackSynchronization(SyncTestType type)
+      throws IOException, InterruptedException {
+    AdminService as = mock(AdminService.class);
+    RMContext rc = mock(RMContext.class);
+    Configuration myConf = new Configuration(conf);
+
+    myConf.setInt(YarnConfiguration.RM_ZK_TIMEOUT_MS, 50);
+    when(rc.getRMAdminService()).thenReturn(as);
+
+    EmbeddedElectorService ees = new EmbeddedElectorService(rc);
+    ees.init(myConf);
+
+    ees.enterNeutralMode();
+
+    switch (type) {
+    case ACTIVE:
+      testCallbackSynchronizationActive(as, ees);
+      break;
+    case STANDBY:
+      testCallbackSynchronizationStandby(as, ees);
+      break;
+    case NEUTRAL:
+      testCallbackSynchronizationNeutral(as, ees);
+      break;
+    case ACTIVE_TIMING:
+      testCallbackSynchronizationTimingActive(as, ees);
+      break;
+    case STANDBY_TIMING:
+      testCallbackSynchronizationTimingStandby(as, ees);
+      break;
+    default:
+      fail("Unknown test type: " + type);
+      break;
+    }
+  }
+
+  /**
+   * Helper method to test that neutral mode plays well with an active
+   * transition.
+   *
+   * @param as the admin service
+   * @param ees the embedded elector service
+   * @throws IOException if there's an issue transitioning
+   * @throws InterruptedException if interrupted
+   */
+  private void testCallbackSynchronizationActive(AdminService as,
+      EmbeddedElectorService ees) throws IOException, InterruptedException {
+    ees.becomeActive();
+
+    Thread.sleep(100);
+
+    verify(as).transitionToActive(any());
+    verify(as, never()).transitionToStandby(any());
+  }
+
+  /**
+   * Helper method to test that neutral mode plays well with a standby
+   * transition.
+   *
+   * @param as the admin service
+   * @param ees the embedded elector service
+   * @throws IOException if there's an issue transitioning
+   * @throws InterruptedException if interrupted
+   */
+  private void testCallbackSynchronizationStandby(AdminService as,
+      EmbeddedElectorService ees) throws IOException, InterruptedException {
+    ees.becomeStandby();
+
+    Thread.sleep(100);
+
+    verify(as, atLeast(1)).transitionToStandby(any());
+    verify(as, atMost(1)).transitionToStandby(any());
+  }
+
+  /**
+   * Helper method to test that neutral mode plays well with itself.
+   *
+   * @param as the admin service
+   * @param ees the embedded elector service
+   * @throws IOException if there's an issue transitioning
+   * @throws InterruptedException if interrupted
+   */
+  private void testCallbackSynchronizationNeutral(AdminService as,
+      EmbeddedElectorService ees) throws IOException, InterruptedException {
+    ees.enterNeutralMode();
+
+    Thread.sleep(100);
+
+    verify(as, atLeast(1)).transitionToStandby(any());
+    verify(as, atMost(1)).transitionToStandby(any());
+  }
+
+  /**
+   * Helper method to test that neutral mode does not race with an active
+   * transition.
+   *
+   * @param as the admin service
+   * @param ees the embedded elector service
+   * @throws IOException if there's an issue transitioning
+   * @throws InterruptedException if interrupted
+   */
+  private void testCallbackSynchronizationTimingActive(AdminService as,
+      EmbeddedElectorService ees) throws IOException, InterruptedException {
+    synchronized (ees.zkDisconnectLock) {
+      // Sleep while holding the lock so that the timer thread can't do
+      // anything when it runs.  Sleep until we're pretty sure the timer thread
+      // has tried to run.
+      Thread.sleep(100);
+      // While still holding the lock cancel the timer by transitioning. This
+      // simulates a race where the callback goes to cancel the timer while the
+      // timer is trying to run.
+      ees.becomeActive();
+    }
+
+    // Sleep just a little more so that the timer thread can do whatever it's
+    // going to do, hopefully nothing.
+    Thread.sleep(50);
+
+    verify(as).transitionToActive(any());
+    verify(as, never()).transitionToStandby(any());
+  }
+
+  /**
+   * Helper method to test that neutral mode does not race with an active
+   * transition.
+   *
+   * @param as the admin service
+   * @param ees the embedded elector service
+   * @throws IOException if there's an issue transitioning
+   * @throws InterruptedException if interrupted
+   */
+  private void testCallbackSynchronizationTimingStandby(AdminService as,
+      EmbeddedElectorService ees) throws IOException, InterruptedException {
+    synchronized (ees.zkDisconnectLock) {
+      // Sleep while holding the lock so that the timer thread can't do
+      // anything when it runs.  Sleep until we're pretty sure the timer thread
+      // has tried to run.
+      Thread.sleep(100);
+      // While still holding the lock cancel the timer by transitioning. This
+      // simulates a race where the callback goes to cancel the timer while the
+      // timer is trying to run.
+      ees.becomeStandby();
+    }
+
+    // Sleep just a little more so that the timer thread can do whatever it's
+    // going to do, hopefully nothing.
+    Thread.sleep(50);
+
+    verify(as, atLeast(1)).transitionToStandby(any());
+    verify(as, atMost(1)).transitionToStandby(any());
+  }
+
   private class MockRMWithElector extends MockRM {
     private long delayMs = 0;
 


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


[08/50] hadoop git commit: HADOOP-13698. Document caveat for KeyShell when underlying KeyProvider does not delete a key.

Posted by um...@apache.org.
HADOOP-13698. Document caveat for KeyShell when underlying KeyProvider does not delete a key.


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

Branch: refs/heads/HDFS-10285
Commit: b84c4891f9eca8d56593e48e9df88be42e24220d
Parents: 3c9a010
Author: Xiao Chen <xi...@apache.org>
Authored: Tue Oct 11 17:05:00 2016 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Tue Oct 11 17:05:00 2016 -0700

----------------------------------------------------------------------
 .../hadoop-common/src/site/markdown/CommandsManual.md            | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b84c4891/hadoop-common-project/hadoop-common/src/site/markdown/CommandsManual.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/CommandsManual.md b/hadoop-common-project/hadoop-common/src/site/markdown/CommandsManual.md
index 4d7d504..2ece71a 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/CommandsManual.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/CommandsManual.md
@@ -202,7 +202,9 @@ Manage keys via the KeyProvider. For details on KeyProviders, see the [Transpare
 
 Providers frequently require that a password or other secret is supplied. If the provider requires a password and is unable to find one, it will use a default password and emit a warning message that the default password is being used. If the `-strict` flag is supplied, the warning message becomes an error message and the command returns immediately with an error status.
 
-NOTE: Some KeyProviders (e.g. org.apache.hadoop.crypto.key.JavaKeyStoreProvider) does not support uppercase key names.
+NOTE: Some KeyProviders (e.g. org.apache.hadoop.crypto.key.JavaKeyStoreProvider) do not support uppercase key names.
+
+NOTE: Some KeyProviders do not directly execute a key deletion (e.g. performs a soft-delete instead, or delay the actual deletion, to prevent mistake). In these cases, one may encounter errors when creating/deleting a key with the same name after deleting it. Please check the underlying KeyProvider for details.
 
 ### `trace`
 


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


[33/50] hadoop git commit: HDFS-10960. TestDataNodeHotSwapVolumes#testRemoveVolumeBeingWritten fails at disk error verification after volume remove. (Manoj Govindassamy via lei)

Posted by um...@apache.org.
HDFS-10960. TestDataNodeHotSwapVolumes#testRemoveVolumeBeingWritten fails at disk error verification after volume remove. (Manoj Govindassamy via lei)


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

Branch: refs/heads/HDFS-10285
Commit: 8c520a27cbd9daba05367d3a83017a2eab5258eb
Parents: adb96e1
Author: Lei Xu <le...@apache.org>
Authored: Fri Oct 14 13:41:59 2016 -0700
Committer: Lei Xu <le...@apache.org>
Committed: Fri Oct 14 13:41:59 2016 -0700

----------------------------------------------------------------------
 .../datanode/TestDataNodeHotSwapVolumes.java     | 19 ++++++++++++-------
 1 file changed, 12 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c520a27/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
index 06387c5..83c231d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
@@ -642,8 +642,6 @@ public class TestDataNodeHotSwapVolumes {
     final DataNode dn = cluster.getDataNodes().get(dataNodeIdx);
     final FileSystem fs = cluster.getFileSystem();
     final Path testFile = new Path("/test");
-    final long lastTimeDiskErrorCheck = dn.getLastDiskErrorCheck();
-
     FSDataOutputStream out = fs.create(testFile, REPLICATION);
 
     Random rb = new Random(0);
@@ -699,17 +697,24 @@ public class TestDataNodeHotSwapVolumes {
 
     reconfigThread.join();
 
+    // Verify if the data directory reconfigure was successful
+    FsDatasetSpi<? extends FsVolumeSpi> fsDatasetSpi = dn.getFSDataset();
+    try (FsDatasetSpi.FsVolumeReferences fsVolumeReferences = fsDatasetSpi
+        .getFsVolumeReferences()) {
+      for (int i =0; i < fsVolumeReferences.size(); i++) {
+        System.out.println("Vol: " +
+            fsVolumeReferences.get(i).getBaseURI().toString());
+      }
+      assertEquals("Volume remove wasn't successful.",
+          1, fsVolumeReferences.size());
+    }
+
     // Verify the file has sufficient replications.
     DFSTestUtil.waitReplication(fs, testFile, REPLICATION);
     // Read the content back
     byte[] content = DFSTestUtil.readFileBuffer(fs, testFile);
     assertEquals(BLOCK_SIZE, content.length);
 
-    // If an IOException thrown from BlockReceiver#run, it triggers
-    // DataNode#checkDiskError(). So we can test whether checkDiskError() is called,
-    // to see whether there is IOException in BlockReceiver#run().
-    assertEquals(lastTimeDiskErrorCheck, dn.getLastDiskErrorCheck());
-
     if (!exceptions.isEmpty()) {
       throw new IOException(exceptions.get(0).getCause());
     }


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


[06/50] hadoop git commit: HDFS-10984. Expose nntop output as metrics. Contributed by Siddharth Wagle.

Posted by um...@apache.org.
HDFS-10984. Expose nntop output as metrics. Contributed by Siddharth Wagle.


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

Branch: refs/heads/HDFS-10285
Commit: 61f0490a73085bbaf6639d9234277e59dc1145db
Parents: dacd3ec
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Tue Oct 11 15:55:02 2016 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Tue Oct 11 15:55:02 2016 -0700

----------------------------------------------------------------------
 .../hdfs/server/namenode/FSNamesystem.java      |  6 ++
 .../server/namenode/top/metrics/TopMetrics.java | 67 ++++++++++++++++++--
 .../server/namenode/metrics/TestTopMetrics.java | 63 ++++++++++++++++++
 3 files changed, 129 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/61f0490a/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 2471dc8..b9b02ef 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
@@ -89,6 +89,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
 import static org.apache.hadoop.hdfs.server.namenode.FSDirStatAndListingOp.*;
 import static org.apache.hadoop.util.Time.now;
 import static org.apache.hadoop.util.Time.monotonicNow;
+import static org.apache.hadoop.hdfs.server.namenode.top.metrics.TopMetrics.TOPMETRICS_METRICS_SOURCE_NAME;
 
 import java.io.BufferedWriter;
 import java.io.ByteArrayInputStream;
@@ -989,6 +990,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     // Add audit logger to calculate top users
     if (topConf.isEnabled) {
       topMetrics = new TopMetrics(conf, topConf.nntopReportingPeriodsMs);
+      if (DefaultMetricsSystem.instance().getSource(
+          TOPMETRICS_METRICS_SOURCE_NAME) == null) {
+        DefaultMetricsSystem.instance().register(TOPMETRICS_METRICS_SOURCE_NAME,
+            "Top N operations by user", topMetrics);
+      }
       auditLoggers.add(new TopAuditLogger(topMetrics));
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/61f0490a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/top/metrics/TopMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/top/metrics/TopMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/top/metrics/TopMetrics.java
index ab55392..2719c88 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/top/metrics/TopMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/top/metrics/TopMetrics.java
@@ -17,24 +17,32 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.top.metrics;
 
-import java.net.InetAddress;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
 import com.google.common.collect.Lists;
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.namenode.top.TopConf;
 import org.apache.hadoop.hdfs.server.namenode.top.window.RollingWindowManager;
+import org.apache.hadoop.hdfs.server.namenode.top.window.RollingWindowManager.Op;
+import org.apache.hadoop.hdfs.server.namenode.top.window.RollingWindowManager.User;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsInfo;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.lib.Interns;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.net.InetAddress;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
 import static org.apache.hadoop.hdfs.server.namenode.top.window.RollingWindowManager.TopWindow;
 
 /**
@@ -58,8 +66,11 @@ import static org.apache.hadoop.hdfs.server.namenode.top.window.RollingWindowMan
  * Thread-safe: relies on thread-safety of RollingWindowManager
  */
 @InterfaceAudience.Private
-public class TopMetrics {
+public class TopMetrics implements MetricsSource {
   public static final Logger LOG = LoggerFactory.getLogger(TopMetrics.class);
+  public static final String TOPMETRICS_METRICS_SOURCE_NAME =
+      "NNTopUserOpCounts";
+  private final boolean isMetricsSourceEnabled;
 
   private static void logConf(Configuration conf) {
     LOG.info("NNTop conf: " + DFSConfigKeys.NNTOP_BUCKETS_PER_WINDOW_KEY +
@@ -83,6 +94,8 @@ public class TopMetrics {
       rollingWindowManagers.put(reportingPeriods[i], new RollingWindowManager(
           conf, reportingPeriods[i]));
     }
+    isMetricsSourceEnabled = conf.getBoolean(DFSConfigKeys.NNTOP_ENABLED_KEY,
+        DFSConfigKeys.NNTOP_ENABLED_DEFAULT);
   }
 
   /**
@@ -128,4 +141,44 @@ public class TopMetrics {
           TopConf.ALL_CMDS, userName, 1);
     }
   }
+
+  /**
+   * Flatten out the top window metrics into
+   * {@link org.apache.hadoop.metrics2.MetricsRecord}s for consumption by
+   * external metrics systems. Each metrics record added corresponds to the
+   * reporting period a.k.a window length of the configured rolling windows.
+   */
+  @Override
+  public void getMetrics(MetricsCollector collector, boolean all) {
+    if (!isMetricsSourceEnabled) {
+      return;
+    }
+
+    for (final TopWindow window : getTopWindows()) {
+      MetricsRecordBuilder rb = collector.addRecord(buildOpRecordName(window))
+          .setContext("dfs");
+      for (final Op op: window.getOps()) {
+        rb.addCounter(buildOpTotalCountMetricsInfo(op), op.getTotalCount());
+        for (User user : op.getTopUsers()) {
+          rb.addCounter(buildOpRecordMetricsInfo(op, user), user.getCount());
+        }
+      }
+    }
+  }
+
+  private String buildOpRecordName(TopWindow window) {
+    return TOPMETRICS_METRICS_SOURCE_NAME + ".windowMs="
+      + window.getWindowLenMs();
+  }
+
+  private MetricsInfo buildOpTotalCountMetricsInfo(Op op) {
+    return Interns.info("op=" + StringUtils.deleteWhitespace(op.getOpType())
+      + ".TotalCount", "Total operation count");
+  }
+
+  private MetricsInfo buildOpRecordMetricsInfo(Op op, User user) {
+    return Interns.info("op=" + StringUtils.deleteWhitespace(op.getOpType())
+      + ".user=" + user.getUser()
+      + ".count", "Total operations performed by user");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/61f0490a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestTopMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestTopMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestTopMetrics.java
new file mode 100644
index 0000000..4d3a4f0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestTopMetrics.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.metrics;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.namenode.top.TopConf;
+import org.apache.hadoop.hdfs.server.namenode.top.metrics.TopMetrics;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.lib.Interns;
+import org.junit.Test;
+
+import static org.apache.hadoop.hdfs.server.namenode.top.metrics.TopMetrics.TOPMETRICS_METRICS_SOURCE_NAME;
+import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Test for MetricsSource part of the {@link TopMetrics} impl.
+ */
+public class TestTopMetrics {
+  @Test
+  public void testPresence() {
+    Configuration conf = new Configuration();
+    TopConf topConf = new TopConf(conf);
+    TopMetrics topMetrics = new TopMetrics(conf,
+        topConf.nntopReportingPeriodsMs);
+    // Dummy command
+    topMetrics.report("test", "listStatus");
+    topMetrics.report("test", "listStatus");
+    topMetrics.report("test", "listStatus");
+
+    MetricsRecordBuilder rb = getMetrics(topMetrics);
+    MetricsCollector mc = rb.parent();
+
+    verify(mc).addRecord(TOPMETRICS_METRICS_SOURCE_NAME + ".windowMs=60000");
+    verify(mc).addRecord(TOPMETRICS_METRICS_SOURCE_NAME + ".windowMs=300000");
+    verify(mc).addRecord(TOPMETRICS_METRICS_SOURCE_NAME + ".windowMs=1500000");
+
+    verify(rb, times(3)).addCounter(Interns.info("op=listStatus.TotalCount",
+        "Total operation count"), 3L);
+    verify(rb, times(3)).addCounter(Interns.info("op=*.TotalCount",
+        "Total operation count"), 3L);
+
+    verify(rb, times(3)).addCounter(Interns.info("op=listStatus." +
+        "user=test.count", "Total operations performed by user"), 3L);
+  }
+}


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


[17/50] hadoop git commit: HDFS-10949. DiskBalancer: deprecate TestDiskBalancer#setVolumeCapacity. Contributed by Xiaobing Zhou.

Posted by um...@apache.org.
HDFS-10949. DiskBalancer: deprecate TestDiskBalancer#setVolumeCapacity. Contributed by Xiaobing Zhou.


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

Branch: refs/heads/HDFS-10285
Commit: b371c56365c14bbab0f5cdfffc0becaabfde8145
Parents: 1291254
Author: Anu Engineer <ae...@apache.org>
Authored: Thu Oct 13 10:26:07 2016 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Thu Oct 13 10:26:07 2016 -0700

----------------------------------------------------------------------
 .../server/diskbalancer/TestDiskBalancer.java   | 44 +++++---------------
 1 file changed, 11 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b371c563/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java
index d911e74..9985210 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java
@@ -44,7 +44,6 @@ import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
-import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
@@ -137,6 +136,7 @@ public class TestDiskBalancer {
     final int dataNodeCount = 1;
     final int dataNodeIndex = 0;
     final int sourceDiskIndex = 0;
+    final long cap = blockSize * 2L * blockCount;
 
     MiniDFSCluster cluster = new ClusterBuilder()
         .setBlockCount(blockCount)
@@ -144,6 +144,7 @@ public class TestDiskBalancer {
         .setDiskCount(diskCount)
         .setNumDatanodes(dataNodeCount)
         .setConf(conf)
+        .setCapacities(new long[] {cap, cap})
         .build();
     try {
       DataMover dataMover = new DataMover(cluster, dataNodeIndex,
@@ -174,7 +175,7 @@ public class TestDiskBalancer {
     final int dataNodeCount = 1;
     final int dataNodeIndex = 0;
     final int sourceDiskIndex = 0;
-
+    final long cap = blockSize * 2L * blockCount;
 
     MiniDFSCluster cluster = new ClusterBuilder()
         .setBlockCount(blockCount)
@@ -182,9 +183,9 @@ public class TestDiskBalancer {
         .setDiskCount(diskCount)
         .setNumDatanodes(dataNodeCount)
         .setConf(conf)
+        .setCapacities(new long[] {cap, cap, cap})
         .build();
 
-
     try {
       DataMover dataMover = new DataMover(cluster, dataNodeIndex,
           sourceDiskIndex, conf, blockSize, blockCount);
@@ -221,6 +222,7 @@ public class TestDiskBalancer {
     final int dataNodeCount = 1;
     final int dataNodeIndex = 0;
     final int sourceDiskIndex = 0;
+    final long cap = blockSize * 2L * blockCount;
 
     MiniDFSCluster cluster = new ClusterBuilder()
         .setBlockCount(blockCount)
@@ -228,6 +230,7 @@ public class TestDiskBalancer {
         .setDiskCount(diskCount)
         .setNumDatanodes(dataNodeCount)
         .setConf(conf)
+        .setCapacities(new long[] {cap, cap})
         .build();
 
     try {
@@ -246,24 +249,6 @@ public class TestDiskBalancer {
   }
 
   /**
-   * Sets alll Disks capacity to size specified.
-   *
-   * @param cluster - DiskBalancerCluster
-   * @param size    - new size of the disk
-   */
-  private void setVolumeCapacity(DiskBalancerCluster cluster, long size,
-                                 String diskType) {
-    Preconditions.checkNotNull(cluster);
-    for (DiskBalancerDataNode node : cluster.getNodes()) {
-      for (DiskBalancerVolume vol :
-          node.getVolumeSets().get(diskType).getVolumes()) {
-        vol.setCapacity(size);
-      }
-      node.getVolumeSets().get(diskType).computeVolumeDataDensity();
-    }
-  }
-
-  /**
    * Helper class that allows us to create different kinds of MiniDFSClusters
    * and populate data.
    */
@@ -274,6 +259,7 @@ public class TestDiskBalancer {
     private int fileLen;
     private int blockCount;
     private int diskCount;
+    private long[] capacities;
 
     public ClusterBuilder setConf(Configuration conf) {
       this.conf = conf;
@@ -300,13 +286,9 @@ public class TestDiskBalancer {
       return this;
     }
 
-    private long[] getCapacities(int diskCount, int bSize, int fSize) {
-      Preconditions.checkState(diskCount > 0);
-      long[] capacities = new long[diskCount];
-      for (int x = 0; x < diskCount; x++) {
-        capacities[x] = diskCount * bSize * fSize * 2L;
-      }
-      return capacities;
+    private ClusterBuilder setCapacities(final long[] caps) {
+      this.capacities = caps;
+      return this;
     }
 
     private StorageType[] getStorageTypes(int diskCount) {
@@ -338,7 +320,7 @@ public class TestDiskBalancer {
       // Write a file and restart the cluster
       MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
           .numDataNodes(numDatanodes)
-          .storageCapacities(getCapacities(diskCount, blockSize, fileLen))
+          .storageCapacities(capacities)
           .storageTypes(getStorageTypes(diskCount))
           .storagesPerDatanode(diskCount)
           .build();
@@ -447,10 +429,6 @@ public class TestDiskBalancer {
       diskBalancerCluster.readClusterInfo();
       List<DiskBalancerDataNode> nodesToProcess = new LinkedList<>();
 
-      // Rewrite the capacity in the model to show that disks need
-      // re-balancing.
-      setVolumeCapacity(diskBalancerCluster, blockSize * 2L * blockCount,
-          "DISK");
       // Pick a node to process.
       nodesToProcess.add(diskBalancerCluster.getNodeByUUID(
           node.getDatanodeUuid()));


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


[42/50] hadoop git commit: HDFS-9820. Improve distcp to support efficient restore to an earlier snapshot. Contributed by Yongjun Zhang.

Posted by um...@apache.org.
HDFS-9820. Improve distcp to support efficient restore to an earlier snapshot. Contributed by Yongjun Zhang.


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

Branch: refs/heads/HDFS-10285
Commit: 412c4c9a342b73bf1c1a7f43ea91245cbf94d02d
Parents: ed9fcbe
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Fri Oct 14 15:17:33 2016 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Mon Oct 17 11:04:42 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/tools/DiffInfo.java  |  47 +-
 .../java/org/apache/hadoop/tools/DistCp.java    |  34 +-
 .../apache/hadoop/tools/DistCpConstants.java    |   1 +
 .../apache/hadoop/tools/DistCpOptionSwitch.java |   5 +
 .../org/apache/hadoop/tools/DistCpOptions.java  |  79 +-
 .../org/apache/hadoop/tools/DistCpSync.java     | 256 ++++--
 .../org/apache/hadoop/tools/OptionsParser.java  |  27 +-
 .../apache/hadoop/tools/SimpleCopyListing.java  |  17 +-
 .../org/apache/hadoop/tools/TestDistCpSync.java |   4 +-
 .../hadoop/tools/TestDistCpSyncReverseBase.java | 868 +++++++++++++++++++
 .../tools/TestDistCpSyncReverseFromSource.java  |  36 +
 .../tools/TestDistCpSyncReverseFromTarget.java  |  36 +
 .../apache/hadoop/tools/TestOptionsParser.java  |  85 +-
 13 files changed, 1340 insertions(+), 155 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/412c4c9a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DiffInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DiffInfo.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DiffInfo.java
index 79bb7fe..7e56301 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DiffInfo.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DiffInfo.java
@@ -44,28 +44,49 @@ class DiffInfo {
   };
 
   /** The source file/dir of the rename or deletion op */
-  final Path source;
+  private Path source;
+  /** The target file/dir of the rename op. Null means the op is deletion. */
+  private Path target;
+
+  private SnapshotDiffReport.DiffType type;
   /**
    * The intermediate file/dir for the op. For a rename or a delete op,
    * we first rename the source to this tmp file/dir.
    */
   private Path tmp;
-  /** The target file/dir of the rename op. Null means the op is deletion. */
-  Path target;
-
-  private final SnapshotDiffReport.DiffType type;
-
-  public SnapshotDiffReport.DiffType getType(){
-    return this.type;
-  }
 
-  DiffInfo(Path source, Path target, SnapshotDiffReport.DiffType type) {
+  DiffInfo(final Path source, final Path target,
+      SnapshotDiffReport.DiffType type) {
     assert source != null;
     this.source = source;
     this.target= target;
     this.type = type;
   }
 
+  void setSource(final Path source) {
+    this.source = source;
+  }
+
+  Path getSource() {
+    return source;
+  }
+
+  void setTarget(final Path target) {
+    this.target = target;
+  }
+
+  Path getTarget() {
+    return target;
+  }
+
+  public void setType(final SnapshotDiffReport.DiffType type){
+    this.type = type;
+  }
+
+  public SnapshotDiffReport.DiffType getType(){
+    return type;
+  }
+
   void setTmp(Path tmp) {
     this.tmp = tmp;
   }
@@ -73,4 +94,10 @@ class DiffInfo {
   Path getTmp() {
     return tmp;
   }
+
+  @Override
+  public String toString() {
+    return type + ": src=" + String.valueOf(source) + " tgt="
+        + String.valueOf(target) + " tmp=" + String.valueOf(tmp);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/412c4c9a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java
index be58f13..e9decd2 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java
@@ -77,6 +77,21 @@ public class DistCp extends Configured implements Tool {
   private boolean submitted;
   private FileSystem jobFS;
 
+  private void prepareFileListing(Job job) throws Exception {
+    if (inputOptions.shouldUseSnapshotDiff()) {
+      try {
+        DistCpSync distCpSync = new DistCpSync(inputOptions, getConf());
+        distCpSync.sync();
+        createInputFileListingWithDiff(job, distCpSync);
+      } catch (IOException e) {
+        throw new Exception("DistCp"
+            + " sync failed, input options: " + inputOptions, e);
+      }
+    } else {
+      createInputFileListing(job);
+    }
+  }
+
   /**
    * Public Constructor. Creates DistCp object with specified input-parameters.
    * (E.g. source-paths, target-location, etc.)
@@ -176,21 +191,7 @@ public class DistCp extends Configured implements Tool {
         jobFS = metaFolder.getFileSystem(getConf());
         job = createJob();
       }
-      if (inputOptions.shouldUseDiff()) {
-        DistCpSync distCpSync = new DistCpSync(inputOptions, getConf());
-        if (distCpSync.sync()) {
-          createInputFileListingWithDiff(job, distCpSync);
-        } else {
-          throw new Exception("DistCp sync failed, input options: "
-              + inputOptions);
-        }
-      }
-
-      // Fallback to default DistCp if without "diff" option or sync failed.
-      if (!inputOptions.shouldUseDiff()) {
-        createInputFileListing(job);
-      }
-
+      prepareFileListing(job);
       job.submit();
       submitted = true;
     } finally {
@@ -200,7 +201,8 @@ public class DistCp extends Configured implements Tool {
     }
 
     String jobID = job.getJobID().toString();
-    job.getConfiguration().set(DistCpConstants.CONF_LABEL_DISTCP_JOB_ID, jobID);
+    job.getConfiguration().set(DistCpConstants.CONF_LABEL_DISTCP_JOB_ID,
+        jobID);
     LOG.info("DistCp job-id: " + jobID);
 
     return job;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/412c4c9a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
index 6171aa9..ff16e44 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
@@ -59,6 +59,7 @@ public class DistCpConstants {
   public static final String CONF_LABEL_OVERWRITE = "distcp.copy.overwrite";
   public static final String CONF_LABEL_APPEND = "distcp.copy.append";
   public static final String CONF_LABEL_DIFF = "distcp.copy.diff";
+  public static final String CONF_LABEL_RDIFF = "distcp.copy.rdiff";
   public static final String CONF_LABEL_BANDWIDTH_MB = "distcp.map.bandwidth.mb";
   public static final String CONF_LABEL_SIMPLE_LISTING_FILESTATUS_SIZE =
       "distcp.simplelisting.file.status.size";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/412c4c9a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
index c104896..fb47d76 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
@@ -150,6 +150,11 @@ public enum DistCpOptionSwitch {
       "Use snapshot diff report to identify the difference between source and target"),
       2),
 
+  RDIFF(DistCpConstants.CONF_LABEL_RDIFF,
+      new Option("rdiff", false,
+      "Use target snapshot diff report to identify changes made on target"),
+      2),
+
   /**
    * Should DisctpExecution be blocking
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/412c4c9a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
index 4c5518f..8c37ff3 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.tools;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.tools.util.DistCpUtils;
@@ -42,8 +43,29 @@ public class DistCpOptions {
   private boolean append = false;
   private boolean skipCRC = false;
   private boolean blocking = true;
+  // When "-diff s1 s2 src tgt" is passed, apply forward snapshot diff (from s1
+  // to s2) of source cluster to the target cluster to sync target cluster with
+  // the source cluster. Referred to as "Fdiff" in the code.
+  // It's required that s2 is newer than s1.
   private boolean useDiff = false;
 
+  // When "-rdiff s2 s1 src tgt" is passed, apply reversed snapshot diff (from
+  // s2 to s1) of target cluster to the target cluster, so to make target
+  // cluster go back to s1. Referred to as "Rdiff" in the code.
+  // It's required that s2 is newer than s1, and src and tgt have exact same
+  // content at their s1, if src is not the same as tgt.
+  private boolean useRdiff = false;
+
+  // For both -diff and -rdiff, given the example command line switches, two
+  // steps are taken:
+  //   1. Sync Step. This step does renaming/deletion ops in the snapshot diff,
+  //      so to avoid copying files copied already but renamed later(HDFS-7535)
+  //   2. Copy Step. This step copy the necessary files from src to tgt
+  //      2.1 For -diff, it copies from snapshot s2 of src (HDFS-8828)
+  //      2.2 For -rdiff, it copies from snapshot s1 of src, where the src
+  //          could be the tgt itself (HDFS-9820).
+  //
+
   public static final int maxNumListstatusThreads = 40;
   private int numListstatusThreads = 0;  // Indicates that flag is not set.
   private int maxMaps = DistCpConstants.DEFAULT_MAPS;
@@ -129,6 +151,8 @@ public class DistCpOptions {
       this.overwrite = that.overwrite;
       this.skipCRC = that.skipCRC;
       this.blocking = that.blocking;
+      this.useDiff = that.useDiff;
+      this.useRdiff = that.useRdiff;
       this.numListstatusThreads = that.numListstatusThreads;
       this.maxMaps = that.maxMaps;
       this.mapBandwidth = that.mapBandwidth;
@@ -273,6 +297,14 @@ public class DistCpOptions {
     return this.useDiff;
   }
 
+  public boolean shouldUseRdiff() {
+    return this.useRdiff;
+  }
+
+  public boolean shouldUseSnapshotDiff() {
+    return shouldUseDiff() || shouldUseRdiff();
+  }
+
   public String getFromSnapshot() {
     return this.fromSnapshot;
   }
@@ -281,14 +313,16 @@ public class DistCpOptions {
     return this.toSnapshot;
   }
 
-  public void setUseDiff(boolean useDiff, String fromSnapshot, String toSnapshot) {
-    this.useDiff = useDiff;
-    this.fromSnapshot = fromSnapshot;
-    this.toSnapshot = toSnapshot;
+  public void setUseDiff(String fromSS, String toSS) {
+    this.useDiff = true;
+    this.fromSnapshot = fromSS;
+    this.toSnapshot = toSS;
   }
 
-  public void disableUsingDiff() {
-    this.useDiff = false;
+  public void setUseRdiff(String fromSS, String toSS) {
+    this.useRdiff = true;
+    this.fromSnapshot = fromSS;
+    this.toSnapshot = toSS;
   }
 
   /**
@@ -545,11 +579,12 @@ public class DistCpOptions {
   }
 
   void validate() {
-    if (useDiff && deleteMissing) {
-      // -delete and -diff are mutually exclusive. For backward compatibility,
-      // we ignore the -delete option here, instead of throwing an
-      // IllegalArgumentException. See HDFS-10397 for more discussion.
-      OptionsParser.LOG.warn("-delete and -diff are mutually exclusive. " +
+    if ((useDiff || useRdiff) && deleteMissing) {
+      // -delete and -diff/-rdiff are mutually exclusive. For backward
+      // compatibility, we ignore the -delete option here, instead of throwing
+      // an IllegalArgumentException. See HDFS-10397 for more discussion.
+      OptionsParser.LOG.warn(
+          "-delete and -diff/-rdiff are mutually exclusive. " +
           "The -delete option will be ignored.");
       setDeleteMissing(false);
     }
@@ -581,16 +616,29 @@ public class DistCpOptions {
       throw new IllegalArgumentException(
           "Append is disallowed when skipping CRC");
     }
-    if (!syncFolder && useDiff) {
+    if (!syncFolder && (useDiff || useRdiff)) {
+      throw new IllegalArgumentException(
+          "-diff/-rdiff is valid only with -update option");
+    }
+
+    if (useDiff || useRdiff) {
+      if (StringUtils.isBlank(fromSnapshot) ||
+          StringUtils.isBlank(toSnapshot)) {
+        throw new IllegalArgumentException(
+            "Must provide both the starting and ending " +
+            "snapshot names for -diff/-rdiff");
+      }
+    }
+    if (useDiff && useRdiff) {
       throw new IllegalArgumentException(
-          "Diff is valid only with update options");
+          "-diff and -rdiff are mutually exclusive");
     }
   }
 
   /**
    * Add options to configuration. These will be used in the Mapper/committer
    *
-   * @param conf - Configruation object to which the options need to be added
+   * @param conf - Configuration object to which the options need to be added
    */
   public void appendToConf(Configuration conf) {
     DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.ATOMIC_COMMIT,
@@ -607,6 +655,8 @@ public class DistCpOptions {
         String.valueOf(append));
     DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.DIFF,
         String.valueOf(useDiff));
+    DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.RDIFF,
+        String.valueOf(useRdiff));
     DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.SKIP_CRC,
         String.valueOf(skipCRC));
     if (mapBandwidth > 0) {
@@ -636,6 +686,7 @@ public class DistCpOptions {
         ", overwrite=" + overwrite +
         ", append=" + append +
         ", useDiff=" + useDiff +
+        ", useRdiff=" + useRdiff +
         ", fromSnapshot=" + fromSnapshot +
         ", toSnapshot=" + toSnapshot +
         ", skipCRC=" + skipCRC +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/412c4c9a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpSync.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpSync.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpSync.java
index 38a1bef..f1fae11 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpSync.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpSync.java
@@ -50,6 +50,11 @@ import java.util.HashSet;
 class DistCpSync {
   private DistCpOptions inputOptions;
   private Configuration conf;
+  // diffMap maps snapshot diff op type to a list of diff ops.
+  // It's initially created based on the snapshot diff. Then the individual
+  // diff stored there maybe modified instead of copied by the distcp algorithm
+  // afterwards, for better performance.
+  //
   private EnumMap<SnapshotDiffReport.DiffType, List<DiffInfo>> diffMap;
   private DiffInfo[] renameDiffs;
 
@@ -58,6 +63,10 @@ class DistCpSync {
     this.conf = conf;
   }
 
+  private boolean isRdiff() {
+    return inputOptions.shouldUseRdiff();
+  }
+
   /**
    * Check if three conditions are met before sync.
    * 1. Only one source directory.
@@ -77,21 +86,25 @@ class DistCpSync {
     final Path sourceDir = sourcePaths.get(0);
     final Path targetDir = inputOptions.getTargetPath();
 
-    final FileSystem sfs = sourceDir.getFileSystem(conf);
-    final FileSystem tfs = targetDir.getFileSystem(conf);
+    final FileSystem srcFs = sourceDir.getFileSystem(conf);
+    final FileSystem tgtFs = targetDir.getFileSystem(conf);
+    final FileSystem snapshotDiffFs = isRdiff() ? tgtFs : srcFs;
+    final Path snapshotDiffDir = isRdiff() ? targetDir : sourceDir;
+
     // currently we require both the source and the target file system are
     // DistributedFileSystem.
-    if (!(sfs instanceof DistributedFileSystem) ||
-        !(tfs instanceof DistributedFileSystem)) {
+    if (!(srcFs instanceof DistributedFileSystem) ||
+        !(tgtFs instanceof DistributedFileSystem)) {
       throw new IllegalArgumentException("The FileSystems needs to" +
           " be DistributedFileSystem for using snapshot-diff-based distcp");
     }
-    final DistributedFileSystem targetFs = (DistributedFileSystem) tfs;
+
+    final DistributedFileSystem targetFs = (DistributedFileSystem) tgtFs;
 
     // make sure targetFS has no change between from and the current states
     if (!checkNoChange(targetFs, targetDir)) {
       // set the source path using the snapshot path
-      inputOptions.setSourcePaths(Arrays.asList(getSourceSnapshotPath(sourceDir,
+      inputOptions.setSourcePaths(Arrays.asList(getSnapshotPath(sourceDir,
           inputOptions.getToSnapshot())));
       return false;
     }
@@ -101,17 +114,27 @@ class DistCpSync {
 
     try {
       final FileStatus fromSnapshotStat =
-          sfs.getFileStatus(getSourceSnapshotPath(sourceDir, from));
+          snapshotDiffFs.getFileStatus(getSnapshotPath(snapshotDiffDir, from));
 
       final FileStatus toSnapshotStat =
-          sfs.getFileStatus(getSourceSnapshotPath(sourceDir, to));
-
-      // If toSnapshot isn't current dir then do a time check
-      if (!to.equals("")
-          && fromSnapshotStat.getModificationTime() > toSnapshotStat
-              .getModificationTime()) {
-        throw new HadoopIllegalArgumentException("Snapshot " + to
-            + " should be newer than " + from);
+          snapshotDiffFs.getFileStatus(getSnapshotPath(snapshotDiffDir, to));
+
+      if (isRdiff()) {
+        // If fromSnapshot isn't current dir then do a time check
+        if (!from.equals("")
+            && fromSnapshotStat.getModificationTime() < toSnapshotStat
+            .getModificationTime()) {
+          throw new HadoopIllegalArgumentException("Snapshot " + from
+              + " should be newer than " + to);
+        }
+      } else {
+        // If toSnapshot isn't current dir then do a time check
+        if(!to.equals("")
+            && fromSnapshotStat.getModificationTime() > toSnapshotStat
+            .getModificationTime()) {
+          throw new HadoopIllegalArgumentException("Snapshot " + to
+              + " should be newer than " + from);
+        }
       }
     } catch (FileNotFoundException nfe) {
       throw new InvalidInputException("Input snapshot is not found", nfe);
@@ -138,7 +161,8 @@ class DistCpSync {
     Path tmpDir = null;
     try {
       tmpDir = createTargetTmpDir(targetFs, targetDir);
-      DiffInfo[] renameAndDeleteDiffs = getRenameAndDeleteDiffs(targetDir);
+      DiffInfo[] renameAndDeleteDiffs =
+          getRenameAndDeleteDiffsForSync(targetDir);
       if (renameAndDeleteDiffs.length > 0) {
         // do the real sync work: deletion and rename
         syncDiff(renameAndDeleteDiffs, targetFs, tmpDir);
@@ -151,7 +175,7 @@ class DistCpSync {
       deleteTargetTmpDir(targetFs, tmpDir);
       // TODO: since we have tmp directory, we can support "undo" with failures
       // set the source path using the snapshot path
-      inputOptions.setSourcePaths(Arrays.asList(getSourceSnapshotPath(sourceDir,
+      inputOptions.setSourcePaths(Arrays.asList(getSnapshotPath(sourceDir,
           inputOptions.getToSnapshot())));
     }
   }
@@ -162,16 +186,16 @@ class DistCpSync {
    * no entry for a given DiffType, the associated value will be an empty list.
    */
   private boolean getAllDiffs() throws IOException {
-    List<Path> sourcePaths = inputOptions.getSourcePaths();
-    final Path sourceDir = sourcePaths.get(0);
+    Path ssDir = isRdiff()?
+        inputOptions.getTargetPath() : inputOptions.getSourcePaths().get(0);
+
     try {
       DistributedFileSystem fs =
-          (DistributedFileSystem) sourceDir.getFileSystem(conf);
+          (DistributedFileSystem) ssDir.getFileSystem(conf);
       final String from = getSnapshotName(inputOptions.getFromSnapshot());
       final String to = getSnapshotName(inputOptions.getToSnapshot());
-      SnapshotDiffReport report = fs.getSnapshotDiffReport(sourceDir,
+      SnapshotDiffReport report = fs.getSnapshotDiffReport(ssDir,
           from, to);
-
       this.diffMap = new EnumMap<>(SnapshotDiffReport.DiffType.class);
       for (SnapshotDiffReport.DiffType type :
           SnapshotDiffReport.DiffType.values()) {
@@ -185,25 +209,25 @@ class DistCpSync {
         if (entry.getSourcePath().length <= 0) {
           continue;
         }
-        List<DiffInfo> list = diffMap.get(entry.getType());
-
-        if (entry.getType() == SnapshotDiffReport.DiffType.MODIFY ||
-            entry.getType() == SnapshotDiffReport.DiffType.CREATE ||
-            entry.getType() == SnapshotDiffReport.DiffType.DELETE) {
+        SnapshotDiffReport.DiffType dt = entry.getType();
+        List<DiffInfo> list = diffMap.get(dt);
+        if (dt == SnapshotDiffReport.DiffType.MODIFY ||
+            dt == SnapshotDiffReport.DiffType.CREATE ||
+            dt == SnapshotDiffReport.DiffType.DELETE) {
           final Path source =
               new Path(DFSUtilClient.bytes2String(entry.getSourcePath()));
-          list.add(new DiffInfo(source, null, entry.getType()));
-        } else if (entry.getType() == SnapshotDiffReport.DiffType.RENAME) {
+          list.add(new DiffInfo(source, null, dt));
+        } else if (dt == SnapshotDiffReport.DiffType.RENAME) {
           final Path source =
               new Path(DFSUtilClient.bytes2String(entry.getSourcePath()));
           final Path target =
               new Path(DFSUtilClient.bytes2String(entry.getTargetPath()));
-          list.add(new DiffInfo(source, target, entry.getType()));
+          list.add(new DiffInfo(source, target, dt));
         }
       }
       return true;
     } catch (IOException e) {
-      DistCp.LOG.warn("Failed to compute snapshot diff on " + sourceDir, e);
+      DistCp.LOG.warn("Failed to compute snapshot diff on " + ssDir, e);
     }
     this.diffMap = null;
     return false;
@@ -213,11 +237,11 @@ class DistCpSync {
     return Path.CUR_DIR.equals(name) ? "" : name;
   }
 
-  private Path getSourceSnapshotPath(Path sourceDir, String snapshotName) {
+  private Path getSnapshotPath(Path inputDir, String snapshotName) {
     if (Path.CUR_DIR.equals(snapshotName)) {
-      return sourceDir;
+      return inputDir;
     } else {
-      return new Path(sourceDir,
+      return new Path(inputDir,
           HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + snapshotName);
     }
   }
@@ -249,8 +273,9 @@ class DistCpSync {
    */
   private boolean checkNoChange(DistributedFileSystem fs, Path path) {
     try {
+      final String from = getSnapshotName(inputOptions.getFromSnapshot());
       SnapshotDiffReport targetDiff =
-          fs.getSnapshotDiffReport(path, inputOptions.getFromSnapshot(), "");
+          fs.getSnapshotDiffReport(path, from, "");
       if (!targetDiff.getDiffList().isEmpty()) {
         DistCp.LOG.warn("The target has been modified since snapshot "
             + inputOptions.getFromSnapshot());
@@ -259,7 +284,8 @@ class DistCpSync {
         return true;
       }
     } catch (IOException e) {
-      DistCp.LOG.warn("Failed to compute snapshot diff on " + path, e);
+      DistCp.LOG.warn("Failed to compute snapshot diff on " + path
+          + " at snapshot " + inputOptions.getFromSnapshot(), e);
     }
     return false;
   }
@@ -281,12 +307,13 @@ class DistCpSync {
     Arrays.sort(diffs, DiffInfo.sourceComparator);
     Random random = new Random();
     for (DiffInfo diff : diffs) {
-      Path tmpTarget = new Path(tmpDir, diff.source.getName());
+      Path tmpTarget = new Path(tmpDir, diff.getSource().getName());
       while (targetFs.exists(tmpTarget)) {
-        tmpTarget = new Path(tmpDir, diff.source.getName() + random.nextInt());
+        tmpTarget = new Path(tmpDir,
+            diff.getSource().getName() + random.nextInt());
       }
       diff.setTmp(tmpTarget);
-      targetFs.rename(diff.source, tmpTarget);
+      targetFs.rename(diff.getSource(), tmpTarget);
     }
   }
 
@@ -300,11 +327,11 @@ class DistCpSync {
     // directories are created first.
     Arrays.sort(diffs, DiffInfo.targetComparator);
     for (DiffInfo diff : diffs) {
-      if (diff.target != null) {
-        if (!targetFs.exists(diff.target.getParent())) {
-          targetFs.mkdirs(diff.target.getParent());
+      if (diff.getTarget() != null) {
+        if (!targetFs.exists(diff.getTarget().getParent())) {
+          targetFs.mkdirs(diff.getTarget().getParent());
         }
-        targetFs.rename(diff.getTmp(), diff.target);
+        targetFs.rename(diff.getTmp(), diff.getTarget());
       }
     }
   }
@@ -313,17 +340,80 @@ class DistCpSync {
    * Get rename and delete diffs and add the targetDir as the prefix of their
    * source and target paths.
    */
-  private DiffInfo[] getRenameAndDeleteDiffs(Path targetDir) {
+  private DiffInfo[] getRenameAndDeleteDiffsForSync(Path targetDir) {
+    // NOTE: when HDFS-10263 is done, getRenameAndDeleteDiffsRdiff
+    // should be the same as getRenameAndDeleteDiffsFdiff. Specifically,
+    // we should just move the body of getRenameAndDeleteDiffsFdiff
+    // to here and remove both getRenameAndDeleteDiffsFdiff
+    // and getRenameAndDeleteDiffsDdiff.
+    if (isRdiff()) {
+      return getRenameAndDeleteDiffsRdiff(targetDir);
+    } else {
+      return getRenameAndDeleteDiffsFdiff(targetDir);
+    }
+  }
+
+  /**
+   * Get rename and delete diffs and add the targetDir as the prefix of their
+   * source and target paths.
+   */
+  private DiffInfo[] getRenameAndDeleteDiffsRdiff(Path targetDir) {
+    List<DiffInfo> renameDiffsList =
+        diffMap.get(SnapshotDiffReport.DiffType.RENAME);
+
+    // Prepare a renameDiffArray for translating deleted items below.
+    // Do a reversion here due to HDFS-10263.
+    List<DiffInfo> renameDiffsListReversed =
+        new ArrayList<DiffInfo>(renameDiffsList.size());
+    for (DiffInfo diff : renameDiffsList) {
+      renameDiffsListReversed.add(new DiffInfo(diff.getTarget(),
+          diff.getSource(), diff.getType()));
+    }
+    DiffInfo[] renameDiffArray =
+        renameDiffsListReversed.toArray(new DiffInfo[renameDiffsList.size()]);
+
+    Arrays.sort(renameDiffArray, DiffInfo.sourceComparator);
+
+    List<DiffInfo> renameAndDeleteDiff = new ArrayList<>();
+    // Traverse DELETE list, which we need to delete them in sync process.
+    // Use the renameDiffArray prepared to translate the path.
+    for (DiffInfo diff : diffMap.get(SnapshotDiffReport.DiffType.DELETE)) {
+      DiffInfo renameItem = getRenameItem(diff, renameDiffArray);
+      Path source;
+      if (renameItem != null) {
+        source = new Path(targetDir,
+            translateRenamedPath(diff.getSource(), renameItem));
+      } else {
+        source = new Path(targetDir, diff.getSource());
+      }
+      renameAndDeleteDiff.add(new DiffInfo(source, null,
+          SnapshotDiffReport.DiffType.DELETE));
+    }
+    for (DiffInfo diff : diffMap.get(SnapshotDiffReport.DiffType.RENAME)) {
+      // swap target and source here for Rdiff
+      Path source = new Path(targetDir, diff.getSource());
+      Path target = new Path(targetDir, diff.getTarget());
+      renameAndDeleteDiff.add(new DiffInfo(source, target, diff.getType()));
+    }
+    return renameAndDeleteDiff.toArray(
+        new DiffInfo[renameAndDeleteDiff.size()]);
+  }
+
+    /**
+   * Get rename and delete diffs and add the targetDir as the prefix of their
+   * source and target paths.
+   */
+  private DiffInfo[] getRenameAndDeleteDiffsFdiff(Path targetDir) {
     List<DiffInfo> renameAndDeleteDiff = new ArrayList<>();
     for (DiffInfo diff : diffMap.get(SnapshotDiffReport.DiffType.DELETE)) {
-      Path source = new Path(targetDir, diff.source);
-      renameAndDeleteDiff.add(new DiffInfo(source, diff.target,
+      Path source = new Path(targetDir, diff.getSource());
+      renameAndDeleteDiff.add(new DiffInfo(source, diff.getTarget(),
           diff.getType()));
     }
 
     for (DiffInfo diff : diffMap.get(SnapshotDiffReport.DiffType.RENAME)) {
-      Path source = new Path(targetDir, diff.source);
-      Path target = new Path(targetDir, diff.target);
+      Path source = new Path(targetDir, diff.getSource());
+      Path target = new Path(targetDir, diff.getTarget());
       renameAndDeleteDiff.add(new DiffInfo(source, target, diff.getType()));
     }
 
@@ -367,7 +457,7 @@ class DistCpSync {
    */
   private DiffInfo getRenameItem(DiffInfo diff, DiffInfo[] renameDiffArray) {
     for (DiffInfo renameItem : renameDiffArray) {
-      if (diff.source.equals(renameItem.source)) {
+      if (diff.getSource().equals(renameItem.getSource())) {
         // The same path string may appear in:
         // 1. both renamed and modified snapshot diff entries.
         // 2. both renamed and created snapshot diff entries.
@@ -377,7 +467,7 @@ class DistCpSync {
         if (diff.getType() == SnapshotDiffReport.DiffType.MODIFY) {
           return renameItem;
         }
-      } else if (isParentOf(renameItem.source, diff.source)) {
+      } else if (isParentOf(renameItem.getSource(), diff.getSource())) {
         // If rename entry is the parent of diff entry, then both MODIFY and
         // CREATE diff entries should be handled.
         return renameItem;
@@ -387,16 +477,27 @@ class DistCpSync {
   }
 
   /**
-   * For a given source path, get its target path based on the rename item.
+   * For a given sourcePath, get its real path if it or its parent was renamed.
+   *
+   * For example, if we renamed dirX to dirY, and created dirY/fileX,
+   * the initial snapshot diff would be a CREATE snapshot diff that looks like
+   *   + dirX/fileX
+   * The rename snapshot diff looks like
+   *   R dirX dirY
+   *
+   * We convert the soucePath dirX/fileX to dirY/fileX here.
+   *
    * @return target path
    */
-  private Path getTargetPath(Path sourcePath, DiffInfo renameItem) {
-    if (sourcePath.equals(renameItem.source)) {
-      return renameItem.target;
+  private Path translateRenamedPath(Path sourcePath,
+      DiffInfo renameItem) {
+    if (sourcePath.equals(renameItem.getSource())) {
+      return renameItem.getTarget();
     }
     StringBuffer sb = new StringBuffer(sourcePath.toString());
-    String remain = sb.substring(renameItem.source.toString().length() + 1);
-    return new Path(renameItem.target, remain);
+    String remain =
+        sb.substring(renameItem.getSource().toString().length() + 1);
+    return new Path(renameItem.getTarget(), remain);
   }
 
   /**
@@ -406,26 +507,35 @@ class DistCpSync {
    *
    * If the parent or self of a source path is renamed, we need to change its
    * target path according the correspondent rename item.
+   *
+   * For RDiff usage, the diff.getSource() is what we will use as its target
+   * path.
+   *
    * @return a diff list
    */
-  public ArrayList<DiffInfo> prepareDiffList() {
+  public ArrayList<DiffInfo> prepareDiffListForCopyListing() {
     DiffInfo[] modifyAndCreateDiffs = getCreateAndModifyDiffs();
-
-    List<DiffInfo> renameDiffsList =
-        diffMap.get(SnapshotDiffReport.DiffType.RENAME);
-    DiffInfo[] renameDiffArray =
-        renameDiffsList.toArray(new DiffInfo[renameDiffsList.size()]);
-    Arrays.sort(renameDiffArray, DiffInfo.sourceComparator);
-
     ArrayList<DiffInfo> finalListWithTarget = new ArrayList<>();
-    for (DiffInfo diff : modifyAndCreateDiffs) {
-      DiffInfo renameItem = getRenameItem(diff, renameDiffArray);
-      if (renameItem == null) {
-        diff.target = diff.source;
-      } else {
-        diff.target = getTargetPath(diff.source, renameItem);
+    if (isRdiff()) {
+      for (DiffInfo diff : modifyAndCreateDiffs) {
+        diff.setTarget(diff.getSource());
+        finalListWithTarget.add(diff);
+      }
+    } else {
+      List<DiffInfo> renameDiffsList =
+          diffMap.get(SnapshotDiffReport.DiffType.RENAME);
+      DiffInfo[] renameDiffArray =
+          renameDiffsList.toArray(new DiffInfo[renameDiffsList.size()]);
+      Arrays.sort(renameDiffArray, DiffInfo.sourceComparator);
+      for (DiffInfo diff : modifyAndCreateDiffs) {
+        DiffInfo renameItem = getRenameItem(diff, renameDiffArray);
+        if (renameItem == null) {
+          diff.setTarget(diff.getSource());
+        } else {
+          diff.setTarget(translateRenamedPath(diff.getSource(), renameItem));
+        }
+        finalListWithTarget.add(diff);
       }
-      finalListWithTarget.add(diff);
     }
     return finalListWithTarget;
   }
@@ -459,9 +569,9 @@ class DistCpSync {
     boolean foundChild = false;
     HashSet<String> excludeList = new HashSet<>();
     for (DiffInfo diff : renameDiffs) {
-      if (isParentOf(newDir, diff.target)) {
+      if (isParentOf(newDir, diff.getTarget())) {
         foundChild = true;
-        excludeList.add(new Path(prefix, diff.target).toUri().getPath());
+        excludeList.add(new Path(prefix, diff.getTarget()).toUri().getPath());
       } else if (foundChild) {
         // The renameDiffs was sorted, the matching section should be
         // contiguous.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/412c4c9a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
index 5eaf4da..d0f82ca 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
@@ -28,6 +28,7 @@ import org.apache.commons.cli.GnuParser;
 import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
@@ -66,6 +67,13 @@ public class OptionsParser {
     }
   }
 
+  private static void checkSnapshotsArgs(final String[] snapshots) {
+    Preconditions.checkArgument(snapshots != null && snapshots.length == 2
+        && !StringUtils.isBlank(snapshots[0])
+        && !StringUtils.isBlank(snapshots[1]),
+        "Must provide both the starting and ending snapshot names");
+  }
+
   /**
    * The parse method parses the command-line options, and creates
    * a corresponding Options object.
@@ -74,7 +82,8 @@ public class OptionsParser {
    * @return The Options object, corresponding to the specified command-line.
    * @throws IllegalArgumentException Thrown if the parse fails.
    */
-  public static DistCpOptions parse(String args[]) throws IllegalArgumentException {
+  public static DistCpOptions parse(String[] args)
+      throws IllegalArgumentException {
 
     CommandLineParser parser = new CustomParser();
 
@@ -142,10 +151,16 @@ public class OptionsParser {
     parsePreserveStatus(command, option);
 
     if (command.hasOption(DistCpOptionSwitch.DIFF.getSwitch())) {
-      String[] snapshots = getVals(command, DistCpOptionSwitch.DIFF.getSwitch());
-      Preconditions.checkArgument(snapshots != null && snapshots.length == 2,
-          "Must provide both the starting and ending snapshot names");
-      option.setUseDiff(true, snapshots[0], snapshots[1]);
+      String[] snapshots = getVals(command,
+          DistCpOptionSwitch.DIFF.getSwitch());
+      checkSnapshotsArgs(snapshots);
+      option.setUseDiff(snapshots[0], snapshots[1]);
+    }
+    if (command.hasOption(DistCpOptionSwitch.RDIFF.getSwitch())) {
+      String[] snapshots = getVals(command,
+          DistCpOptionSwitch.RDIFF.getSwitch());
+      checkSnapshotsArgs(snapshots);
+      option.setUseRdiff(snapshots[0], snapshots[1]);
     }
 
     parseFileLimit(command);
@@ -332,7 +347,7 @@ public class OptionsParser {
             "source paths present");
       }
       option = new DistCpOptions(new Path(getVal(command, DistCpOptionSwitch.
-              SOURCE_FILE_LISTING.getSwitch())), targetPath);
+          SOURCE_FILE_LISTING.getSwitch())), targetPath);
     } else {
       if (sourcePaths.isEmpty()) {
         throw new IllegalArgumentException("Neither source file listing nor " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/412c4c9a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/SimpleCopyListing.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/SimpleCopyListing.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/SimpleCopyListing.java
index bc30aa1..0002d4f 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/SimpleCopyListing.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/SimpleCopyListing.java
@@ -194,7 +194,7 @@ public class SimpleCopyListing extends CopyListing {
   @Override
   protected void doBuildListing(Path pathToListingFile,
                                 DistCpOptions options) throws IOException {
-    if(options.shouldUseDiff()) {
+    if(options.shouldUseSnapshotDiff()) {
       doBuildListingWithSnapshotDiff(getWriter(pathToListingFile), options);
     }else {
       doBuildListing(getWriter(pathToListingFile), options);
@@ -256,7 +256,7 @@ public class SimpleCopyListing extends CopyListing {
   protected void doBuildListingWithSnapshotDiff(
       SequenceFile.Writer fileListWriter, DistCpOptions options)
       throws IOException {
-    ArrayList<DiffInfo> diffList = distCpSync.prepareDiffList();
+    ArrayList<DiffInfo> diffList = distCpSync.prepareDiffListForCopyListing();
     Path sourceRoot = options.getSourcePaths().get(0);
     FileSystem sourceFS = sourceRoot.getFileSystem(getConf());
 
@@ -264,13 +264,16 @@ public class SimpleCopyListing extends CopyListing {
       List<FileStatusInfo> fileStatuses = Lists.newArrayList();
       for (DiffInfo diff : diffList) {
         // add snapshot paths prefix
-        diff.target = new Path(options.getSourcePaths().get(0), diff.target);
+        diff.setTarget(
+            new Path(options.getSourcePaths().get(0), diff.getTarget()));
         if (diff.getType() == SnapshotDiffReport.DiffType.MODIFY) {
-          addToFileListing(fileListWriter, sourceRoot, diff.target, options);
+          addToFileListing(fileListWriter,
+              sourceRoot, diff.getTarget(), options);
         } else if (diff.getType() == SnapshotDiffReport.DiffType.CREATE) {
-          addToFileListing(fileListWriter, sourceRoot, diff.target, options);
+          addToFileListing(fileListWriter,
+              sourceRoot, diff.getTarget(), options);
 
-          FileStatus sourceStatus = sourceFS.getFileStatus(diff.target);
+          FileStatus sourceStatus = sourceFS.getFileStatus(diff.getTarget());
           if (sourceStatus.isDirectory()) {
             if (LOG.isDebugEnabled()) {
               LOG.debug("Adding source dir for traverse: " +
@@ -278,7 +281,7 @@ public class SimpleCopyListing extends CopyListing {
             }
 
             HashSet<String> excludeList =
-                distCpSync.getTraverseExcludeList(diff.source,
+                distCpSync.getTraverseExcludeList(diff.getSource(),
                     options.getSourcePaths().get(0));
 
             ArrayList<FileStatus> sourceDirs = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/412c4c9a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSync.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSync.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSync.java
index 3419b2f..94e8604 100644
--- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSync.java
+++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSync.java
@@ -64,7 +64,7 @@ public class TestDistCpSync {
 
     options = new DistCpOptions(Arrays.asList(source), target);
     options.setSyncFolder(true);
-    options.setUseDiff(true, "s1", "s2");
+    options.setUseDiff("s1", "s2");
     options.appendToConf(conf);
 
     conf.set(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH, target.toString());
@@ -312,7 +312,7 @@ public class TestDistCpSync {
    */
   @Test
   public void testSyncWithCurrent() throws Exception {
-    options.setUseDiff(true, "s1", ".");
+    options.setUseDiff("s1", ".");
     initData(source);
     initData(target);
     enableAndCreateFirstSnapshot();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/412c4c9a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSyncReverseBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSyncReverseBase.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSyncReverseBase.java
new file mode 100644
index 0000000..fea374e
--- /dev/null
+++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSyncReverseBase.java
@@ -0,0 +1,868 @@
+/**
+ * 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.tools;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.tools.mapred.CopyMapper;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.StringTokenizer;
+
+/**
+ * Base class to test "-rdiff s2 s1".
+ * Shared by "-rdiff s2 s1 src tgt" and "-rdiff s2 s1 tgt tgt"
+ */
+public abstract class TestDistCpSyncReverseBase {
+  private MiniDFSCluster cluster;
+  private final Configuration conf = new HdfsConfiguration();
+  private DistributedFileSystem dfs;
+  private DistCpOptions options;
+  private Path source;
+  private boolean isSrcNotSameAsTgt = true;
+  private final Path target = new Path("/target");
+  private final long blockSize = 1024;
+  private final short dataNum = 1;
+
+  abstract void initSourcePath();
+
+  private static List<String> lsr(final String prefix,
+      final FsShell shell, Path rootDir) throws Exception {
+    return lsr(prefix, shell, rootDir.toString(), null);
+  }
+
+  private List<String> lsrSource(final String prefix,
+      final FsShell shell, Path rootDir) throws Exception {
+    final Path spath = isSrcNotSameAsTgt? rootDir :
+      new Path(rootDir.toString(),
+          HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + "s1");
+    return lsr(prefix, shell, spath.toString(), null);
+  }
+
+  private static List<String> lsr(final String prefix,
+      final FsShell shell, String rootDir, String glob) throws Exception {
+    final String dir = glob == null ? rootDir : glob;
+    System.out.println(prefix + " lsr root=" + rootDir);
+    final ByteArrayOutputStream bytes = new ByteArrayOutputStream();
+    final PrintStream out = new PrintStream(bytes);
+    final PrintStream oldOut = System.out;
+    final PrintStream oldErr = System.err;
+    System.setOut(out);
+    System.setErr(out);
+    final String results;
+    try {
+      Assert.assertEquals(0, shell.run(new String[] {"-lsr", dir }));
+      results = bytes.toString();
+    } finally {
+      IOUtils.closeStream(out);
+      System.setOut(oldOut);
+      System.setErr(oldErr);
+    }
+    System.out.println("lsr results:\n" + results);
+    String dirname = rootDir;
+    if (rootDir.lastIndexOf(Path.SEPARATOR) != -1) {
+      dirname = rootDir.substring(rootDir.lastIndexOf(Path.SEPARATOR));
+    }
+
+    final List<String> paths = new ArrayList<String>();
+    for (StringTokenizer t = new StringTokenizer(results, "\n"); t
+        .hasMoreTokens();) {
+      final String s = t.nextToken();
+      final int i = s.indexOf(dirname);
+      if (i >= 0) {
+        paths.add(s.substring(i + dirname.length()));
+      }
+    }
+    Collections.sort(paths);
+    System.out
+        .println("lsr paths = " + paths.toString().replace(", ", ",\n  "));
+    return paths;
+  }
+
+  public void setSource(final Path src) {
+    this.source = src;
+  }
+
+  public void setSrcNotSameAsTgt(final boolean srcNotSameAsTgt) {
+    isSrcNotSameAsTgt = srcNotSameAsTgt;
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    initSourcePath();
+
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(dataNum).build();
+    cluster.waitActive();
+
+    dfs = cluster.getFileSystem();
+    if (isSrcNotSameAsTgt) {
+      dfs.mkdirs(source);
+    }
+    dfs.mkdirs(target);
+
+    options = new DistCpOptions(Arrays.asList(source), target);
+    options.setSyncFolder(true);
+    options.setUseRdiff("s2", "s1");
+    options.appendToConf(conf);
+
+    conf.set(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH, target.toString());
+    conf.set(DistCpConstants.CONF_LABEL_TARGET_FINAL_PATH, target.toString());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    IOUtils.cleanup(null, dfs);
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Test the sync returns false in the following scenarios:
+   * 1. the source/target dir are not snapshottable dir
+   * 2. the source/target does not have the given snapshots
+   * 3. changes have been made in target
+   */
+  @Test
+  public void testFallback() throws Exception {
+    // the source/target dir are not snapshottable dir
+    Assert.assertFalse(sync());
+    // make sure the source path has been updated to the snapshot path
+    final Path spath = new Path(source,
+        HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + "s1");
+    Assert.assertEquals(spath, options.getSourcePaths().get(0));
+
+    // reset source path in options
+    options.setSourcePaths(Arrays.asList(source));
+    // the source/target does not have the given snapshots
+    dfs.allowSnapshot(source);
+    dfs.allowSnapshot(target);
+    Assert.assertFalse(sync());
+    Assert.assertEquals(spath, options.getSourcePaths().get(0));
+
+    // reset source path in options
+    options.setSourcePaths(Arrays.asList(source));
+    this.enableAndCreateFirstSnapshot();
+    dfs.createSnapshot(target, "s2");
+    Assert.assertTrue(sync());
+
+    // reset source paths in options
+    options.setSourcePaths(Arrays.asList(source));
+    // changes have been made in target
+    final Path subTarget = new Path(target, "sub");
+    dfs.mkdirs(subTarget);
+    Assert.assertFalse(sync());
+    // make sure the source path has been updated to the snapshot path
+    Assert.assertEquals(spath, options.getSourcePaths().get(0));
+
+    // reset source paths in options
+    options.setSourcePaths(Arrays.asList(source));
+    dfs.delete(subTarget, true);
+    Assert.assertTrue(sync());
+  }
+
+  private void syncAndVerify() throws Exception {
+
+    final FsShell shell = new FsShell(conf);
+    lsrSource("Before sync source: ", shell, source);
+    lsr("Before sync target: ", shell, target);
+
+    Assert.assertTrue(sync());
+
+    lsrSource("After sync source: ", shell, source);
+    lsr("After sync target: ", shell, target);
+
+    verifyCopy(dfs.getFileStatus(source), dfs.getFileStatus(target), false);
+  }
+
+  private boolean sync() throws Exception {
+    DistCpSync distCpSync = new DistCpSync(options, conf);
+    return distCpSync.sync();
+  }
+
+  private void enableAndCreateFirstSnapshot() throws Exception {
+    if (isSrcNotSameAsTgt) {
+      dfs.allowSnapshot(source);
+      dfs.createSnapshot(source, "s1");
+    }
+    dfs.allowSnapshot(target);
+    dfs.createSnapshot(target, "s1");
+  }
+
+  private void createSecondSnapshotAtTarget() throws Exception {
+    dfs.createSnapshot(target, "s2");
+  }
+
+  private void createMiddleSnapshotAtTarget() throws Exception {
+    dfs.createSnapshot(target, "s1.5");
+  }
+
+  /**
+   * create some files and directories under the given directory.
+   * the final subtree looks like this:
+   *                     dir/
+   *              foo/          bar/
+   *           d1/    f1     d2/    f2
+   *         f3            f4
+   */
+  private void initData(Path dir) throws Exception {
+    final Path foo = new Path(dir, "foo");
+    final Path bar = new Path(dir, "bar");
+    final Path d1 = new Path(foo, "d1");
+    final Path f1 = new Path(foo, "f1");
+    final Path d2 = new Path(bar, "d2");
+    final Path f2 = new Path(bar, "f2");
+    final Path f3 = new Path(d1, "f3");
+    final Path f4 = new Path(d2, "f4");
+
+    DFSTestUtil.createFile(dfs, f1, blockSize, dataNum, 0);
+    DFSTestUtil.createFile(dfs, f2, blockSize, dataNum, 0);
+    DFSTestUtil.createFile(dfs, f3, blockSize, dataNum, 0);
+    DFSTestUtil.createFile(dfs, f4, blockSize, dataNum, 0);
+  }
+
+  /**
+   * make some changes under the given directory (created in the above way).
+   * 1. rename dir/foo/d1 to dir/bar/d1
+   * 2. delete dir/bar/d1/f3
+   * 3. rename dir/foo to /dir/bar/d1/foo
+   * 4. delete dir/bar/d1/foo/f1
+   * 5. create file dir/bar/d1/foo/f1 whose size is 2*BLOCK_SIZE
+   * 6. append one BLOCK to file dir/bar/f2
+   * 7. rename dir/bar to dir/foo
+   *
+   * Thus after all these ops the subtree looks like this:
+   *                       dir/
+   *                       foo/
+   *                 d1/    f2(A)    d2/
+   *                foo/             f4
+   *                f1(new)
+   */
+  private int changeData(Path dir) throws Exception {
+    final Path foo = new Path(dir, "foo");
+    final Path bar = new Path(dir, "bar");
+    final Path d1 = new Path(foo, "d1");
+    final Path f2 = new Path(bar, "f2");
+
+    final Path bar_d1 = new Path(bar, "d1");
+    int numDeletedModified = 0;
+    dfs.rename(d1, bar_d1);
+    numDeletedModified += 1; // modify ./foo
+    numDeletedModified += 1; // modify ./bar
+    final Path f3 = new Path(bar_d1, "f3");
+    dfs.delete(f3, true);
+    numDeletedModified += 1; // delete f3
+    final Path newfoo = new Path(bar_d1, "foo");
+    dfs.rename(foo, newfoo);
+    numDeletedModified += 1; // modify ./foo/d1
+    final Path f1 = new Path(newfoo, "f1");
+    dfs.delete(f1, true);
+    numDeletedModified += 1; // delete ./foo/f1
+    DFSTestUtil.createFile(dfs, f1, 2 * blockSize, dataNum, 0);
+    DFSTestUtil.appendFile(dfs, f2, (int) blockSize);
+    numDeletedModified += 1; // modify ./bar/f2
+    dfs.rename(bar, new Path(dir, "foo"));
+    return numDeletedModified;
+  }
+
+  /**
+   * Test the basic functionality.
+   */
+  @Test
+  public void testSync() throws Exception {
+    if (isSrcNotSameAsTgt) {
+      initData(source);
+    }
+    initData(target);
+    enableAndCreateFirstSnapshot();
+
+    final FsShell shell = new FsShell(conf);
+
+    lsrSource("Before source: ", shell, source);
+    lsr("Before target: ", shell, target);
+
+    // make changes under target
+    int numDeletedModified = changeData(target);
+
+    createSecondSnapshotAtTarget();
+
+    SnapshotDiffReport report = dfs.getSnapshotDiffReport(target, "s2", "s1");
+    System.out.println(report);
+
+    DistCpSync distCpSync = new DistCpSync(options, conf);
+
+    lsr("Before sync target: ", shell, target);
+
+    // do the sync
+    Assert.assertTrue(distCpSync.sync());
+
+    lsr("After sync target: ", shell, target);
+
+    // make sure the source path has been updated to the snapshot path
+    final Path spath = new Path(source,
+        HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + "s1");
+    Assert.assertEquals(spath, options.getSourcePaths().get(0));
+
+    // build copy listing
+    final Path listingPath = new Path("/tmp/META/fileList.seq");
+    CopyListing listing = new SimpleCopyListing(conf, new Credentials(),
+        distCpSync);
+    listing.buildListing(listingPath, options);
+
+    Map<Text, CopyListingFileStatus> copyListing = getListing(listingPath);
+    CopyMapper copyMapper = new CopyMapper();
+    StubContext stubContext = new StubContext(conf, null, 0);
+    Mapper<Text, CopyListingFileStatus, Text, Text>.Context context =
+        stubContext.getContext();
+    // Enable append
+    context.getConfiguration().setBoolean(
+        DistCpOptionSwitch.APPEND.getConfigLabel(), true);
+    copyMapper.setup(context);
+    for (Map.Entry<Text, CopyListingFileStatus> entry :
+      copyListing.entrySet()) {
+      copyMapper.map(entry.getKey(), entry.getValue(), context);
+    }
+
+    lsrSource("After mapper source: ", shell, source);
+    lsr("After mapper target: ", shell, target);
+
+    // verify that we only list modified and created files/directories
+    Assert.assertEquals(numDeletedModified, copyListing.size());
+
+    // verify that we only copied new appended data of f2 and the new file f1
+    Assert.assertEquals(blockSize * 3, stubContext.getReporter()
+        .getCounter(CopyMapper.Counter.BYTESCOPIED).getValue());
+
+    // verify the source and target now has the same structure
+    verifyCopy(dfs.getFileStatus(spath), dfs.getFileStatus(target), false);
+  }
+
+  private Map<Text, CopyListingFileStatus> getListing(Path listingPath)
+      throws Exception {
+    SequenceFile.Reader reader = null;
+    Map<Text, CopyListingFileStatus> values = new HashMap<>();
+    try {
+      reader = new SequenceFile.Reader(conf,
+          SequenceFile.Reader.file(listingPath));
+      Text key = new Text();
+      CopyListingFileStatus value = new CopyListingFileStatus();
+      while (reader.next(key, value)) {
+        values.put(key, value);
+        key = new Text();
+        value = new CopyListingFileStatus();
+      }
+    } finally {
+      if (reader != null) {
+        reader.close();
+      }
+    }
+    return values;
+  }
+
+  private void verifyCopy(FileStatus s, FileStatus t, boolean compareName)
+      throws Exception {
+    Assert.assertEquals(s.isDirectory(), t.isDirectory());
+    if (compareName) {
+      Assert.assertEquals(s.getPath().getName(), t.getPath().getName());
+    }
+    if (!s.isDirectory()) {
+      // verify the file content is the same
+      byte[] sbytes = DFSTestUtil.readFileBuffer(dfs, s.getPath());
+      byte[] tbytes = DFSTestUtil.readFileBuffer(dfs, t.getPath());
+      Assert.assertArrayEquals(sbytes, tbytes);
+    } else {
+      FileStatus[] slist = dfs.listStatus(s.getPath());
+      FileStatus[] tlist = dfs.listStatus(t.getPath());
+      Assert.assertEquals(slist.length, tlist.length);
+      for (int i = 0; i < slist.length; i++) {
+        verifyCopy(slist[i], tlist[i], true);
+      }
+    }
+  }
+
+  /**
+   * Test the case that "current" is snapshotted as "s2".
+   * @throws Exception
+   */
+  @Test
+  public void testSyncWithCurrent() throws Exception {
+    options.setUseRdiff(".", "s1");
+    if (isSrcNotSameAsTgt) {
+      initData(source);
+    }
+    initData(target);
+    enableAndCreateFirstSnapshot();
+
+    // make changes under target
+    changeData(target);
+
+    // do the sync
+    Assert.assertTrue(sync());
+    final Path spath = new Path(source,
+        HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + "s1");
+    // make sure the source path is still unchanged
+    Assert.assertEquals(spath, options.getSourcePaths().get(0));
+  }
+
+  private void initData2(Path dir) throws Exception {
+    final Path test = new Path(dir, "test");
+    final Path foo = new Path(dir, "foo");
+    final Path bar = new Path(dir, "bar");
+    final Path f1 = new Path(test, "f1");
+    final Path f2 = new Path(foo, "f2");
+    final Path f3 = new Path(bar, "f3");
+
+    DFSTestUtil.createFile(dfs, f1, blockSize, dataNum, 0L);
+    DFSTestUtil.createFile(dfs, f2, blockSize, dataNum, 1L);
+    DFSTestUtil.createFile(dfs, f3, blockSize, dataNum, 2L);
+  }
+
+  private void changeData2(Path dir) throws Exception {
+    final Path tmpFoo = new Path(dir, "tmpFoo");
+    final Path test = new Path(dir, "test");
+    final Path foo = new Path(dir, "foo");
+    final Path bar = new Path(dir, "bar");
+
+    dfs.rename(test, tmpFoo);
+    dfs.rename(foo, test);
+    dfs.rename(bar, foo);
+    dfs.rename(tmpFoo, bar);
+  }
+
+  @Test
+  public void testSync2() throws Exception {
+    if (isSrcNotSameAsTgt) {
+      initData2(source);
+    }
+    initData2(target);
+    enableAndCreateFirstSnapshot();
+
+    // make changes under target
+    changeData2(target);
+
+    createSecondSnapshotAtTarget();
+
+    SnapshotDiffReport report = dfs.getSnapshotDiffReport(target, "s2", "s1");
+    System.out.println(report);
+
+    syncAndVerify();
+  }
+
+  private void initData3(Path dir) throws Exception {
+    final Path test = new Path(dir, "test");
+    final Path foo = new Path(dir, "foo");
+    final Path bar = new Path(dir, "bar");
+    final Path f1 = new Path(test, "file");
+    final Path f2 = new Path(foo, "file");
+    final Path f3 = new Path(bar, "file");
+
+    DFSTestUtil.createFile(dfs, f1, blockSize, dataNum, 0L);
+    DFSTestUtil.createFile(dfs, f2, blockSize * 2, dataNum, 1L);
+    DFSTestUtil.createFile(dfs, f3, blockSize * 3, dataNum, 2L);
+  }
+
+  private void changeData3(Path dir) throws Exception {
+    final Path test = new Path(dir, "test");
+    final Path foo = new Path(dir, "foo");
+    final Path bar = new Path(dir, "bar");
+    final Path f1 = new Path(test, "file");
+    final Path f2 = new Path(foo, "file");
+    final Path f3 = new Path(bar, "file");
+    final Path newf1 = new Path(test, "newfile");
+    final Path newf2 = new Path(foo, "newfile");
+    final Path newf3 = new Path(bar, "newfile");
+
+    dfs.rename(f1, newf1);
+    dfs.rename(f2, newf2);
+    dfs.rename(f3, newf3);
+  }
+
+  /**
+   * Test a case where there are multiple source files with the same name.
+   */
+  @Test
+  public void testSync3() throws Exception {
+    if (isSrcNotSameAsTgt) {
+      initData3(source);
+    }
+    initData3(target);
+    enableAndCreateFirstSnapshot();
+
+    // make changes under target
+    changeData3(target);
+
+    createSecondSnapshotAtTarget();
+
+    SnapshotDiffReport report = dfs.getSnapshotDiffReport(target, "s2", "s1");
+    System.out.println(report);
+
+    syncAndVerify();
+  }
+
+  private void initData4(Path dir) throws Exception {
+    final Path d1 = new Path(dir, "d1");
+    final Path d2 = new Path(d1, "d2");
+    final Path f1 = new Path(d2, "f1");
+
+    DFSTestUtil.createFile(dfs, f1, blockSize, dataNum, 0L);
+  }
+
+  private int changeData4(Path dir) throws Exception {
+    final Path d1 = new Path(dir, "d1");
+    final Path d11 = new Path(dir, "d11");
+    final Path d2 = new Path(d1, "d2");
+    final Path d21 = new Path(d1, "d21");
+    final Path f1 = new Path(d2, "f1");
+
+    int numDeletedAndModified = 0;
+    dfs.delete(f1, false);
+    numDeletedAndModified += 1;
+    dfs.rename(d2, d21);
+    numDeletedAndModified += 1;
+    dfs.rename(d1, d11);
+    numDeletedAndModified += 1;
+    return numDeletedAndModified;
+  }
+
+  /**
+   * Test a case where multiple level dirs are renamed.
+   */
+  @Test
+  public void testSync4() throws Exception {
+    if (isSrcNotSameAsTgt) {
+      initData4(source);
+    }
+    initData4(target);
+    enableAndCreateFirstSnapshot();
+
+    final FsShell shell = new FsShell(conf);
+    lsr("Before change target: ", shell, target);
+
+    // make changes under target
+    int numDeletedAndModified = changeData4(target);
+
+    createSecondSnapshotAtTarget();
+
+    SnapshotDiffReport report = dfs.getSnapshotDiffReport(target, "s2", "s1");
+    System.out.println(report);
+
+    testAndVerify(numDeletedAndModified);
+  }
+
+  private void initData5(Path dir) throws Exception {
+    final Path d1 = new Path(dir, "d1");
+    final Path d2 = new Path(dir, "d2");
+    final Path f1 = new Path(d1, "f1");
+    final Path f2 = new Path(d2, "f2");
+
+    DFSTestUtil.createFile(dfs, f1, blockSize, dataNum, 0L);
+    DFSTestUtil.createFile(dfs, f2, blockSize, dataNum, 0L);
+  }
+
+  private int changeData5(Path dir) throws Exception {
+    final Path d1 = new Path(dir, "d1");
+    final Path d2 = new Path(dir, "d2");
+    final Path f1 = new Path(d1, "f1");
+    final Path tmp = new Path(dir, "tmp");
+
+    int numDeletedAndModified = 0;
+    dfs.delete(f1, false);
+    numDeletedAndModified += 1;
+    dfs.rename(d1, tmp);
+    numDeletedAndModified += 1;
+    dfs.rename(d2, d1);
+    numDeletedAndModified += 1;
+    final Path f2 = new Path(d1, "f2");
+    dfs.delete(f2, false);
+    numDeletedAndModified += 1;
+    return numDeletedAndModified;
+  }
+
+   /**
+   * Test a case with different delete and rename sequences.
+   */
+  @Test
+  public void testSync5() throws Exception {
+    if (isSrcNotSameAsTgt) {
+      initData5(source);
+    }
+    initData5(target);
+    enableAndCreateFirstSnapshot();
+
+    // make changes under target
+    int numDeletedAndModified = changeData5(target);
+
+    createSecondSnapshotAtTarget();
+
+    SnapshotDiffReport report = dfs.getSnapshotDiffReport(target, "s2", "s1");
+    System.out.println(report);
+
+    testAndVerify(numDeletedAndModified);
+  }
+
+  private void testAndVerify(int numDeletedAndModified)
+          throws Exception{
+    SnapshotDiffReport report = dfs.getSnapshotDiffReport(target, "s2", "s1");
+    System.out.println(report);
+
+    final FsShell shell = new FsShell(conf);
+
+    lsrSource("Before sync source: ", shell, source);
+    lsr("Before sync target: ", shell, target);
+
+    DistCpSync distCpSync = new DistCpSync(options, conf);
+    // do the sync
+    distCpSync.sync();
+
+    lsr("After sync target: ", shell, target);
+
+    // make sure the source path has been updated to the snapshot path
+    final Path spath = new Path(source,
+            HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + "s1");
+    Assert.assertEquals(spath, options.getSourcePaths().get(0));
+
+    // build copy listing
+    final Path listingPath = new Path("/tmp/META/fileList.seq");
+    CopyListing listing = new SimpleCopyListing(conf, new Credentials(), distCpSync);
+    listing.buildListing(listingPath, options);
+
+    Map<Text, CopyListingFileStatus> copyListing = getListing(listingPath);
+    CopyMapper copyMapper = new CopyMapper();
+    StubContext stubContext = new StubContext(conf, null, 0);
+    Mapper<Text, CopyListingFileStatus, Text, Text>.Context context =
+            stubContext.getContext();
+    // Enable append
+    context.getConfiguration().setBoolean(
+            DistCpOptionSwitch.APPEND.getConfigLabel(), true);
+    copyMapper.setup(context);
+    for (Map.Entry<Text, CopyListingFileStatus> entry :
+            copyListing.entrySet()) {
+      copyMapper.map(entry.getKey(), entry.getValue(), context);
+    }
+
+    // verify that we only list modified and created files/directories
+    Assert.assertEquals(numDeletedAndModified, copyListing.size());
+
+    lsr("After Copy target: ", shell, target);
+
+    // verify the source and target now has the same structure
+    verifyCopy(dfs.getFileStatus(spath), dfs.getFileStatus(target), false);
+  }
+
+  private void initData6(Path dir) throws Exception {
+    final Path foo = new Path(dir, "foo");
+    final Path bar = new Path(dir, "bar");
+    final Path foo_f1 = new Path(foo, "f1");
+    final Path bar_f1 = new Path(bar, "f1");
+
+    DFSTestUtil.createFile(dfs, foo_f1, blockSize, dataNum, 0L);
+    DFSTestUtil.createFile(dfs, bar_f1, blockSize, dataNum, 0L);
+  }
+
+  private int changeData6(Path dir) throws Exception {
+    final Path foo = new Path(dir, "foo");
+    final Path bar = new Path(dir, "bar");
+    final Path foo2 = new Path(dir, "foo2");
+    final Path foo_f1 = new Path(foo, "f1");
+
+    int numDeletedModified = 0;
+    dfs.rename(foo, foo2);
+    dfs.rename(bar, foo);
+    dfs.rename(foo2, bar);
+    DFSTestUtil.appendFile(dfs, foo_f1, (int) blockSize);
+    numDeletedModified += 1; // modify ./bar/f1
+    return numDeletedModified;
+  }
+
+  /**
+   * Test a case where there is a cycle in renaming dirs.
+   */
+  @Test
+  public void testSync6() throws Exception {
+    if (isSrcNotSameAsTgt) {
+      initData6(source);
+    }
+    initData6(target);
+    enableAndCreateFirstSnapshot();
+    int numDeletedModified = changeData6(target);
+
+    createSecondSnapshotAtTarget();
+
+    testAndVerify(numDeletedModified);
+  }
+
+  private void initData7(Path dir) throws Exception {
+    final Path foo = new Path(dir, "foo");
+    final Path bar = new Path(dir, "bar");
+    final Path foo_f1 = new Path(foo, "f1");
+    final Path bar_f1 = new Path(bar, "f1");
+
+    DFSTestUtil.createFile(dfs, foo_f1, blockSize, dataNum, 0L);
+    DFSTestUtil.createFile(dfs, bar_f1, blockSize, dataNum, 0L);
+  }
+
+  private int changeData7(Path dir) throws Exception {
+    final Path foo = new Path(dir, "foo");
+    final Path foo2 = new Path(dir, "foo2");
+    final Path foo_f1 = new Path(foo, "f1");
+    final Path foo2_f2 = new Path(foo2, "f2");
+    final Path foo_d1 = new Path(foo, "d1");
+    final Path foo_d1_f3 = new Path(foo_d1, "f3");
+
+    int numDeletedAndModified = 0;
+    dfs.rename(foo, foo2);
+    DFSTestUtil.createFile(dfs, foo_f1, blockSize, dataNum, 0L);
+    DFSTestUtil.appendFile(dfs, foo_f1, (int) blockSize);
+    dfs.rename(foo_f1, foo2_f2);
+    /*
+     * Difference between snapshot s1 and current directory under directory
+       /target:
+M       .
++       ./foo
+R       ./foo -> ./foo2
+M       ./foo
++       ./foo/f2
+     */
+    numDeletedAndModified += 1; // "M ./foo"
+    DFSTestUtil.createFile(dfs, foo_d1_f3, blockSize, dataNum, 0L);
+    return numDeletedAndModified;
+  }
+
+  /**
+   * Test a case where rename a dir, then create a new dir with the same name
+   * and sub dir.
+   */
+  @Test
+  public void testSync7() throws Exception {
+    if (isSrcNotSameAsTgt) {
+      initData7(source);
+    }
+    initData7(target);
+    enableAndCreateFirstSnapshot();
+    int numDeletedAndModified = changeData7(target);
+
+    createSecondSnapshotAtTarget();
+
+    testAndVerify(numDeletedAndModified);
+  }
+
+  private void initData8(Path dir) throws Exception {
+    final Path foo = new Path(dir, "foo");
+    final Path bar = new Path(dir, "bar");
+    final Path d1 = new Path(dir, "d1");
+    final Path foo_f1 = new Path(foo, "f1");
+    final Path bar_f1 = new Path(bar, "f1");
+    final Path d1_f1 = new Path(d1, "f1");
+
+    DFSTestUtil.createFile(dfs, foo_f1, blockSize, dataNum, 0L);
+    DFSTestUtil.createFile(dfs, bar_f1, blockSize, dataNum, 0L);
+    DFSTestUtil.createFile(dfs, d1_f1, blockSize, dataNum, 0L);
+  }
+
+  private int changeData8(Path dir, boolean createMiddleSnapshot)
+      throws Exception {
+    final Path foo = new Path(dir, "foo");
+    final Path createdDir = new Path(dir, "c");
+    final Path d1 = new Path(dir, "d1");
+    final Path d1_f1 = new Path(d1, "f1");
+    final Path createdDir_f1 = new Path(createdDir, "f1");
+    final Path foo_f3 = new Path(foo, "f3");
+    final Path new_foo = new Path(createdDir, "foo");
+    final Path foo_f4 = new Path(foo, "f4");
+    final Path foo_d1 = new Path(foo, "d1");
+    final Path bar = new Path(dir, "bar");
+    final Path bar1 = new Path(dir, "bar1");
+
+    int numDeletedAndModified = 0;
+    DFSTestUtil.createFile(dfs, foo_f3, blockSize, dataNum, 0L);
+    DFSTestUtil.createFile(dfs, createdDir_f1, blockSize, dataNum, 0L);
+    dfs.rename(createdDir_f1, foo_f4);
+    dfs.rename(d1_f1, createdDir_f1); // rename ./d1/f1 -> ./c/f1
+    numDeletedAndModified += 1; // modify ./c/foo/d1
+
+    if (createMiddleSnapshot) {
+      this.createMiddleSnapshotAtTarget();
+    }
+
+    dfs.rename(d1, foo_d1);
+    numDeletedAndModified += 1; // modify ./c/foo
+    dfs.rename(foo, new_foo);
+    dfs.rename(bar, bar1);
+    return numDeletedAndModified;
+  }
+
+  /**
+   * Test a case where create a dir, then mv a existed dir into it.
+   */
+  @Test
+  public void testSync8() throws Exception {
+    if (isSrcNotSameAsTgt) {
+      initData8(source);
+    }
+    initData8(target);
+    enableAndCreateFirstSnapshot();
+    int numDeletedModified = changeData8(target, false);
+
+    createSecondSnapshotAtTarget();
+
+    testAndVerify(numDeletedModified);
+  }
+
+  /**
+   * Test a case where create a dir, then mv a existed dir into it.
+   * The difference between this one and testSync8 is, this one
+   * also creates a snapshot s1.5 in between s1 and s2.
+   */
+  @Test
+  public void testSync9() throws Exception {
+    if (isSrcNotSameAsTgt) {
+      initData8(source);
+    }
+    initData8(target);
+    enableAndCreateFirstSnapshot();
+    int numDeletedModified = changeData8(target, true);
+
+    createSecondSnapshotAtTarget();
+
+    testAndVerify(numDeletedModified);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/412c4c9a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSyncReverseFromSource.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSyncReverseFromSource.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSyncReverseFromSource.java
new file mode 100644
index 0000000..30cc930
--- /dev/null
+++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSyncReverseFromSource.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.tools;
+
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Test the case "-rdiff s2 s1 src tgt".
+ */
+public class TestDistCpSyncReverseFromSource
+  extends TestDistCpSyncReverseBase {
+  /*
+   * Initialize the source path to /target.
+   * @see org.apache.hadoop.tools.TestDistCpSyncReverseBase#initSourcePath()
+   */
+  @Override
+  void initSourcePath() {
+    setSource(new Path("/source"));
+    setSrcNotSameAsTgt(true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/412c4c9a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSyncReverseFromTarget.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSyncReverseFromTarget.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSyncReverseFromTarget.java
new file mode 100644
index 0000000..c1fb24b
--- /dev/null
+++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSyncReverseFromTarget.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.tools;
+
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Test the case "-rdiff s2 s1 tgt tgt".
+ */
+public class TestDistCpSyncReverseFromTarget
+  extends TestDistCpSyncReverseBase {
+  /*
+   * Initialize the source path to /target.
+   * @see org.apache.hadoop.tools.TestDistCpSyncReverseBase#initSourcePath()
+   */
+  @Override
+  void initSourcePath() {
+    setSource(new Path("/target"));
+    setSrcNotSameAsTgt(false);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/412c4c9a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java
index 218de4e..efe4627 100644
--- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java
+++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java
@@ -387,7 +387,8 @@ public class TestOptionsParser {
     DistCpOptions option = new DistCpOptions(new Path("abc"), new Path("xyz"));
     String val = "DistCpOptions{atomicCommit=false, syncFolder=false, "
         + "deleteMissing=false, ignoreFailures=false, overwrite=false, "
-        + "append=false, useDiff=false, fromSnapshot=null, toSnapshot=null, "
+        + "append=false, useDiff=false, useRdiff=false, "
+        + "fromSnapshot=null, toSnapshot=null, "
         + "skipCRC=false, blocking=true, numListstatusThreads=0, maxMaps=20, "
         + "mapBandwidth=0.0, "
         + "copyStrategy='uniformsize', preserveStatus=[], "
@@ -701,61 +702,66 @@ public class TestOptionsParser {
     }
   }
 
-  @Test
-  public void testDiffOption() {
+  // Test -diff or -rdiff
+  private void testSnapshotDiffOption(boolean isDiff) {
+    final String optionStr = isDiff? "-diff" : "-rdiff";
+    final String optionLabel = isDiff?
+        DistCpOptionSwitch.DIFF.getConfigLabel() :
+          DistCpOptionSwitch.RDIFF.getConfigLabel();
     Configuration conf = new Configuration();
-    Assert.assertFalse(conf.getBoolean(DistCpOptionSwitch.DIFF.getConfigLabel(),
-        false));
+    Assert.assertFalse(conf.getBoolean(optionLabel, false));
 
     DistCpOptions options = OptionsParser.parse(new String[] { "-update",
-        "-diff", "s1", "s2",
+        optionStr, "s1", "s2",
         "hdfs://localhost:9820/source/first",
         "hdfs://localhost:9820/target/" });
     options.appendToConf(conf);
-    Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.DIFF.getConfigLabel(), false));
-    Assert.assertTrue(options.shouldUseDiff());
+    Assert.assertTrue(conf.getBoolean(optionLabel, false));
+    Assert.assertTrue(isDiff?
+        options.shouldUseDiff() : options.shouldUseRdiff());
     Assert.assertEquals("s1", options.getFromSnapshot());
     Assert.assertEquals("s2", options.getToSnapshot());
 
     options = OptionsParser.parse(new String[] {
-        "-diff", "s1", ".", "-update",
+        optionStr, "s1", ".", "-update",
         "hdfs://localhost:9820/source/first",
         "hdfs://localhost:9820/target/" });
     options.appendToConf(conf);
-    Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.DIFF.getConfigLabel(),
-        false));
-    Assert.assertTrue(options.shouldUseDiff());
+    Assert.assertTrue(conf.getBoolean(optionLabel, false));
+    Assert.assertTrue(isDiff?
+        options.shouldUseDiff() : options.shouldUseRdiff());
     Assert.assertEquals("s1", options.getFromSnapshot());
     Assert.assertEquals(".", options.getToSnapshot());
 
-    // -diff requires two option values
+    // -diff/-rdiff requires two option values
     try {
-      OptionsParser.parse(new String[] {"-diff", "s1", "-update",
+      OptionsParser.parse(new String[] {optionStr, "s1", "-update",
           "hdfs://localhost:9820/source/first",
           "hdfs://localhost:9820/target/" });
-      fail("-diff should fail with only one snapshot name");
+      fail(optionStr + " should fail with only one snapshot name");
     } catch (IllegalArgumentException e) {
       GenericTestUtils.assertExceptionContains(
           "Must provide both the starting and ending snapshot names", e);
     }
 
-    // make sure -diff is only valid when -update is specified
+    // make sure -diff/-rdiff is only valid when -update is specified
     try {
-      OptionsParser.parse(new String[] { "-diff", "s1", "s2",
+      OptionsParser.parse(new String[] {optionStr, "s1", "s2",
           "hdfs://localhost:9820/source/first",
           "hdfs://localhost:9820/target/" });
-      fail("-diff should fail if -update option is not specified");
+      fail(optionStr + " should fail if -update option is not specified");
     } catch (IllegalArgumentException e) {
       GenericTestUtils.assertExceptionContains(
-          "Diff is valid only with update options", e);
+          "-diff/-rdiff is valid only with -update option", e);
     }
 
     try {
       options = OptionsParser.parse(new String[] {
-          "-diff", "s1", "s2", "-update", "-delete",
+          optionStr, "s1", "s2", "-update", "-delete",
           "hdfs://localhost:9820/source/first",
           "hdfs://localhost:9820/target/" });
-      assertFalse("-delete should be ignored when -diff is specified",
+      assertFalse("-delete should be ignored when "
+          + optionStr + " is specified",
           options.shouldDeleteMissing());
     } catch (IllegalArgumentException e) {
       fail("Got unexpected IllegalArgumentException: " + e.getMessage());
@@ -763,27 +769,52 @@ public class TestOptionsParser {
 
     try {
       options = OptionsParser.parse(new String[] {
-          "-diff", "s1", "s2", "-delete",
+          optionStr, "s1", "s2", "-delete",
           "hdfs://localhost:9820/source/first",
           "hdfs://localhost:9820/target/" });
-      fail("-diff should fail if -update option is not specified");
+      fail(optionStr + " should fail if -update option is not specified");
     } catch (IllegalArgumentException e) {
       assertFalse("-delete should be ignored when -diff is specified",
           options.shouldDeleteMissing());
       GenericTestUtils.assertExceptionContains(
-          "Diff is valid only with update options", e);
+          "-diff/-rdiff is valid only with -update option", e);
     }
 
     try {
-      OptionsParser.parse(new String[] { "-diff", "s1", "s2",
+      OptionsParser.parse(new String[] {optionStr, "s1", "s2",
           "-delete", "-overwrite",
           "hdfs://localhost:9820/source/first",
           "hdfs://localhost:9820/target/" });
-      fail("-diff should fail if -update option is not specified");
+      fail(optionStr + " should fail if -update option is not specified");
     } catch (IllegalArgumentException e) {
       GenericTestUtils.assertExceptionContains(
-          "Diff is valid only with update options", e);
+          "-diff/-rdiff is valid only with -update option", e);
     }
+
+    final String optionStrOther = isDiff? "-rdiff" : "-diff";
+    try {
+      OptionsParser.parse(new String[] {
+          optionStr, "s1", "s2",
+          optionStrOther, "s2", "s1",
+          "-update",
+          "hdfs://localhost:9820/source/first",
+          "hdfs://localhost:9820/target/" });
+      fail(optionStr + " should fail if " + optionStrOther
+          + " is also specified");
+    } catch (IllegalArgumentException e) {
+      GenericTestUtils.assertExceptionContains(
+          "-diff and -rdiff are mutually exclusive", e);
+    }
+  }
+
+  @Test
+  public void testDiffOption() {
+    testSnapshotDiffOption(true);
+  }
+
+  @Test
+  public void testRdiffOption() {
+    testSnapshotDiffOption(false);
   }
 
   @Test


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


[31/50] hadoop git commit: HDFS-10883. 's behavior is not consistent in DFS after enabling EZ. Contributed by Yuanbo Liu.

Posted by um...@apache.org.
HDFS-10883. 's behavior is not consistent in DFS after enabling EZ. Contributed by Yuanbo Liu.


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

Branch: refs/heads/HDFS-10285
Commit: 0007360c3344b3485fa17de0fd2015a628de947c
Parents: 701c27a
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Oct 14 11:41:29 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Oct 14 11:41:29 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/DistributedFileSystem.java      |   5 +-
 .../src/site/markdown/TransparentEncryption.md  |   4 +-
 .../apache/hadoop/hdfs/TestEncryptionZones.java |  10 +-
 .../namenode/TestNestedEncryptionZones.java     | 175 +++++++++++++------
 4 files changed, 139 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0007360c/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 548815f..18a29e8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -2478,11 +2478,12 @@ public class DistributedFileSystem extends FileSystem {
    */
   @Override
   public Path getTrashRoot(Path path) {
-    if ((path == null) || path.isRoot() || !dfs.isHDFSEncryptionEnabled()) {
+    if ((path == null) || !dfs.isHDFSEncryptionEnabled()) {
       return super.getTrashRoot(path);
     }
 
-    String parentSrc = path.getParent().toUri().getPath();
+    String parentSrc = path.isRoot()?
+        path.toUri().getPath():path.getParent().toUri().getPath();
     try {
       EncryptionZone ez = dfs.getEZForPath(parentSrc);
       if ((ez != null)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0007360c/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/TransparentEncryption.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/TransparentEncryption.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/TransparentEncryption.md
index e7d9f1d..b82b400 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/TransparentEncryption.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/TransparentEncryption.md
@@ -242,12 +242,14 @@ By default, distcp compares checksums provided by the filesystem to verify that
 <a name="Rename_and_Trash_considerations"></a>Rename and Trash considerations
 ---------------------
 
-HDFS restricts file and directory renames across encryption zone boundaries. This includes renaming an encrypted file / directory into an unencrypted directory (e.g., `hdfs dfs mv /zone/encryptedFile /home/bob`), renaming an unencrypted file / directory into an encryption zone (e.g., `hdfs dfs mv /home/bob/unEncryptedFile /zone`), and renaming between two different encryption zones (e.g., `hdfs dfs mv /home/alice/zone1/foo /home/alice/zone2`). In these examples, `/zone`, `/home/alice/zone1`, and `/home/alice/zone2` are encryption zones, while `/home/bob` is not. A rename is only allowed if the source and destination paths are in the same encryption zone, or both paths are unencrypted (not in any encryption zone).
+HDFS restricts file and directory renames across encryption zone boundaries. This includes renaming an encrypted file / directory into an unencrypted directory (e.g., `hdfs dfs mv /zone/encryptedFile /home/bob`), renaming an unencrypted file or directory into an encryption zone (e.g., `hdfs dfs mv /home/bob/unEncryptedFile /zone`), and renaming between two different encryption zones (e.g., `hdfs dfs mv /home/alice/zone1/foo /home/alice/zone2`). In these examples, `/zone`, `/home/alice/zone1`, and `/home/alice/zone2` are encryption zones, while `/home/bob` is not. A rename is only allowed if the source and destination paths are in the same encryption zone, or both paths are unencrypted (not in any encryption zone).
 
 This restriction enhances security and eases system management significantly. All file EDEKs under an encryption zone are encrypted with the encryption zone key. Therefore, if the encryption zone key is compromised, it is important to identify all vulnerable files and re-encrypt them. This is fundamentally difficult if a file initially created in an encryption zone can be renamed to an arbitrary location in the filesystem.
 
 To comply with the above rule, each encryption zone has its own `.Trash` directory under the "zone directory". E.g., after `hdfs dfs rm /zone/encryptedFile`, `encryptedFile` will be moved to `/zone/.Trash`, instead of the `.Trash` directory under the user's home directory. When the entire encryption zone is deleted, the "zone directory" will be moved to the `.Trash` directory under the user's home directory.
 
+If the encryption zone is the root directory (e.g., `/` directory), the trash path of root directory is `/.Trash`, not the `.Trash` directory under the user's home directory, and the behavior of renaming sub-directories or sub-files in root directory will keep consistent with the behavior in a general encryption zone, such as `/zone` which is mentioned at the top of this section.
+
 The `crypto` command before Hadoop 2.8.0 does not provision the `.Trash` directory automatically. If an encryption zone is created before Hadoop 2.8.0, and then the cluster is upgraded to Hadoop 2.8.0 or above, the trash directory can be provisioned using `-provisionTrash` option (e.g., `hdfs crypto -provisionTrash -path /zone`).
 <a name="Attack_vectors"></a>Attack vectors
 --------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0007360c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
index 67019c3..8cce7ef 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
@@ -1566,7 +1566,8 @@ public class TestEncryptionZones {
   public void testRootDirEZTrash() throws Exception {
     final HdfsAdmin dfsAdmin =
         new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
-    dfsAdmin.createEncryptionZone(new Path("/"), TEST_KEY, NO_TRASH);
+    final Path rootDir = new Path("/");
+    dfsAdmin.createEncryptionZone(rootDir, TEST_KEY, NO_TRASH);
     final Path encFile = new Path("/encFile");
     final int len = 8192;
     DFSTestUtil.createFile(fs, encFile, len, (short) 1, 0xFEED);
@@ -1574,6 +1575,13 @@ public class TestEncryptionZones {
     clientConf.setLong(FS_TRASH_INTERVAL_KEY, 1);
     FsShell shell = new FsShell(clientConf);
     verifyShellDeleteWithTrash(shell, encFile);
+
+    // Trash path should be consistent
+    // if root path is an encryption zone
+    Path encFileTrash = shell.getCurrentTrashDir(encFile);
+    Path rootDirTrash = shell.getCurrentTrashDir(rootDir);
+    assertEquals("Root trash should be equal with ezFile trash",
+        encFileTrash, rootDirTrash);
   }
 
   @Test(timeout = 120000)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0007360c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNestedEncryptionZones.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNestedEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNestedEncryptionZones.java
index 13fc985..59d980c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNestedEncryptionZones.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNestedEncryptionZones.java
@@ -20,7 +20,9 @@ package org.apache.hadoop.hdfs.server.namenode;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -29,6 +31,8 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.ToolRunner;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.junit.Before;
@@ -53,16 +57,19 @@ public class TestNestedEncryptionZones {
 
   private final Path rootDir = new Path("/");
   private final Path rawDir = new Path("/.reserved/raw/");
-  private final Path topEZDir = new Path(rootDir, "topEZ");
-  private final Path nestedEZDir = new Path(topEZDir, "nestedEZ");
 
-  private final Path topEZBaseFile = new Path(rootDir, "topEZBaseFile");
-  private Path topEZFile = new Path(topEZDir, "file");
-  private Path topEZRawFile = new Path(rawDir, "topEZ/file");
+  private Path nestedEZBaseFile = new Path(rootDir, "nestedEZBaseFile");
+  private Path topEZBaseFile = new Path(rootDir, "topEZBaseFile");
+
+  private Path topEZDir;
+  private Path nestedEZDir;
+
+  private Path topEZFile;
+  private Path nestedEZFile;
+
+  private Path topEZRawFile;
+  private Path nestedEZRawFile;
 
-  private final Path nestedEZBaseFile = new Path(rootDir, "nestedEZBaseFile");
-  private Path nestedEZFile = new Path(nestedEZDir, "file");
-  private Path nestedEZRawFile = new Path(rawDir, "topEZ/nestedEZ/file");
 
   // File length
   private final int len = 8196;
@@ -92,6 +99,8 @@ public class TestNestedEncryptionZones {
     // Lower the batch size for testing
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES,
         2);
+    // enable trash for testing
+    conf.setLong(DFSConfigKeys.FS_TRASH_INTERVAL_KEY, 1);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
     Logger.getLogger(EncryptionZoneManager.class).setLevel(Level.TRACE);
     fs = cluster.getFileSystem();
@@ -100,24 +109,17 @@ public class TestNestedEncryptionZones {
     // Create test keys and EZs
     DFSTestUtil.createKey(TOP_EZ_KEY, cluster, conf);
     DFSTestUtil.createKey(NESTED_EZ_KEY, cluster, conf);
-    fs.mkdir(topEZDir, FsPermission.getDirDefault());
-    fs.createEncryptionZone(topEZDir, TOP_EZ_KEY);
-    fs.mkdir(nestedEZDir, FsPermission.getDirDefault());
-    fs.createEncryptionZone(nestedEZDir, NESTED_EZ_KEY);
-
-    DFSTestUtil.createFile(fs, topEZBaseFile, len, (short) 1, 0xFEED);
-    DFSTestUtil.createFile(fs, topEZFile, len, (short) 1, 0xFEED);
-    DFSTestUtil.createFile(fs, nestedEZBaseFile, len, (short) 1, 0xFEED);
-    DFSTestUtil.createFile(fs, nestedEZFile, len, (short) 1, 0xFEED);
   }
 
   @Test(timeout = 60000)
   public void testNestedEncryptionZones() throws Exception {
+    initTopEZDirAndNestedEZDir(new Path(rootDir, "topEZ"));
     verifyEncryption();
 
     // Restart NameNode to test if nested EZs can be loaded from edit logs
     cluster.restartNameNodes();
     cluster.waitActive();
+    fs = cluster.getFileSystem();
     verifyEncryption();
 
     // Checkpoint and restart NameNode, to test if nested EZs can be loaded
@@ -127,21 +129,88 @@ public class TestNestedEncryptionZones {
     fs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
     cluster.restartNameNodes();
     cluster.waitActive();
+    fs = cluster.getFileSystem();
     verifyEncryption();
 
-    Path renamedTopEZFile = new Path(topEZDir, "renamedFile");
-    Path renamedNestedEZFile = new Path(nestedEZDir, "renamedFile");
+    renameChildrenOfEZ();
+
+    // Verify that a non-nested EZ cannot be moved into another EZ
+    Path topEZ2Dir = new Path(rootDir, "topEZ2");
+    fs.mkdir(topEZ2Dir, FsPermission.getDirDefault());
+    fs.createEncryptionZone(topEZ2Dir, TOP_EZ_KEY);
     try {
-      fs.rename(topEZFile, renamedTopEZFile);
-      fs.rename(nestedEZFile, renamedNestedEZFile);
-    } catch (Exception e) {
-      fail("Should be able to rename files within the same EZ.");
+      fs.rename(topEZ2Dir, new Path(topEZDir, "topEZ2"));
+      fail("Shouldn't be able to move a non-nested EZ into another " +
+          "existing EZ.");
+    } catch (Exception e){
+      assertTrue(e.getMessage().contains(
+          "can't be moved into an encryption zone"));
     }
 
+    // Should be able to rename the root dir of an EZ.
+    fs.rename(topEZDir, new Path(rootDir, "newTopEZ"));
+
+    // Should be able to rename the nested EZ dir within the same top EZ.
+    fs.rename(new Path(rootDir, "newTopEZ/nestedEZ"),
+        new Path(rootDir, "newTopEZ/newNestedEZ"));
+  }
+
+  @Test(timeout = 60000)
+  public void testNestedEZWithRoot() throws Exception {
+    initTopEZDirAndNestedEZDir(rootDir);
+    verifyEncryption();
+
+    // test rename file
+    renameChildrenOfEZ();
+
+    final String currentUser =
+        UserGroupInformation.getCurrentUser().getShortUserName();
+    final Path suffixTrashPath = new Path(
+        FileSystem.TRASH_PREFIX, currentUser);
+
+    final Path rootTrash = fs.getTrashRoot(rootDir);
+    final Path topEZTrash = fs.getTrashRoot(topEZFile);
+    final Path nestedEZTrash = fs.getTrashRoot(nestedEZFile);
+
+    final Path expectedTopEZTrash = fs.makeQualified(
+        new Path(topEZDir, suffixTrashPath));
+    final Path expectedNestedEZTrash = fs.makeQualified(
+        new Path(nestedEZDir, suffixTrashPath));
+
+    assertEquals("Top ez trash should be " + expectedTopEZTrash,
+        expectedTopEZTrash, topEZTrash);
+    assertEquals("Root trash should be equal with TopEZFile trash",
+        topEZTrash, rootTrash);
+    assertEquals("Nested ez Trash should be " + expectedNestedEZTrash,
+        expectedNestedEZTrash, nestedEZTrash);
+
+    // delete rename file and test trash
+    FsShell shell = new FsShell(fs.getConf());
+    final Path topTrashFile = new Path(
+        shell.getCurrentTrashDir(topEZFile) + "/" + topEZFile);
+    final Path nestedTrashFile = new Path(
+        shell.getCurrentTrashDir(nestedEZFile) + "/" + nestedEZFile);
+
+    ToolRunner.run(shell, new String[]{"-rm", topEZFile.toString()});
+    ToolRunner.run(shell, new String[]{"-rm", nestedEZFile.toString()});
+
+    assertTrue("File not in trash : " + topTrashFile, fs.exists(topTrashFile));
+    assertTrue(
+        "File not in trash : " + nestedTrashFile, fs.exists(nestedTrashFile));
+  }
+
+  private void renameChildrenOfEZ() throws Exception{
+    Path renamedTopEZFile = new Path(topEZDir, "renamedFile");
+    Path renamedNestedEZFile = new Path(nestedEZDir, "renamedFile");
+
+    //Should be able to rename files within the same EZ.
+    fs.rename(topEZFile, renamedTopEZFile);
+    fs.rename(nestedEZFile, renamedNestedEZFile);
+
     topEZFile = renamedTopEZFile;
     nestedEZFile = renamedNestedEZFile;
-    topEZRawFile = new Path(rawDir, "topEZ/renamedFile");
-    nestedEZRawFile = new Path(rawDir, "topEZ/nestedEZ/renamedFile");
+    topEZRawFile = new Path(rawDir + topEZFile.toUri().getPath());
+    nestedEZRawFile = new Path(rawDir + nestedEZFile.toUri().getPath());
     verifyEncryption();
 
     // Verify that files in top EZ cannot be moved into the nested EZ, and
@@ -168,36 +237,40 @@ public class TestNestedEncryptionZones {
       fs.rename(nestedEZFile, new Path(rootDir, "movedNestedEZFile"));
       fail("Shouldn't be able to move the nested EZ out of the top EZ.");
     } catch (Exception e) {
-      assertTrue(e.getMessage().contains(
-          "can't be moved from an encryption zone"));
+      String exceptionMsg = e.getMessage();
+      assertTrue(exceptionMsg.contains(
+          "can't be moved from") && exceptionMsg.contains("encryption zone"));
     }
+  }
 
-    // Verify that a non-nested EZ cannot be moved into another EZ
-    Path topEZ2Dir = new Path(rootDir, "topEZ2");
-    fs.mkdir(topEZ2Dir, FsPermission.getDirDefault());
-    fs.createEncryptionZone(topEZ2Dir, TOP_EZ_KEY);
-    try {
-      fs.rename(topEZ2Dir, new Path(topEZDir, "topEZ2"));
-      fail("Shouldn't be able to move a non-nested EZ into another " +
-          "existing EZ.");
-    } catch (Exception e){
-      assertTrue(e.getMessage().contains(
-          "can't be moved into an encryption zone"));
-    }
+  private void initTopEZDirAndNestedEZDir(Path topPath) throws Exception {
 
-    try {
-      fs.rename(topEZDir, new Path(rootDir, "newTopEZDir"));
-    } catch (Exception e) {
-      fail("Should be able to rename the root dir of an EZ.");
-    }
+    // init fs root directory
+    fs.delete(rootDir, true);
+
+
+    // init top and nested path or file
+    topEZDir = topPath;
+    nestedEZDir = new Path(topEZDir, "nestedEZ");
+
+    topEZFile = new Path(topEZDir, "file");
+    nestedEZFile = new Path(nestedEZDir, "file");
+
+    topEZRawFile = new Path(rawDir + topEZFile.toUri().getPath());
+    nestedEZRawFile = new Path(rawDir + nestedEZFile.toUri().getPath());
+
+    // create ez zone
+    fs.mkdir(topEZDir, FsPermission.getDirDefault());
+    fs.createEncryptionZone(topEZDir, TOP_EZ_KEY);
+    fs.mkdir(nestedEZDir, FsPermission.getDirDefault());
+    fs.createEncryptionZone(nestedEZDir, NESTED_EZ_KEY);
+
+    // create files
+    DFSTestUtil.createFile(fs, topEZBaseFile, len, (short) 1, 0xFEED);
+    DFSTestUtil.createFile(fs, topEZFile, len, (short) 1, 0xFEED);
+    DFSTestUtil.createFile(fs, nestedEZBaseFile, len, (short) 1, 0xFEED);
+    DFSTestUtil.createFile(fs, nestedEZFile, len, (short) 1, 0xFEED);
 
-    try {
-      fs.rename(new Path(rootDir, "newTopEZDir/nestedEZDir"),
-          new Path(rootDir, "newTopEZDir/newNestedEZDir"));
-    } catch (Exception e) {
-      fail("Should be able to rename the nested EZ dir within " +
-          "the same top EZ.");
-    }
   }
 
   private void verifyEncryption() throws Exception {


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


[04/50] hadoop git commit: HADOOP-13705. Revert HADOOP-13534 Remove unused TrashPolicy#getInstance and initialize code.

Posted by um...@apache.org.
HADOOP-13705. Revert HADOOP-13534 Remove unused TrashPolicy#getInstance and initialize code.


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

Branch: refs/heads/HDFS-10285
Commit: 8a09bf7c19d9d2f6d6853d45e11b0d38c7c67f2a
Parents: 4b32b14
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Oct 11 13:46:07 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue Oct 11 13:46:07 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/TrashPolicy.java  | 30 ++++++++++++++++++++
 .../apache/hadoop/fs/TrashPolicyDefault.java    | 15 ++++++++++
 .../java/org/apache/hadoop/fs/TestTrash.java    |  4 +++
 3 files changed, 49 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a09bf7c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java
index bd99db4..157b9ab 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java
@@ -38,6 +38,17 @@ public abstract class TrashPolicy extends Configured {
 
   /**
    * Used to setup the trash policy. Must be implemented by all TrashPolicy
+   * implementations.
+   * @param conf the configuration to be used
+   * @param fs the filesystem to be used
+   * @param home the home directory
+   * @deprecated Use {@link #initialize(Configuration, FileSystem)} instead.
+   */
+  @Deprecated
+  public abstract void initialize(Configuration conf, FileSystem fs, Path home);
+
+  /**
+   * Used to setup the trash policy. Must be implemented by all TrashPolicy
    * implementations. Different from initialize(conf, fs, home), this one does
    * not assume trash always under /user/$USER due to HDFS encryption zone.
    * @param conf the configuration to be used
@@ -105,6 +116,25 @@ public abstract class TrashPolicy extends Configured {
    *
    * @param conf the configuration to be used
    * @param fs the file system to be used
+   * @param home the home directory
+   * @return an instance of TrashPolicy
+   * @deprecated Use {@link #getInstance(Configuration, FileSystem)} instead.
+   */
+  @Deprecated
+  public static TrashPolicy getInstance(Configuration conf, FileSystem fs, Path home) {
+    Class<? extends TrashPolicy> trashClass = conf.getClass(
+        "fs.trash.classname", TrashPolicyDefault.class, TrashPolicy.class);
+    TrashPolicy trash = ReflectionUtils.newInstance(trashClass, conf);
+    trash.initialize(conf, fs, home); // initialize TrashPolicy
+    return trash;
+  }
+
+  /**
+   * Get an instance of the configured TrashPolicy based on the value
+   * of the configuration parameter fs.trash.classname.
+   *
+   * @param conf the configuration to be used
+   * @param fs the file system to be used
    * @return an instance of TrashPolicy
    */
   public static TrashPolicy getInstance(Configuration conf, FileSystem fs)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a09bf7c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java
index f4a825c..72222be 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java
@@ -75,6 +75,21 @@ public class TrashPolicyDefault extends TrashPolicy {
     initialize(conf, fs);
   }
 
+  /**
+   * @deprecated Use {@link #initialize(Configuration, FileSystem)} instead.
+   */
+  @Override
+  @Deprecated
+  public void initialize(Configuration conf, FileSystem fs, Path home) {
+    this.fs = fs;
+    this.deletionInterval = (long)(conf.getFloat(
+        FS_TRASH_INTERVAL_KEY, FS_TRASH_INTERVAL_DEFAULT)
+        * MSECS_PER_MINUTE);
+    this.emptierInterval = (long)(conf.getFloat(
+        FS_TRASH_CHECKPOINT_INTERVAL_KEY, FS_TRASH_CHECKPOINT_INTERVAL_DEFAULT)
+        * MSECS_PER_MINUTE);
+   }
+
   @Override
   public void initialize(Configuration conf, FileSystem fs) {
     this.fs = fs;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a09bf7c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java
index 2aba01f..338aff6 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java
@@ -692,6 +692,10 @@ public class TestTrash extends TestCase {
     public TestTrashPolicy() { }
 
     @Override
+    public void initialize(Configuration conf, FileSystem fs, Path home) {
+    }
+
+    @Override
     public void initialize(Configuration conf, FileSystem fs) {
     }
 


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


[25/50] hadoop git commit: HADOOP-13669. Addendum patch for KMS Server should log exceptions before throwing.

Posted by um...@apache.org.
HADOOP-13669. Addendum patch for KMS Server should log exceptions before throwing.


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

Branch: refs/heads/HDFS-10285
Commit: ae51b11f7872eaac558acf00fd23f6d7b1841cfe
Parents: cf3f43e
Author: Xiao Chen <xi...@apache.org>
Authored: Thu Oct 13 22:32:08 2016 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Thu Oct 13 22:32:08 2016 -0700

----------------------------------------------------------------------
 .../hadoop-kms/dev-support/findbugsExcludeFile.xml            | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae51b11f/hadoop-common-project/hadoop-kms/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/dev-support/findbugsExcludeFile.xml b/hadoop-common-project/hadoop-kms/dev-support/findbugsExcludeFile.xml
index bc92ed7..78c4ca6 100644
--- a/hadoop-common-project/hadoop-kms/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-common-project/hadoop-kms/dev-support/findbugsExcludeFile.xml
@@ -38,4 +38,11 @@
     <Class name="org.apache.hadoop.crypto.key.kms.server.KMSWebApp"/>
     <Bug pattern="DM_EXIT"/>
   </Match>
+  <!--
+    KMS wants to log the exception before it's thrown to tomcat and disappear.
+  -->
+  <Match>
+    <Class name="org.apache.hadoop.crypto.key.kms.server.KMS"/>
+    <Bug pattern="REC"/>
+  </Match>
 </FindBugsFilter>


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


[16/50] hadoop git commit: HDFS-10986. DFSAdmin should log detailed error message if any. Contributed by MingLiang Liu

Posted by um...@apache.org.
HDFS-10986. DFSAdmin should log detailed error message if any. 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/12912540
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/12912540
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/12912540

Branch: refs/heads/HDFS-10285
Commit: 129125404244f35ee63b8f0491a095371685e9ba
Parents: 9454dc5
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Thu Oct 13 21:39:50 2016 +0530
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Thu Oct 13 22:05:00 2016 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |   8 +-
 .../apache/hadoop/hdfs/tools/TestDFSAdmin.java  | 106 +++++++++----------
 2 files changed, 51 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/12912540/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
index 32401dc..a60f24b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
@@ -936,8 +936,7 @@ public class DFSAdmin extends FsShell {
       System.out.println("Balancer bandwidth is " + bandwidth
           + " bytes per second.");
     } catch (IOException ioe) {
-      System.err.println("Datanode unreachable.");
-      return -1;
+      throw new IOException("Datanode unreachable. " + ioe, ioe);
     }
     return 0;
   }
@@ -2207,7 +2206,7 @@ public class DFSAdmin extends FsShell {
       dnProxy.evictWriters();
       System.out.println("Requested writer eviction to datanode " + dn);
     } catch (IOException ioe) {
-      return -1;
+      throw new IOException("Datanode unreachable. " + ioe, ioe);
     }
     return 0;
   }
@@ -2218,8 +2217,7 @@ public class DFSAdmin extends FsShell {
       DatanodeLocalInfo dnInfo = dnProxy.getDatanodeInfo();
       System.out.println(dnInfo.getDatanodeLocalReport());
     } catch (IOException ioe) {
-      System.err.println("Datanode unreachable.");
-      return -1;
+      throw new IOException("Datanode unreachable. " + ioe, ioe);
     }
     return 0;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12912540/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
index b49f73d..dca42ea 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.tools;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
@@ -79,6 +80,7 @@ public class TestDFSAdmin {
   @Before
   public void setUp() throws Exception {
     conf = new Configuration();
+    conf.setInt(IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 3);
     restartCluster();
 
     admin = new DFSAdmin();
@@ -116,7 +118,7 @@ public class TestDFSAdmin {
     if (cluster != null) {
       cluster.shutdown();
     }
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     cluster.waitActive();
     datanode = cluster.getDataNodes().get(0);
     namenode = cluster.getNameNode();
@@ -171,70 +173,58 @@ public class TestDFSAdmin {
   @Test(timeout = 30000)
   public void testGetDatanodeInfo() throws Exception {
     redirectStream();
-    final Configuration dfsConf = new HdfsConfiguration();
-    final int numDn = 2;
-
-    /* init cluster */
-    try (MiniDFSCluster miniCluster = new MiniDFSCluster.Builder(dfsConf)
-        .numDataNodes(numDn).build()) {
-
-      miniCluster.waitActive();
-      assertEquals(numDn, miniCluster.getDataNodes().size());
-      final DFSAdmin dfsAdmin = new DFSAdmin(dfsConf);
+    final DFSAdmin dfsAdmin = new DFSAdmin(conf);
 
-      /* init reused vars */
-      List<String> outs = null;
-      int ret;
-
-      /**
-       * test erroneous run
-       */
+    for (int i = 0; i < cluster.getDataNodes().size(); i++) {
       resetStream();
-      outs = Lists.newArrayList();
-
-      /* invoke getDatanodeInfo */
-      ret = ToolRunner.run(
-          dfsAdmin,
-          new String[] {"-getDatanodeInfo", "128.0.0.1:1234"});
+      final DataNode dn = cluster.getDataNodes().get(i);
+      final String addr = String.format(
+          "%s:%d",
+          dn.getXferAddress().getHostString(),
+          dn.getIpcPort());
+      final int ret = ToolRunner.run(dfsAdmin,
+          new String[]{"-getDatanodeInfo", addr});
+      assertEquals(0, ret);
 
       /* collect outputs */
+      final List<String> outs = Lists.newArrayList();
       scanIntoList(out, outs);
-
       /* verify results */
+      assertEquals(
+          "One line per DataNode like: Uptime: XXX, Software version: x.y.z,"
+              + " Config version: core-x.y.z,hdfs-x",
+          1, outs.size());
+      assertThat(outs.get(0),
+          is(allOf(containsString("Uptime:"),
+              containsString("Software version"),
+              containsString("Config version"))));
+    }
+  }
+
+  /**
+   * Test that if datanode is not reachable, some DFSAdmin commands will fail
+   * elegantly with non-zero ret error code along with exception error message.
+   */
+  @Test(timeout = 60000)
+  public void testDFSAdminUnreachableDatanode() throws Exception {
+    redirectStream();
+    final DFSAdmin dfsAdmin = new DFSAdmin(conf);
+    for (String command : new String[]{"-getDatanodeInfo",
+        "-evictWriters", "-getBalancerBandwidth"}) {
+      // Connecting to Xfer port instead of IPC port will get
+      // Datanode unreachable. java.io.EOFException
+      final String dnDataAddr = datanode.getXferAddress().getHostString() + ":"
+          + datanode.getXferPort();
+      resetStream();
+      final List<String> outs = Lists.newArrayList();
+      final int ret = ToolRunner.run(dfsAdmin,
+          new String[]{command, dnDataAddr});
       assertEquals(-1, ret);
-      assertTrue("Unexpected getDatanodeInfo stdout", outs.isEmpty());
-
-      /**
-       * test normal run
-       */
-      for (int i = 0; i < numDn; i++) {
-        resetStream();
-        final DataNode dn = miniCluster.getDataNodes().get(i);
-
-        /* invoke getDatanodeInfo */
-        final String addr = String.format(
-            "%s:%d",
-            dn.getXferAddress().getHostString(),
-            dn.getIpcPort());
-        ret = ToolRunner.run(
-            dfsAdmin,
-            new String[] {"-getDatanodeInfo", addr});
-
-        /* collect outputs */
-        outs = Lists.newArrayList();
-        scanIntoList(out, outs);
-
-        /* verify results */
-        assertEquals(0, ret);
-        assertEquals(
-            "One line per DataNode like: Uptime: XXX, Software version: x.y.z,"
-                + " Config version: core-x.y.z,hdfs-x",
-            1, outs.size());
-        assertThat(outs.get(0),
-            is(allOf(containsString("Uptime:"),
-                containsString("Software version"),
-                containsString("Config version"))));
-      }
+
+      scanIntoList(out, outs);
+      assertTrue("Unexpected " + command + " stdout: " + out, outs.isEmpty());
+      assertTrue("Unexpected " + command + " stderr: " + err,
+          err.toString().contains("Exception"));
     }
   }
 


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


[19/50] hadoop git commit: HADOOP-13710. Supress CachingGetSpaceUsed from logging interrupted exception stacktrace. Contributed by Hanisha Koneru.

Posted by um...@apache.org.
HADOOP-13710. Supress CachingGetSpaceUsed from logging interrupted exception stacktrace. Contributed by Hanisha Koneru.


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

Branch: refs/heads/HDFS-10285
Commit: 008122b3c927767ac96dc876124bc591e10c9df4
Parents: 9097e2e
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Oct 13 11:37:03 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Oct 13 11:37:03 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/008122b3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java
index 505f76d..a2b6980 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java
@@ -177,7 +177,8 @@ public abstract class CachingGetSpaceUsed implements Closeable, GetSpaceUsed {
           // update the used variable
           spaceUsed.refresh();
         } catch (InterruptedException e) {
-          LOG.warn("Thread Interrupted waiting to refresh disk information", e);
+          LOG.warn("Thread Interrupted waiting to refresh disk information: "
+              + e.getMessage());
           Thread.currentThread().interrupt();
         }
       }


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


[46/50] hadoop git commit: HDFS-10922. Adding additional unit tests for Trash (II). Contributed by Weiwei Yang.

Posted by um...@apache.org.
HDFS-10922. Adding additional unit tests for Trash (II). Contributed by Weiwei Yang.


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

Branch: refs/heads/HDFS-10285
Commit: 8fd4c37c45585d761d279f2f6032ff9c6c049895
Parents: b671ee6
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Mon Oct 17 08:22:31 2016 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Mon Oct 17 14:21:36 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  40 +++++
 .../apache/hadoop/hdfs/TestDFSPermission.java   |  30 ++--
 .../org/apache/hadoop/hdfs/TestHDFSTrash.java   | 145 ++++++++++++++++++-
 3 files changed, 189 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd4c37c/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 f80cd78..963aaa6 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
@@ -70,6 +70,7 @@ 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.Strings;
 import com.google.common.base.Supplier;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
@@ -2014,4 +2015,43 @@ public class DFSTestUtil {
       }
     }, 1000, 60000);
   }
+
+  /**
+   * Close current file system and create a new instance as given
+   * {@link UserGroupInformation}.
+   */
+  public static FileSystem login(final FileSystem fs,
+      final Configuration conf, final UserGroupInformation ugi)
+          throws IOException, InterruptedException {
+    if (fs != null) {
+      fs.close();
+    }
+    return DFSTestUtil.getFileSystemAs(ugi, conf);
+  }
+
+  /**
+   * Test if the given {@link FileStatus} user, group owner and its permission
+   * are expected, throw {@link AssertionError} if any value is not expected.
+   */
+  public static void verifyFilePermission(FileStatus stat, String owner,
+      String group, FsAction u, FsAction g, FsAction o) {
+    if(stat != null) {
+      if(!Strings.isNullOrEmpty(owner)) {
+        assertEquals(owner, stat.getOwner());
+      }
+      if(!Strings.isNullOrEmpty(group)) {
+        assertEquals(group, stat.getGroup());
+      }
+      FsPermission permission = stat.getPermission();
+      if(u != null) {
+        assertEquals(u, permission.getUserAction());
+      }
+      if (g != null) {
+        assertEquals(g, permission.getGroupAction());
+      }
+      if (o != null) {
+        assertEquals(o, permission.getOtherAction());
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd4c37c/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 d0d00e5..2705e67 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
@@ -305,7 +305,7 @@ public class TestDFSPermission {
       fs.mkdirs(rootDir);
       fs.setPermission(rootDir, new FsPermission((short) 0777));
 
-      login(USER1);
+      fs = DFSTestUtil.login(fs, conf, USER1);
       fs.mkdirs(user1Dir);
       fs.setPermission(user1Dir, new FsPermission((short) 0755));
       fs.setOwner(user1Dir, USER1.getShortUserName(), GROUP2_NAME);
@@ -318,7 +318,7 @@ public class TestDFSPermission {
         // login as user2, attempt to delete /BSS/user1
         // this should fail because user2 has no permission to
         // its sub directory.
-        login(USER2);
+        fs = DFSTestUtil.login(fs, conf, USER2);
         fs.delete(user1Dir, true);
         fail("User2 should not be allowed to delete user1's dir.");
       } catch (AccessControlException e) {
@@ -331,7 +331,7 @@ public class TestDFSPermission {
       assertTrue(fs.exists(user1Dir));
 
       try {
-        login(SUPERUSER);
+        fs = DFSTestUtil.login(fs, conf, SUPERUSER);
         Trash trash = new Trash(fs, conf);
         Path trashRoot = trash.getCurrentTrashDir(user1Dir);
         while(true) {
@@ -346,7 +346,7 @@ public class TestDFSPermission {
         // login as user2, attempt to move /BSS/user1 to trash
         // this should also fail otherwise the directory will be
         // removed by trash emptier (emptier is running by superuser)
-        login(USER2);
+        fs = DFSTestUtil.login(fs, conf, USER2);
         Trash userTrash = new Trash(fs, conf);
         assertTrue(userTrash.isEnabled());
         userTrash.moveToTrash(user1Dir);
@@ -363,7 +363,7 @@ public class TestDFSPermission {
       // ensure /BSS/user1 still exists
       assertEquals(fs.exists(user1Dir), true);
     } finally {
-      login(SUPERUSER);
+      fs = DFSTestUtil.login(fs, conf, SUPERUSER);
       fs.delete(rootDir, true);
       conf.set(CommonConfigurationKeys.FS_TRASH_INTERVAL_KEY, "0");
     }
@@ -405,7 +405,7 @@ public class TestDFSPermission {
     setOwner(FILE_DIR_PATH, USER1.getShortUserName(), GROUP3_NAME, false);
 
     // case 3: user1 changes FILE_DIR_PATH's owner to be user2
-    login(USER1);
+    fs = DFSTestUtil.login(fs, conf, USER1);
     setOwner(FILE_DIR_PATH, USER2.getShortUserName(), null, true);
 
     // case 4: user1 changes FILE_DIR_PATH's group to be group1 which it belongs
@@ -417,14 +417,14 @@ public class TestDFSPermission {
     setOwner(FILE_DIR_PATH, null, GROUP3_NAME, true);
 
     // case 6: user2 (non-owner) changes FILE_DIR_PATH's group to be group3
-    login(USER2);
+    fs = DFSTestUtil.login(fs, conf, USER2);
     setOwner(FILE_DIR_PATH, null, GROUP3_NAME, true);
 
     // case 7: user2 (non-owner) changes FILE_DIR_PATH's user to be user2
     setOwner(FILE_DIR_PATH, USER2.getShortUserName(), null, true);
 
     // delete the file/directory
-    login(SUPERUSER);
+    fs = DFSTestUtil.login(fs, conf, SUPERUSER);
     fs.delete(FILE_DIR_PATH, true);
   }
 
@@ -666,7 +666,7 @@ public class TestDFSPermission {
       short[] filePermission, Path[] parentDirs, Path[] files, Path[] dirs)
       throws Exception {
     boolean[] isDirEmpty = new boolean[NUM_TEST_PERMISSIONS];
-    login(SUPERUSER);
+    fs = DFSTestUtil.login(fs, conf, SUPERUSER);
     for (int i = 0; i < NUM_TEST_PERMISSIONS; i++) {
       create(OpType.CREATE, files[i]);
       create(OpType.MKDIRS, dirs[i]);
@@ -682,7 +682,7 @@ public class TestDFSPermission {
       isDirEmpty[i] = (fs.listStatus(dirs[i]).length == 0);
     }
 
-    login(ugi);
+    fs = DFSTestUtil.login(fs, conf, ugi);
     for (int i = 0; i < NUM_TEST_PERMISSIONS; i++) {
       testCreateMkdirs(ugi, new Path(parentDirs[i], FILE_DIR_NAME),
           ancestorPermission[i], parentPermission[i]);
@@ -1237,16 +1237,6 @@ public class TestDFSPermission {
     ddpv.verifyPermission(ugi);
   }
 
-  /* log into dfs as the given user */
-  private void login(UserGroupInformation ugi) throws IOException,
-      InterruptedException {
-    if (fs != null) {
-      fs.close();
-    }
-
-    fs = DFSTestUtil.getFileSystemAs(ugi, conf);
-  }
-
   /* test non-existent file */
   private void checkNonExistentFile() {
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd4c37c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSTrash.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSTrash.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSTrash.java
index ad4d600..b81cdb1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSTrash.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSTrash.java
@@ -17,27 +17,79 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import java.io.IOException;
+import java.util.UUID;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.TestTrash;
-
+import org.apache.hadoop.fs.Trash;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 /**
  * Test trash using HDFS
  */
 public class TestHDFSTrash {
+
+  public static final Log LOG = LogFactory.getLog(TestHDFSTrash.class);
+
   private static MiniDFSCluster cluster = null;
+  private static FileSystem fs;
+  private static Configuration conf = new HdfsConfiguration();
+
+  private final static Path TEST_ROOT = new Path("/TestHDFSTrash-ROOT");
+  private final static Path TRASH_ROOT = new Path("/TestHDFSTrash-TRASH");
+
+  final private static String GROUP1_NAME = "group1";
+  final private static String GROUP2_NAME = "group2";
+  final private static String GROUP3_NAME = "group3";
+  final private static String USER1_NAME = "user1";
+  final private static String USER2_NAME = "user2";
+
+  private static UserGroupInformation superUser;
+  private static UserGroupInformation user1;
+  private static UserGroupInformation user2;
 
   @BeforeClass
   public static void setUp() throws Exception {
-    Configuration conf = new HdfsConfiguration();
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
+    fs = FileSystem.get(conf);
+
+    superUser = UserGroupInformation.getCurrentUser();
+    user1 = UserGroupInformation.createUserForTesting(USER1_NAME,
+        new String[] {GROUP1_NAME, GROUP2_NAME});
+    user2 = UserGroupInformation.createUserForTesting(USER2_NAME,
+        new String[] {GROUP2_NAME, GROUP3_NAME});
+
+    // Init test and trash root dirs in HDFS
+    fs.mkdirs(TEST_ROOT);
+    fs.setPermission(TEST_ROOT, new FsPermission((short) 0777));
+    DFSTestUtil.verifyFilePermission(
+        fs.getFileStatus(TEST_ROOT),
+        superUser.getShortUserName(),
+        null, FsAction.ALL, FsAction.ALL, FsAction.ALL);
+
+    fs.mkdirs(TRASH_ROOT);
+    fs.setPermission(TRASH_ROOT, new FsPermission((short) 0777));
+    DFSTestUtil.verifyFilePermission(
+        fs.getFileStatus(TRASH_ROOT),
+        superUser.getShortUserName(),
+        null, FsAction.ALL, FsAction.ALL, FsAction.ALL);
   }
 
   @AfterClass
@@ -52,9 +104,90 @@ public class TestHDFSTrash {
 
   @Test
   public void testNonDefaultFS() throws IOException {
-    FileSystem fs = cluster.getFileSystem();
-    Configuration conf = fs.getConf();
-    conf.set(DFSConfigKeys.FS_DEFAULT_NAME_KEY, fs.getUri().toString());
-    TestTrash.trashNonDefaultFS(conf);
+    FileSystem fileSystem = cluster.getFileSystem();
+    Configuration config = fileSystem.getConf();
+    config.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY,
+        fileSystem.getUri().toString());
+    TestTrash.trashNonDefaultFS(config);
+  }
+
+  @Test
+  public void testHDFSTrashPermission() throws IOException {
+    FileSystem fileSystem = cluster.getFileSystem();
+    Configuration config = fileSystem.getConf();
+    config.set(CommonConfigurationKeys.FS_TRASH_INTERVAL_KEY, "0.2");
+    TestTrash.verifyTrashPermission(fileSystem, config);
+  }
+
+  @Test
+  public void testMoveEmptyDirToTrash() throws IOException {
+    FileSystem fileSystem = cluster.getFileSystem();
+    Configuration config = fileSystem.getConf();
+    config.set(CommonConfigurationKeys.FS_TRASH_INTERVAL_KEY, "1");
+    TestTrash.verifyMoveEmptyDirToTrash(fileSystem, config);
+  }
+
+  @Test
+  public void testDeleteTrash() throws Exception {
+    Configuration testConf = new Configuration(conf);
+    testConf.set(CommonConfigurationKeys.FS_TRASH_INTERVAL_KEY, "10");
+
+    Path user1Tmp = new Path(TEST_ROOT, "test-del-u1");
+    Path user2Tmp = new Path(TEST_ROOT, "test-del-u2");
+
+    // login as user1, move something to trash
+    // verify user1 can remove its own trash dir
+    fs = DFSTestUtil.login(fs, testConf, user1);
+    fs.mkdirs(user1Tmp);
+    Trash u1Trash = getPerUserTrash(user1, fs, testConf);
+    Path u1t = u1Trash.getCurrentTrashDir(user1Tmp);
+    assertTrue(String.format("Failed to move %s to trash", user1Tmp),
+        u1Trash.moveToTrash(user1Tmp));
+    assertTrue(
+        String.format(
+            "%s should be allowed to remove its own trash directory %s",
+            user1.getUserName(), u1t),
+        fs.delete(u1t, true));
+    assertFalse(fs.exists(u1t));
+
+    // login as user2, move something to trash
+    fs = DFSTestUtil.login(fs, testConf, user2);
+    fs.mkdirs(user2Tmp);
+    Trash u2Trash = getPerUserTrash(user2, fs, testConf);
+    u2Trash.moveToTrash(user2Tmp);
+    Path u2t = u2Trash.getCurrentTrashDir(user2Tmp);
+
+    try {
+      // user1 should not be able to remove user2's trash dir
+      fs = DFSTestUtil.login(fs, testConf, user1);
+      fs.delete(u2t, true);
+      fail(String.format("%s should not be able to remove %s trash directory",
+              USER1_NAME, USER2_NAME));
+    } catch (AccessControlException e) {
+      assertTrue(e instanceof AccessControlException);
+      assertTrue("Permission denied messages must carry the username",
+          e.getMessage().contains(USER1_NAME));
+    }
+  }
+
+  /**
+   * Return a {@link Trash} instance using giving configuration.
+   * The trash root directory is set to an unique directory under
+   * {@link #TRASH_ROOT}. Use this method to isolate trash
+   * directories for different users.
+   */
+  private Trash getPerUserTrash(UserGroupInformation ugi,
+      FileSystem fileSystem, Configuration config) throws IOException {
+    // generate an unique path per instance
+    UUID trashId = UUID.randomUUID();
+    StringBuffer sb = new StringBuffer()
+        .append(ugi.getUserName())
+        .append("-")
+        .append(trashId.toString());
+    Path userTrashRoot = new Path(TRASH_ROOT, sb.toString());
+    FileSystem spyUserFs = Mockito.spy(fileSystem);
+    Mockito.when(spyUserFs.getTrashRoot(Mockito.any()))
+        .thenReturn(userTrashRoot);
+    return new Trash(spyUserFs, config);
   }
 }


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


[23/50] hadoop git commit: HADOOP-13024. Distcp with -delete feature on raw data not implemented. Contributed by Mavin Martin.

Posted by um...@apache.org.
HADOOP-13024. Distcp with -delete feature on raw data not implemented. Contributed by Mavin Martin.


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

Branch: refs/heads/HDFS-10285
Commit: 0a85d079838f532a13ca237300386d1b3bc1b178
Parents: 8c721aa
Author: Jing Zhao <ji...@apache.org>
Authored: Thu Oct 13 13:24:37 2016 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Thu Oct 13 13:24:54 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/tools/DistCpConstants.java    | 12 +++++-
 .../hadoop/tools/mapred/CopyCommitter.java      |  5 ++-
 .../hadoop/tools/TestDistCpWithRawXAttrs.java   | 45 +++++++++-----------
 .../hadoop/tools/util/DistCpTestUtils.java      | 32 ++++++++------
 4 files changed, 56 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a85d079/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
index 96f364c..6171aa9 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
@@ -18,6 +18,8 @@ package org.apache.hadoop.tools;
  * limitations under the License.
  */
 
+import org.apache.hadoop.fs.Path;
+
 /**
  * Utility class to hold commonly used constants.
  */
@@ -125,9 +127,17 @@ public class DistCpConstants {
   public static final int SPLIT_RATIO_DEFAULT  = 2;
 
   /**
+   * Constants for NONE file deletion
+   */
+  public static final String NONE_PATH_NAME = "/NONE";
+  public static final Path NONE_PATH = new Path(NONE_PATH_NAME);
+  public static final Path RAW_NONE_PATH = new Path(
+      DistCpConstants.HDFS_RESERVED_RAW_DIRECTORY_NAME + NONE_PATH_NAME);
+
+  /**
    * Value of reserved raw HDFS directory when copying raw.* xattrs.
    */
-  static final String HDFS_RESERVED_RAW_DIRECTORY_NAME = "/.reserved/raw";
+  public static final String HDFS_RESERVED_RAW_DIRECTORY_NAME = "/.reserved/raw";
 
   static final String HDFS_DISTCP_DIFF_DIRECTORY_NAME = ".distcp.diff.tmp";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a85d079/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyCommitter.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyCommitter.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyCommitter.java
index 6d2fef5..dd653b2 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyCommitter.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyCommitter.java
@@ -238,7 +238,10 @@ public class CopyCommitter extends FileOutputCommitter {
     List<Path> targets = new ArrayList<Path>(1);
     Path targetFinalPath = new Path(conf.get(DistCpConstants.CONF_LABEL_TARGET_FINAL_PATH));
     targets.add(targetFinalPath);
-    DistCpOptions options = new DistCpOptions(targets, new Path("/NONE"));
+    Path resultNonePath = Path.getPathWithoutSchemeAndAuthority(targetFinalPath)
+        .toString().startsWith(DistCpConstants.HDFS_RESERVED_RAW_DIRECTORY_NAME)
+        ? DistCpConstants.RAW_NONE_PATH : DistCpConstants.NONE_PATH;
+    DistCpOptions options = new DistCpOptions(targets, resultNonePath);
     //
     // Set up options to be the same from the CopyListing.buildListing's perspective,
     // so to collect similar listings as when doing the copy

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a85d079/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpWithRawXAttrs.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpWithRawXAttrs.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpWithRawXAttrs.java
index 5aef51a..8adc2cf 100644
--- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpWithRawXAttrs.java
+++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpWithRawXAttrs.java
@@ -82,14 +82,7 @@ public class TestDistCpWithRawXAttrs {
     final String relDst = "/./.reserved/../.reserved/raw/../raw/dest/../dest";
     doTestPreserveRawXAttrs(relSrc, relDst, "-px", true, true,
         DistCpConstants.SUCCESS);
-    doTestPreserveRawXAttrs(rootedSrcName, rootedDestName, "-px",
-        false, true, DistCpConstants.SUCCESS);
-    doTestPreserveRawXAttrs(rootedSrcName, rawDestName, "-px",
-        false, true, DistCpConstants.INVALID_ARGUMENT);
-    doTestPreserveRawXAttrs(rawSrcName, rootedDestName, "-px",
-        false, true, DistCpConstants.INVALID_ARGUMENT);
-    doTestPreserveRawXAttrs(rawSrcName, rawDestName, "-px",
-        true, true, DistCpConstants.SUCCESS);
+    doTestStandardPreserveRawXAttrs("-px", true);
     final Path savedWd = fs.getWorkingDirectory();
     try {
       fs.setWorkingDirectory(new Path("/.reserved/raw"));
@@ -103,27 +96,18 @@ public class TestDistCpWithRawXAttrs {
   /* Test that XAttrs are not preserved and raw.* are when appropriate. */
   @Test
   public void testPreserveRawXAttrs2() throws Exception {
-    doTestPreserveRawXAttrs(rootedSrcName, rootedDestName, "-p",
-        false, false, DistCpConstants.SUCCESS);
-    doTestPreserveRawXAttrs(rootedSrcName, rawDestName, "-p",
-        false, false, DistCpConstants.INVALID_ARGUMENT);
-    doTestPreserveRawXAttrs(rawSrcName, rootedDestName, "-p",
-        false, false, DistCpConstants.INVALID_ARGUMENT);
-    doTestPreserveRawXAttrs(rawSrcName, rawDestName, "-p",
-        true, false, DistCpConstants.SUCCESS);
+    doTestStandardPreserveRawXAttrs("-p", false);
   }
 
   /* Test that XAttrs are not preserved and raw.* are when appropriate. */
   @Test
   public void testPreserveRawXAttrs3() throws Exception {
-    doTestPreserveRawXAttrs(rootedSrcName, rootedDestName, null,
-        false, false, DistCpConstants.SUCCESS);
-    doTestPreserveRawXAttrs(rootedSrcName, rawDestName, null,
-        false, false, DistCpConstants.INVALID_ARGUMENT);
-    doTestPreserveRawXAttrs(rawSrcName, rootedDestName, null,
-        false, false, DistCpConstants.INVALID_ARGUMENT);
-    doTestPreserveRawXAttrs(rawSrcName, rawDestName, null,
-        true, false, DistCpConstants.SUCCESS);
+    doTestStandardPreserveRawXAttrs(null, false);
+  }
+
+  @Test
+  public void testPreserveRawXAttrs4() throws Exception {
+    doTestStandardPreserveRawXAttrs("-update -delete", false);
   }
 
   private static Path[] pathnames = { new Path("dir1"),
@@ -145,6 +129,19 @@ public class TestDistCpWithRawXAttrs {
     }
   }
 
+  private void doTestStandardPreserveRawXAttrs(String options,
+      boolean expectUser)
+      throws Exception {
+    doTestPreserveRawXAttrs(rootedSrcName, rootedDestName, options,
+        false, expectUser, DistCpConstants.SUCCESS);
+    doTestPreserveRawXAttrs(rootedSrcName, rawDestName, options,
+        false, expectUser, DistCpConstants.INVALID_ARGUMENT);
+    doTestPreserveRawXAttrs(rawSrcName, rootedDestName, options,
+        false, expectUser, DistCpConstants.INVALID_ARGUMENT);
+    doTestPreserveRawXAttrs(rawSrcName, rawDestName, options,
+        true, expectUser, DistCpConstants.SUCCESS);
+  }
+
   private void doTestPreserveRawXAttrs(String src, String dest,
       String preserveOpts, boolean expectRaw, boolean expectUser,
       int expectedExitCode) throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a85d079/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/DistCpTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/DistCpTestUtils.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/DistCpTestUtils.java
index 2721638..624f7d5 100644
--- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/DistCpTestUtils.java
+++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/DistCpTestUtils.java
@@ -18,21 +18,20 @@
 
 package org.apache.hadoop.tools.util;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Map.Entry;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-
 import org.apache.hadoop.tools.DistCp;
 import org.apache.hadoop.util.ToolRunner;
 
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 /**
  * Utility class for DistCpTests
  */
@@ -79,10 +78,19 @@ public class DistCpTestUtils {
   public static void assertRunDistCp(int exitCode, String src, String dst,
       String options, Configuration conf)
       throws Exception {
+    assertRunDistCp(exitCode, src, dst,
+        options == null ? new String[0] : options.trim().split(" "), conf);
+  }
+
+  private static void assertRunDistCp(int exitCode, String src, String dst,
+      String[] options, Configuration conf)
+      throws Exception {
     DistCp distCp = new DistCp(conf, null);
-    String[] optsArr = options == null ?
-        new String[] { src, dst } :
-        new String[] { options, src, dst };
+    String[] optsArr = new String[options.length + 2];
+    System.arraycopy(options, 0, optsArr, 0, options.length);
+    optsArr[optsArr.length - 2] = src;
+    optsArr[optsArr.length - 1] = dst;
+
     assertEquals(exitCode,
         ToolRunner.run(conf, distCp, optsArr));
   }


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


[27/50] hadoop git commit: HADOOP-13417. Fix javac and checkstyle warnings in hadoop-auth package.

Posted by um...@apache.org.
HADOOP-13417. Fix javac and checkstyle warnings in hadoop-auth package.


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

Branch: refs/heads/HDFS-10285
Commit: 5a5a724731b74df9eed2de5f3370bcb8023fa2eb
Parents: d9f73f1
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Oct 14 14:45:55 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Oct 14 14:45:55 2016 +0900

----------------------------------------------------------------------
 .../client/AuthenticatorTestCase.java           | 49 ++++++++++++--------
 1 file changed, 29 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a5a7247/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/AuthenticatorTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/AuthenticatorTestCase.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/AuthenticatorTestCase.java
index 8f35e13..35e40d8 100644
--- a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/AuthenticatorTestCase.java
+++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/AuthenticatorTestCase.java
@@ -20,14 +20,15 @@ import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.http.HttpResponse;
 import org.apache.http.auth.AuthScope;
 import org.apache.http.auth.Credentials;
+import org.apache.http.client.CredentialsProvider;
 import org.apache.http.client.HttpClient;
 import org.apache.http.client.methods.HttpGet;
 import org.apache.http.client.methods.HttpPost;
 import org.apache.http.client.methods.HttpUriRequest;
-import org.apache.http.client.params.AuthPolicy;
 import org.apache.http.entity.InputStreamEntity;
-import org.apache.http.impl.auth.SPNegoSchemeFactory;
-import org.apache.http.impl.client.SystemDefaultHttpClient;
+import org.apache.http.impl.auth.SPNegoScheme;
+import org.apache.http.impl.client.BasicCredentialsProvider;
+import org.apache.http.impl.client.HttpClientBuilder;
 import org.apache.http.util.EntityUtils;
 import org.mortbay.jetty.Server;
 import org.mortbay.jetty.servlet.Context;
@@ -53,6 +54,7 @@ import java.net.ServerSocket;
 import java.net.URL;
 import java.security.Principal;
 import java.util.Properties;
+
 import org.junit.Assert;
 
 public class AuthenticatorTestCase {
@@ -241,22 +243,29 @@ public class AuthenticatorTestCase {
     }
   }
 
-  private SystemDefaultHttpClient getHttpClient() {
-    final SystemDefaultHttpClient httpClient = new SystemDefaultHttpClient();
-    httpClient.getAuthSchemes().register(AuthPolicy.SPNEGO, new SPNegoSchemeFactory(true));
-     Credentials use_jaas_creds = new Credentials() {
-       public String getPassword() {
-         return null;
-       }
-
-       public Principal getUserPrincipal() {
-         return null;
-       }
-     };
-
-     httpClient.getCredentialsProvider().setCredentials(
-       AuthScope.ANY, use_jaas_creds);
-     return httpClient;
+  private HttpClient getHttpClient() {
+    HttpClientBuilder builder = HttpClientBuilder.create();
+    // Register auth schema
+    builder.setDefaultAuthSchemeRegistry(
+        s-> httpContext -> new SPNegoScheme(true, true)
+    );
+
+    Credentials useJaasCreds = new Credentials() {
+      public String getPassword() {
+        return null;
+      }
+      public Principal getUserPrincipal() {
+        return null;
+      }
+    };
+
+    CredentialsProvider jaasCredentialProvider
+        = new BasicCredentialsProvider();
+    jaasCredentialProvider.setCredentials(AuthScope.ANY, useJaasCreds);
+    // Set credential provider
+    builder.setDefaultCredentialsProvider(jaasCredentialProvider);
+
+    return builder.build();
   }
 
   private void doHttpClientRequest(HttpClient httpClient, HttpUriRequest request) throws Exception {
@@ -273,7 +282,7 @@ public class AuthenticatorTestCase {
   protected void _testAuthenticationHttpClient(Authenticator authenticator, boolean doPost) throws Exception {
     start();
     try {
-      SystemDefaultHttpClient httpClient = getHttpClient();
+      HttpClient httpClient = getHttpClient();
       doHttpClientRequest(httpClient, new HttpGet(getBaseURL()));
 
       // Always do a GET before POST to trigger the SPNego negotiation


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


[14/50] hadoop git commit: HDFS-11002. Fix broken attr/getfattr/setfattr links in ExtendedAttributes.md. Contributed by Mingliang Liu.

Posted by um...@apache.org.
HDFS-11002. Fix broken attr/getfattr/setfattr links in ExtendedAttributes.md. 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/901eca00
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/901eca00
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/901eca00

Branch: refs/heads/HDFS-10285
Commit: 901eca004d0e7e413b109a93128892176c808d61
Parents: 12d739a
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Oct 13 14:29:30 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Oct 13 14:29:30 2016 +0900

----------------------------------------------------------------------
 .../hadoop-hdfs/src/site/markdown/ExtendedAttributes.md          | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/901eca00/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ExtendedAttributes.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ExtendedAttributes.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ExtendedAttributes.md
index 5a20986..eb527ab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ExtendedAttributes.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ExtendedAttributes.md
@@ -30,7 +30,7 @@ Overview
 
 ### HDFS extended attributes
 
-Extended attributes in HDFS are modeled after extended attributes in Linux (see the Linux manpage for [attr(5)](http://www.bestbits.at/acl/man/man5/attr.txt) and [related documentation](http://www.bestbits.at/acl/)). An extended attribute is a *name-value pair*, with a string name and binary value. Xattrs names must also be prefixed with a *namespace*. For example, an xattr named *myXattr* in the *user* namespace would be specified as **user.myXattr**. Multiple xattrs can be associated with a single inode.
+Extended attributes in HDFS are modeled after extended attributes in Linux (see the Linux manpage for [attr(5)](http://man7.org/linux/man-pages/man5/attr.5.html)). An extended attribute is a *name-value pair*, with a string name and binary value. Xattrs names must also be prefixed with a *namespace*. For example, an xattr named *myXattr* in the *user* namespace would be specified as **user.myXattr**. Multiple xattrs can be associated with a single inode.
 
 ### Namespaces and Permissions
 
@@ -49,7 +49,7 @@ The `raw` namespace is reserved for internal system attributes that sometimes ne
 Interacting with extended attributes
 ------------------------------------
 
-The Hadoop shell has support for interacting with extended attributes via `hadoop fs -getfattr` and `hadoop fs -setfattr`. These commands are styled after the Linux [getfattr(1)](http://www.bestbits.at/acl/man/man1/getfattr.txt) and [setfattr(1)](http://www.bestbits.at/acl/man/man1/setfattr.txt) commands.
+The Hadoop shell has support for interacting with extended attributes via `hadoop fs -getfattr` and `hadoop fs -setfattr`. These commands are styled after the Linux [getfattr(1)](http://man7.org/linux/man-pages/man1/getfattr.1.html) and [setfattr(1)](http://man7.org/linux/man-pages/man1/setfattr.1.html) commands.
 
 ### getfattr
 


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


[28/50] hadoop git commit: HADOOP-13686. Adding additional unit test for Trash (I). Contributed by Weiwei Yang.

Posted by um...@apache.org.
HADOOP-13686. Adding additional unit test for Trash (I). Contributed by Weiwei Yang.


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

Branch: refs/heads/HDFS-10285
Commit: dbe663d5241feea0c88a3a9391ad48a029001d94
Parents: 5a5a724
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Thu Oct 13 23:05:16 2016 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Thu Oct 13 23:05:16 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/fs/TrashPolicyDefault.java    |  11 +-
 .../java/org/apache/hadoop/fs/TestTrash.java    | 352 ++++++++++++++++++-
 2 files changed, 356 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dbe663d5/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java
index 72222be..4f4c937 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java
@@ -40,6 +40,8 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.Time;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /** Provides a <i>trash</i> feature.  Files are moved to a user's trash
  * directory, a subdirectory of their home directory named ".Trash".  Files are
  * initially moved to a <i>current</i> sub-directory of the trash directory.
@@ -215,7 +217,7 @@ public class TrashPolicyDefault extends TrashPolicy {
     return new Emptier(getConf(), emptierInterval);
   }
 
-  private class Emptier implements Runnable {
+  protected class Emptier implements Runnable {
 
     private Configuration conf;
     private long emptierInterval;
@@ -223,7 +225,7 @@ public class TrashPolicyDefault extends TrashPolicy {
     Emptier(Configuration conf, long emptierInterval) throws IOException {
       this.conf = conf;
       this.emptierInterval = emptierInterval;
-      if (emptierInterval > deletionInterval || emptierInterval == 0) {
+      if (emptierInterval > deletionInterval || emptierInterval <= 0) {
         LOG.info("The configured checkpoint interval is " +
                  (emptierInterval / MSECS_PER_MINUTE) + " minutes." +
                  " Using an interval of " +
@@ -287,6 +289,11 @@ public class TrashPolicyDefault extends TrashPolicy {
     private long floor(long time, long interval) {
       return (time / interval) * interval;
     }
+
+    @VisibleForTesting
+    protected long getEmptierInterval() {
+      return this.emptierInterval/MSECS_PER_MINUTE;
+    }
   }
 
   private void createCheckpoint(Path trashRoot, Date date) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dbe663d5/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java
index 338aff6..7a5b25e 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java
@@ -29,13 +29,19 @@ import java.net.URI;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.util.HashSet;
+import java.util.Random;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import junit.framework.TestCase;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.TrashPolicyDefault.Emptier;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
+import org.junit.Before;
+import org.junit.Test;
 
 /**
  * This class tests commands from Trash.
@@ -45,6 +51,13 @@ public class TestTrash extends TestCase {
   private final static Path TEST_DIR = new Path(GenericTestUtils.getTempPath(
       "testTrash"));
 
+  @Before
+  public void setUp() throws IOException {
+    // ensure each test initiates a FileSystem instance,
+    // avoid getting an old instance from cache.
+    FileSystem.closeAll();
+  }
+
   protected static Path mkdir(FileSystem fs, Path p) throws IOException {
     assertTrue(fs.mkdirs(p));
     assertTrue(fs.exists(p));
@@ -516,6 +529,81 @@ public class TestTrash extends TestCase {
     assertTrue(trash.getTrashPolicy().getClass().equals(TestTrashPolicy.class));
   }
 
+  @Test
+  public void testCheckpointInterval() throws IOException {
+    // Verify if fs.trash.checkpoint.interval is set to positive number
+    // but bigger than fs.trash.interval,
+    // the value should be reset to fs.trash.interval
+    verifyDefaultPolicyIntervalValues(10, 12, 10);
+
+    // Verify if fs.trash.checkpoint.interval is set to positive number
+    // and smaller than fs.trash.interval, the value should be respected
+    verifyDefaultPolicyIntervalValues(10, 5, 5);
+
+    // Verify if fs.trash.checkpoint.interval sets to 0
+    // the value should be reset to fs.trash.interval
+    verifyDefaultPolicyIntervalValues(10, 0, 10);
+
+    // Verify if fs.trash.checkpoint.interval sets to a negative number
+    // the value should be reset to fs.trash.interval
+    verifyDefaultPolicyIntervalValues(10, -1, 10);
+  }
+
+  @Test
+  public void testMoveEmptyDirToTrash() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setClass(FS_FILE_IMPL_KEY,
+        RawLocalFileSystem.class,
+        FileSystem.class);
+    conf.setLong(FS_TRASH_INTERVAL_KEY, 1); // 1 min
+    FileSystem fs = FileSystem.get(conf);
+    verifyMoveEmptyDirToTrash(fs, conf);
+  }
+
+  /**
+   * Simulate the carrier process of the trash emptier restarts,
+   * verify it honors the <b>fs.trash.interval</b> before and after restart.
+   * @throws Exception
+   */
+  @Test
+  public void testTrashRestarts() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setClass("fs.trash.classname",
+        AuditableTrashPolicy.class,
+        TrashPolicy.class);
+    conf.setClass("fs.file.impl", TestLFS.class, FileSystem.class);
+    conf.set(FS_TRASH_INTERVAL_KEY, "50"); // in milliseconds for test
+    Trash trash = new Trash(conf);
+    // create 5 checkpoints
+    for(int i=0; i<5; i++) {
+      trash.checkpoint();
+    }
+
+    // Run the trash emptier for 120ms, it should run
+    // 2 times deletion as the interval is 50ms.
+    // Verify the checkpoints number when shutting down the emptier.
+    verifyAuditableTrashEmptier(trash, 120, 3);
+
+    // reconfigure the interval to 100 ms
+    conf.set(FS_TRASH_INTERVAL_KEY, "100");
+    Trash trashNew = new Trash(conf);
+
+    // Run the trash emptier for 120ms, it should run
+    // 1 time deletion.
+    verifyAuditableTrashEmptier(trashNew, 120, 2);
+  }
+
+  @Test
+  public void testTrashPermission()  throws IOException {
+    Configuration conf = new Configuration();
+    conf.setClass("fs.trash.classname",
+        TrashPolicyDefault.class,
+        TrashPolicy.class);
+    conf.setClass("fs.file.impl", TestLFS.class, FileSystem.class);
+    conf.set(FS_TRASH_INTERVAL_KEY, "0.2");
+    verifyTrashPermission(FileSystem.getLocal(conf), conf);
+  }
+
   public void testTrashEmptier() throws Exception {
     Configuration conf = new Configuration();
     // Trash with 12 second deletes and 6 seconds checkpoints
@@ -679,12 +767,143 @@ public class TestTrash extends TestCase {
         long factoredTime = first*factor;
         assertTrue(iterTime<factoredTime); //no more then twice of median first 10
       }
-    } 
+    }
   }
-  
-  public static void main(String [] arg) throws IOException{
-    // run performance piece as a separate test
-    performanceTestDeleteSameFile();
+
+  public static void verifyMoveEmptyDirToTrash(FileSystem fs,
+      Configuration conf) throws IOException {
+    Path caseRoot = new Path(
+        GenericTestUtils.getTempPath("testUserTrash"));
+    Path testRoot = new Path(caseRoot, "trash-users");
+    Path emptyDir = new Path(testRoot, "empty-dir");
+    try (FileSystem fileSystem = fs){
+      fileSystem.mkdirs(emptyDir);
+      Trash trash = new Trash(fileSystem, conf);
+      // Make sure trash root is clean
+      Path trashRoot = trash.getCurrentTrashDir(emptyDir);
+      fileSystem.delete(trashRoot, true);
+      // Move to trash should be succeed
+      assertTrue("Move an empty directory to trash failed",
+          trash.moveToTrash(emptyDir));
+      // Verify the empty dir is removed
+      assertFalse("The empty directory still exists on file system",
+          fileSystem.exists(emptyDir));
+      emptyDir = fileSystem.makeQualified(emptyDir);
+      Path dirInTrash = Path.mergePaths(trashRoot, emptyDir);
+      assertTrue("Directory wasn't moved to trash",
+          fileSystem.exists(dirInTrash));
+      FileStatus[] flist = fileSystem.listStatus(dirInTrash);
+      assertTrue("Directory is not empty",
+          flist!= null && flist.length == 0);
+    }
+  }
+
+  /**
+   * Create a bunch of files and set with different permission, after
+   * moved to trash, verify the location in trash directory is expected
+   * and the permission is reserved.
+   *
+   * @throws IOException
+   */
+  public static void verifyTrashPermission(FileSystem fs, Configuration conf)
+      throws IOException {
+    Path caseRoot = new Path(
+        GenericTestUtils.getTempPath("testTrashPermission"));
+    try (FileSystem fileSystem = fs){
+      Trash trash = new Trash(fileSystem, conf);
+      FileSystemTestWrapper wrapper =
+          new FileSystemTestWrapper(fileSystem);
+
+      short[] filePermssions = {
+          (short) 0600,
+          (short) 0644,
+          (short) 0660,
+          (short) 0700,
+          (short) 0750,
+          (short) 0755,
+          (short) 0775,
+          (short) 0777
+      };
+
+      for(int i=0; i<filePermssions.length; i++) {
+        // Set different permission to files
+        FsPermission fsPermission = new FsPermission(filePermssions[i]);
+        Path file = new Path(caseRoot, "file" + i);
+        byte[] randomBytes = new byte[new Random().nextInt(10)];
+        wrapper.writeFile(file, randomBytes);
+        wrapper.setPermission(file, fsPermission);
+
+        // Move file to trash
+        trash.moveToTrash(file);
+
+        // Verify the file is moved to trash, at expected location
+        Path trashDir = trash.getCurrentTrashDir(file);
+        if(!file.isAbsolute()) {
+          file = wrapper.makeQualified(file);
+        }
+        Path fileInTrash = Path.mergePaths(trashDir, file);
+        FileStatus fstat = wrapper.getFileStatus(fileInTrash);
+        assertTrue(String.format("File %s is not moved to trash",
+            fileInTrash.toString()),
+            wrapper.exists(fileInTrash));
+        // Verify permission not change
+        assertTrue(String.format("Expected file: %s is %s, but actual is %s",
+            fileInTrash.toString(),
+            fsPermission.toString(),
+            fstat.getPermission().toString()),
+            fstat.getPermission().equals(fsPermission));
+      }
+
+      // Verify the trash directory can be removed
+      Path trashRoot = trash.getCurrentTrashDir();
+      assertTrue(wrapper.delete(trashRoot, true));
+    }
+  }
+
+  private void verifyDefaultPolicyIntervalValues(long trashInterval,
+      long checkpointInterval, long expectedInterval) throws IOException {
+    Configuration conf = new Configuration();
+    conf.setLong(FS_TRASH_INTERVAL_KEY, trashInterval);
+    conf.set("fs.trash.classname", TrashPolicyDefault.class.getName());
+    conf.setLong(FS_TRASH_CHECKPOINT_INTERVAL_KEY, checkpointInterval);
+    Trash trash = new Trash(conf);
+    Emptier emptier = (Emptier)trash.getEmptier();
+    assertEquals(expectedInterval, emptier.getEmptierInterval());
+  }
+
+  /**
+   * Launch the {@link Trash} emptier for given milliseconds,
+   * verify the number of checkpoints is expected.
+   */
+  private void verifyAuditableTrashEmptier(Trash trash,
+      long timeAlive,
+      int expectedNumOfCheckpoints)
+          throws IOException {
+    Thread emptierThread = null;
+    try {
+      Runnable emptier = trash.getEmptier();
+      emptierThread = new Thread(emptier);
+      emptierThread.start();
+
+      // Shutdown the emptier thread after a given time
+      Thread.sleep(timeAlive);
+      emptierThread.interrupt();
+      emptierThread.join();
+
+      AuditableTrashPolicy at = (AuditableTrashPolicy) trash.getTrashPolicy();
+      assertEquals(
+          String.format("Expected num of checkpoints is %s, but actual is %s",
+              expectedNumOfCheckpoints, at.getNumberOfCheckpoints()),
+          expectedNumOfCheckpoints,
+          at.getNumberOfCheckpoints());
+    } catch (InterruptedException  e) {
+      // Ignore
+    } finally {
+      // Avoid thread leak
+      if(emptierThread != null) {
+        emptierThread.interrupt();
+      }
+    }
   }
 
   // Test TrashPolicy. Don't care about implementation.
@@ -732,4 +951,127 @@ public class TestTrash extends TestCase {
       return null;
     }
   }
+
+  /**
+   * A fake {@link TrashPolicy} implementation, it keeps a count
+   * on number of checkpoints in the trash. It doesn't do anything
+   * other than updating the count.
+   *
+   */
+  public static class AuditableTrashPolicy extends TrashPolicy {
+
+    public AuditableTrashPolicy() {}
+
+    public AuditableTrashPolicy(Configuration conf)
+        throws IOException {
+      this.initialize(conf, null);
+    }
+
+    @Override
+    @Deprecated
+    public void initialize(Configuration conf, FileSystem fs, Path home) {
+      this.deletionInterval = (long)(conf.getFloat(
+          FS_TRASH_INTERVAL_KEY, FS_TRASH_INTERVAL_DEFAULT));
+    }
+
+    @Override
+    public void initialize(Configuration conf, FileSystem fs) {
+      this.deletionInterval = (long)(conf.getFloat(
+          FS_TRASH_INTERVAL_KEY, FS_TRASH_INTERVAL_DEFAULT));
+    }
+
+    @Override
+    public boolean moveToTrash(Path path) throws IOException {
+      return false;
+    }
+
+    @Override
+    public void createCheckpoint() throws IOException {
+      AuditableCheckpoints.add();
+    }
+
+    @Override
+    public void deleteCheckpoint() throws IOException {
+      AuditableCheckpoints.delete();
+    }
+
+    @Override
+    public Path getCurrentTrashDir() {
+      return null;
+    }
+
+    @Override
+    public Runnable getEmptier() throws IOException {
+      return new AuditableEmptier(getConf());
+    }
+
+    public int getNumberOfCheckpoints() {
+      return AuditableCheckpoints.get();
+    }
+
+    /**
+     * A fake emptier that simulates to delete a checkpoint
+     * in a fixed interval.
+     */
+    private class AuditableEmptier implements Runnable {
+      private Configuration conf = null;
+      public AuditableEmptier(Configuration conf) {
+        this.conf = conf;
+      }
+
+      @Override
+      public void run() {
+        AuditableTrashPolicy trash = null;
+        try {
+          trash = new AuditableTrashPolicy(conf);
+        } catch (IOException e1) {}
+        while(true) {
+          try {
+            Thread.sleep(deletionInterval);
+            trash.deleteCheckpoint();
+          } catch (IOException e) {
+            // no exception
+          } catch (InterruptedException e) {
+            break;
+          }
+        }
+      }
+    }
+
+    @Override
+    public boolean isEnabled() {
+      return true;
+    }
+  }
+
+  /**
+   * Only counts the number of checkpoints, not do anything more.
+   * Declared as an inner static class to share state between
+   * testing threads.
+   */
+  private static class AuditableCheckpoints {
+
+    private static AtomicInteger numOfCheckpoint =
+        new AtomicInteger(0);
+
+    private static void add() {
+      numOfCheckpoint.incrementAndGet();
+      System.out.println(String
+          .format("Create a checkpoint, current number of checkpoints %d",
+              numOfCheckpoint.get()));
+    }
+
+    private static void delete() {
+      if(numOfCheckpoint.get() > 0) {
+        numOfCheckpoint.decrementAndGet();
+        System.out.println(String
+            .format("Delete a checkpoint, current number of checkpoints %d",
+                numOfCheckpoint.get()));
+      }
+    }
+
+    private static int get() {
+      return numOfCheckpoint.get();
+    }
+  }
 }


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


[35/50] hadoop git commit: HDFS-11008. Change unit test for testing parsing "-source" parameter in Balancer CLI. Contributed by Mingliang Liu

Posted by um...@apache.org.
HDFS-11008. Change unit test for testing parsing "-source" parameter in Balancer CLI. 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/76cc84e6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/76cc84e6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/76cc84e6

Branch: refs/heads/HDFS-10285
Commit: 76cc84e6d41c2b02218c2c98d60481cd565e067c
Parents: aee538b
Author: Mingliang Liu <li...@apache.org>
Authored: Thu Oct 13 17:51:38 2016 -0700
Committer: Mingliang Liu <li...@apache.org>
Committed: Fri Oct 14 14:29:02 2016 -0700

----------------------------------------------------------------------
 .../hdfs/server/balancer/TestBalancer.java      | 61 ++++++++++++--------
 1 file changed, 38 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/76cc84e6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
index 73a4cbc..f58a3ad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
@@ -1282,6 +1282,14 @@ public class TestBalancer {
     } catch (IllegalArgumentException e) {
 
     }
+
+    parameters = new String[] {"-source"};
+    try {
+      Balancer.Cli.parse(parameters);
+      fail(reason + " for -source parameter");
+    } catch (IllegalArgumentException ignored) {
+      // expected
+    }
   }
 
   @Test
@@ -1800,11 +1808,12 @@ public class TestBalancer {
     final Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
 
     { // run Balancer with min-block-size=50
-      BalancerParameters.Builder b =
-          new BalancerParameters.Builder();
-      b.setBalancingPolicy(BalancingPolicy.Node.INSTANCE);
-      b.setThreshold(1);
-      final BalancerParameters p = b.build();
+      final BalancerParameters p = Balancer.Cli.parse(new String[] {
+          "-policy", BalancingPolicy.Node.INSTANCE.getName(),
+          "-threshold", "1"
+      });
+      assertEquals(p.getBalancingPolicy(), BalancingPolicy.Node.INSTANCE);
+      assertEquals(p.getThreshold(), 1.0, 0.001);
 
       conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 50);
       final int r = Balancer.run(namenodes, p, conf);
@@ -1819,12 +1828,14 @@ public class TestBalancer {
       for(int i = capacities.length; i < datanodes.size(); i++) {
         sourceNodes.add(datanodes.get(i).getDisplayName());
       }
-      BalancerParameters.Builder b =
-          new BalancerParameters.Builder();
-      b.setBalancingPolicy(BalancingPolicy.Node.INSTANCE);
-      b.setThreshold(1);
-      b.setSourceNodes(sourceNodes);
-      final BalancerParameters p = b.build();
+      final BalancerParameters p = Balancer.Cli.parse(new String[] {
+          "-policy", BalancingPolicy.Node.INSTANCE.getName(),
+          "-threshold", "1",
+          "-source", StringUtils.join(sourceNodes, ',')
+      });
+      assertEquals(p.getBalancingPolicy(), BalancingPolicy.Node.INSTANCE);
+      assertEquals(p.getThreshold(), 1.0, 0.001);
+      assertEquals(p.getSourceNodes(), sourceNodes);
 
       conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 50);
       final int r = Balancer.run(namenodes, p, conf);
@@ -1835,12 +1846,14 @@ public class TestBalancer {
       final Set<String> sourceNodes = new HashSet<>();
       final List<DataNode> datanodes = cluster.getDataNodes();
       sourceNodes.add(datanodes.get(0).getDisplayName());
-      BalancerParameters.Builder b =
-          new BalancerParameters.Builder();
-      b.setBalancingPolicy(BalancingPolicy.Node.INSTANCE);
-      b.setThreshold(1);
-      b.setSourceNodes(sourceNodes);
-      final BalancerParameters p = b.build();
+      final BalancerParameters p = Balancer.Cli.parse(new String[] {
+          "-policy", BalancingPolicy.Node.INSTANCE.getName(),
+          "-threshold", "1",
+          "-source", StringUtils.join(sourceNodes, ',')
+      });
+      assertEquals(p.getBalancingPolicy(), BalancingPolicy.Node.INSTANCE);
+      assertEquals(p.getThreshold(), 1.0, 0.001);
+      assertEquals(p.getSourceNodes(), sourceNodes);
 
       conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1);
       final int r = Balancer.run(namenodes, p, conf);
@@ -1853,12 +1866,14 @@ public class TestBalancer {
       for(int i = 0; i < capacities.length; i++) {
         sourceNodes.add(datanodes.get(i).getDisplayName());
       }
-      BalancerParameters.Builder b =
-          new BalancerParameters.Builder();
-      b.setBalancingPolicy(BalancingPolicy.Node.INSTANCE);
-      b.setThreshold(1);
-      b.setSourceNodes(sourceNodes);
-      final BalancerParameters p = b.build();
+      final BalancerParameters p = Balancer.Cli.parse(new String[] {
+          "-policy", BalancingPolicy.Node.INSTANCE.getName(),
+          "-threshold", "1",
+          "-source", StringUtils.join(sourceNodes, ',')
+      });
+      assertEquals(p.getBalancingPolicy(), BalancingPolicy.Node.INSTANCE);
+      assertEquals(p.getThreshold(), 1.0, 0.001);
+      assertEquals(p.getSourceNodes(), sourceNodes);
 
       conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1);
       final int r = Balancer.run(namenodes, p, conf);


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


[10/50] hadoop git commit: YARN-4464. Lower the default max applications stored in the RM and store. (Daniel Templeton via kasha)

Posted by um...@apache.org.
YARN-4464. Lower the default max applications stored in the RM and store. (Daniel Templeton via kasha)


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

Branch: refs/heads/HDFS-10285
Commit: 6378845f9ef789c3fda862c43bcd498aa3f35068
Parents: 7ba7092
Author: Karthik Kambatla <ka...@cloudera.com>
Authored: Tue Oct 11 21:41:58 2016 -0700
Committer: Karthik Kambatla <ka...@cloudera.com>
Committed: Tue Oct 11 21:42:08 2016 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     | 20 ++++++++++++++++----
 .../src/main/resources/yarn-default.xml         |  4 ++--
 .../server/resourcemanager/RMAppManager.java    |  2 +-
 3 files changed, 19 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6378845f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 4d43357..3bd0dcc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -719,17 +719,29 @@ public class YarnConfiguration extends Configuration {
       + "leveldb-state-store.compaction-interval-secs";
   public static final long DEFAULT_RM_LEVELDB_COMPACTION_INTERVAL_SECS = 3600;
 
-  /** The maximum number of completed applications RM keeps. */ 
+  /**
+   * The maximum number of completed applications RM keeps. By default equals
+   * to {@link #DEFAULT_RM_MAX_COMPLETED_APPLICATIONS}.
+   */
   public static final String RM_MAX_COMPLETED_APPLICATIONS =
     RM_PREFIX + "max-completed-applications";
-  public static final int DEFAULT_RM_MAX_COMPLETED_APPLICATIONS = 10000;
+  public static final int DEFAULT_RM_MAX_COMPLETED_APPLICATIONS = 1000;
 
   /**
-   * The maximum number of completed applications RM state store keeps, by
-   * default equals to DEFAULT_RM_MAX_COMPLETED_APPLICATIONS
+   * The maximum number of completed applications RM state store keeps. By
+   * default equals to value of {@link #RM_MAX_COMPLETED_APPLICATIONS}.
    */
   public static final String RM_STATE_STORE_MAX_COMPLETED_APPLICATIONS =
       RM_PREFIX + "state-store.max-completed-applications";
+  /**
+   * The default value for
+   * {@code yarn.resourcemanager.state-store.max-completed-applications}.
+   * @deprecated This default value is ignored and will be removed in a future
+   * release. The default value of
+   * {@code yarn.resourcemanager.state-store.max-completed-applications} is the
+   * value of {@link #RM_MAX_COMPLETED_APPLICATIONS}.
+   */
+  @Deprecated
   public static final int DEFAULT_RM_STATE_STORE_MAX_COMPLETED_APPLICATIONS =
       DEFAULT_RM_MAX_COMPLETED_APPLICATIONS;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6378845f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 524afec..f37c689 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -417,7 +417,7 @@
     the applications remembered in RM memory.
     Any values larger than ${yarn.resourcemanager.max-completed-applications} will
     be reset to ${yarn.resourcemanager.max-completed-applications}.
-    Note that this value impacts the RM recovery performance.Typically,
+    Note that this value impacts the RM recovery performance. Typically,
     a smaller value indicates better performance on RM recovery.
     </description>
     <name>yarn.resourcemanager.state-store.max-completed-applications</name>
@@ -687,7 +687,7 @@
   <property>
     <description>The maximum number of completed applications RM keeps. </description>
     <name>yarn.resourcemanager.max-completed-applications</name>
-    <value>10000</value>
+    <value>1000</value>
   </property>
 
   <property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6378845f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.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/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
index 7352a28..c065b60 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
@@ -101,7 +101,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
     this.maxCompletedAppsInStateStore =
         conf.getInt(
           YarnConfiguration.RM_STATE_STORE_MAX_COMPLETED_APPLICATIONS,
-          YarnConfiguration.DEFAULT_RM_STATE_STORE_MAX_COMPLETED_APPLICATIONS);
+          this.maxCompletedAppsInMemory);
     if (this.maxCompletedAppsInStateStore > this.maxCompletedAppsInMemory) {
       this.maxCompletedAppsInStateStore = this.maxCompletedAppsInMemory;
     }


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


[13/50] hadoop git commit: HADOOP-13700. Remove unthrown IOException from TrashPolicy#initialize and #getInstance signatures.

Posted by um...@apache.org.
HADOOP-13700. Remove unthrown IOException from TrashPolicy#initialize and #getInstance signatures.


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

Branch: refs/heads/HDFS-10285
Commit: 12d739a34ba868b3f7f5adf7f37a60d4aca9061b
Parents: 85cd06f
Author: Andrew Wang <wa...@apache.org>
Authored: Wed Oct 12 15:19:52 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Wed Oct 12 15:19:52 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/fs/TrashPolicy.java        | 6 ++----
 1 file changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/12d739a3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java
index 157b9ab..2fe3fd1 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java
@@ -53,9 +53,8 @@ public abstract class TrashPolicy extends Configured {
    * not assume trash always under /user/$USER due to HDFS encryption zone.
    * @param conf the configuration to be used
    * @param fs the filesystem to be used
-   * @throws IOException
    */
-  public void initialize(Configuration conf, FileSystem fs) throws IOException{
+  public void initialize(Configuration conf, FileSystem fs) {
     throw new UnsupportedOperationException();
   }
 
@@ -137,8 +136,7 @@ public abstract class TrashPolicy extends Configured {
    * @param fs the file system to be used
    * @return an instance of TrashPolicy
    */
-  public static TrashPolicy getInstance(Configuration conf, FileSystem fs)
-      throws IOException {
+  public static TrashPolicy getInstance(Configuration conf, FileSystem fs) {
     Class<? extends TrashPolicy> trashClass = conf.getClass(
         "fs.trash.classname", TrashPolicyDefault.class, TrashPolicy.class);
     TrashPolicy trash = ReflectionUtils.newInstance(trashClass, conf);


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


[36/50] hadoop git commit: HDFS-10558. DiskBalancer: Print the full path to plan file. Contributed by Xiaobing Zhou.

Posted by um...@apache.org.
HDFS-10558. DiskBalancer: Print the full path to plan file. Contributed by Xiaobing Zhou.


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

Branch: refs/heads/HDFS-10285
Commit: 30bb1970cc27c1345871a35bcf1220e520c1804b
Parents: 76cc84e
Author: Anu Engineer <ae...@apache.org>
Authored: Fri Oct 14 17:07:59 2016 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Fri Oct 14 17:07:59 2016 -0700

----------------------------------------------------------------------
 .../server/diskbalancer/command/Command.java    | 21 +++++-
 .../diskbalancer/command/HelpCommand.java       |  2 +-
 .../diskbalancer/command/PlanCommand.java       | 55 +++++++++++----
 .../diskbalancer/command/ReportCommand.java     | 11 +--
 .../hadoop/hdfs/tools/DiskBalancerCLI.java      | 27 +++++---
 .../diskbalancer/DiskBalancerTestUtil.java      | 72 ++++++++++++++++++++
 .../server/diskbalancer/TestDiskBalancer.java   |  4 +-
 .../command/TestDiskBalancerCommand.java        | 61 +++++++++++++++--
 8 files changed, 218 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/30bb1970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
index 2497669..11c8e7f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
@@ -53,6 +53,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.io.PrintStream;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URL;
@@ -82,6 +83,7 @@ public abstract class Command extends Configured {
   private FileSystem fs = null;
   private DiskBalancerCluster cluster = null;
   private int topNodes;
+  private PrintStream ps;
 
   private static final Path DEFAULT_LOG_DIR = new Path("/system/diskbalancer");
 
@@ -91,9 +93,25 @@ public abstract class Command extends Configured {
    * Constructs a command.
    */
   public Command(Configuration conf) {
+    this(conf, System.out);
+  }
+
+  /**
+   * Constructs a command.
+   */
+  public Command(Configuration conf, final PrintStream ps) {
     super(conf);
     // These arguments are valid for all commands.
     topNodes = 0;
+    this.ps = ps;
+  }
+
+  /**
+   * Gets printing stream.
+   * @return print stream
+   */
+  PrintStream getPrintStream() {
+    return ps;
   }
 
   /**
@@ -423,7 +441,8 @@ public abstract class Command extends Configured {
    *
    * @return Cluster.
    */
-  protected DiskBalancerCluster getCluster() {
+  @VisibleForTesting
+  DiskBalancerCluster getCluster() {
     return cluster;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/30bb1970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java
index c735299..f7c84e1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java
@@ -78,7 +78,7 @@ public class HelpCommand extends Command {
       command = new CancelCommand(getConf());
       break;
     case DiskBalancerCLI.REPORT:
-      command = new ReportCommand(getConf(), null);
+      command = new ReportCommand(getConf());
       break;
     default:
       command = this;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/30bb1970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
index 9749409..1d07a63 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
@@ -18,9 +18,12 @@
 package org.apache.hadoop.hdfs.server.diskbalancer.command;
 
 import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang.text.StrBuilder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -31,6 +34,7 @@ import org.apache.hadoop.hdfs.server.diskbalancer.planner.Step;
 import org.apache.hadoop.hdfs.tools.DiskBalancerCLI;
 import java.nio.charset.StandardCharsets;
 import java.util.List;
+import java.io.PrintStream;
 
 /**
  * Class that implements Plan Command.
@@ -49,7 +53,14 @@ public class PlanCommand extends Command {
    * Constructs a plan command.
    */
   public PlanCommand(Configuration conf) {
-    super(conf);
+    this(conf, System.out);
+  }
+
+  /**
+   * Constructs a plan command.
+   */
+  public PlanCommand(Configuration conf, final PrintStream ps) {
+    super(conf, ps);
     this.thresholdPercentage = 1;
     this.bandwidth = 0;
     this.maxError = 0;
@@ -73,9 +84,12 @@ public class PlanCommand extends Command {
    * -plan -node IP -plan -node hostName -plan -node DatanodeUUID
    *
    * @param cmd - CommandLine
+   * @throws Exception
    */
   @Override
   public void execute(CommandLine cmd) throws Exception {
+    StrBuilder result = new StrBuilder();
+    String outputLine = "";
     LOG.debug("Processing Plan Command.");
     Preconditions.checkState(cmd.hasOption(DiskBalancerCLI.PLAN));
     verifyCommandOptions(DiskBalancerCLI.PLAN, cmd);
@@ -131,22 +145,35 @@ public class PlanCommand extends Command {
           .getBytes(StandardCharsets.UTF_8));
     }
 
-    if (plan != null && plan.getVolumeSetPlans().size() > 0) {
-      LOG.info("Writing plan to : {}", getOutputPath());
-      try (FSDataOutputStream planStream = create(String.format(
-          DiskBalancerCLI.PLAN_TEMPLATE,
-          cmd.getOptionValue(DiskBalancerCLI.PLAN)))) {
-        planStream.write(plan.toJson().getBytes(StandardCharsets.UTF_8));
+    try {
+      if (plan != null && plan.getVolumeSetPlans().size() > 0) {
+        outputLine = String.format("Writing plan to: %s", getOutputPath());
+        recordOutput(result, outputLine);
+        try (FSDataOutputStream planStream = create(String.format(
+            DiskBalancerCLI.PLAN_TEMPLATE,
+            cmd.getOptionValue(DiskBalancerCLI.PLAN)))) {
+          planStream.write(plan.toJson().getBytes(StandardCharsets.UTF_8));
+        }
+      } else {
+        outputLine = String.format(
+            "No plan generated. DiskBalancing not needed for node: %s"
+                + " threshold used: %s",
+            cmd.getOptionValue(DiskBalancerCLI.PLAN), this.thresholdPercentage);
+        recordOutput(result, outputLine);
       }
-    } else {
-      LOG.info("No plan generated. DiskBalancing not needed for node: {} " +
-              "threshold used: {}", cmd.getOptionValue(DiskBalancerCLI.PLAN),
-          this.thresholdPercentage);
-    }
 
-    if (cmd.hasOption(DiskBalancerCLI.VERBOSE) && plans.size() > 0) {
-      printToScreen(plans);
+      if (cmd.hasOption(DiskBalancerCLI.VERBOSE) && plans.size() > 0) {
+        printToScreen(plans);
+      }
+    } catch (Exception e) {
+      final String errMsg =
+          "Errors while recording the output of plan command.";
+      LOG.error(errMsg, e);
+      result.appendln(errMsg);
+      result.appendln(Throwables.getStackTraceAsString(e));
     }
+
+    getPrintStream().println(result.toString());
   }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/30bb1970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
index 79ba14f..e10ffac 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
@@ -47,11 +47,12 @@ import com.google.common.collect.Lists;
  */
 public class ReportCommand extends Command {
 
-  private PrintStream out;
+  public ReportCommand(Configuration conf) {
+    this(conf, System.out);
+  }
 
-  public ReportCommand(Configuration conf, final PrintStream out) {
-    super(conf);
-    this.out = out;
+  public ReportCommand(Configuration conf, final PrintStream ps) {
+    super(conf, ps);
 
     addValidCommandParameters(DiskBalancerCLI.REPORT,
         "Report volume information of nodes.");
@@ -95,7 +96,7 @@ public class ReportCommand extends Command {
       handleTopReport(cmd, result, nodeFormat);
     }
 
-    out.println(result.toString());
+    getPrintStream().println(result.toString());
   }
 
   private void handleTopReport(final CommandLine cmd, final StrBuilder result,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/30bb1970/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancerCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancerCLI.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancerCLI.java
index c216a30..5bcf939 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancerCLI.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancerCLI.java
@@ -137,6 +137,8 @@ public class DiskBalancerCLI extends Configured implements Tool {
 
   private final PrintStream printStream;
 
+  private Command currentCommand = null;
+
   /**
    * Construct a DiskBalancer.
    *
@@ -432,6 +434,13 @@ public class DiskBalancerCLI extends Configured implements Tool {
   }
 
   /**
+   * Gets current command associated with this instance of DiskBalancer.
+   */
+  public Command getCurrentCommand() {
+    return currentCommand;
+  }
+
+  /**
    * Dispatches calls to the right command Handler classes.
    *
    * @param cmd  - CommandLine
@@ -440,38 +449,38 @@ public class DiskBalancerCLI extends Configured implements Tool {
    */
   private int dispatch(CommandLine cmd, Options opts)
       throws Exception {
-    Command currentCommand = null;
+    Command dbCmd = null;
     if (cmd.hasOption(DiskBalancerCLI.PLAN)) {
-      currentCommand = new PlanCommand(getConf());
+      dbCmd = new PlanCommand(getConf(), printStream);
     }
 
     if (cmd.hasOption(DiskBalancerCLI.EXECUTE)) {
-      currentCommand = new ExecuteCommand(getConf());
+      dbCmd = new ExecuteCommand(getConf());
     }
 
     if (cmd.hasOption(DiskBalancerCLI.QUERY)) {
-      currentCommand = new QueryCommand(getConf());
+      dbCmd = new QueryCommand(getConf());
     }
 
     if (cmd.hasOption(DiskBalancerCLI.CANCEL)) {
-      currentCommand = new CancelCommand(getConf());
+      dbCmd = new CancelCommand(getConf());
     }
 
     if (cmd.hasOption(DiskBalancerCLI.REPORT)) {
-      currentCommand = new ReportCommand(getConf(), this.printStream);
+      dbCmd = new ReportCommand(getConf(), this.printStream);
     }
 
     if (cmd.hasOption(DiskBalancerCLI.HELP)) {
-      currentCommand = new HelpCommand(getConf());
+      dbCmd = new HelpCommand(getConf());
     }
 
     // Invoke main help here.
-    if (currentCommand == null) {
+    if (dbCmd == null) {
       new HelpCommand(getConf()).execute(null);
       return 1;
     }
 
-    currentCommand.execute(cmd);
+    dbCmd.execute(cmd);
     return 0;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/30bb1970/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
index 5e98eb2..a575097 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
@@ -18,10 +18,20 @@
 package org.apache.hadoop.hdfs.server.diskbalancer;
 
 import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.balancer.TestBalancer;
+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.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.NullConnector;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
@@ -29,9 +39,12 @@ import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolumeSet;
 import org.apache.hadoop.util.Time;
 
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.util.Random;
 import java.util.UUID;
+import java.util.concurrent.TimeoutException;
 
 /**
  * Helper class to create various cluster configrations at run time.
@@ -242,6 +255,65 @@ public class DiskBalancerTestUtil {
     return count;
   }
 
+  public static MiniDFSCluster newImbalancedCluster(
+      final Configuration conf,
+      final int numDatanodes,
+      final long[] storageCapacities,
+      final int defaultBlockSize,
+      final int fileLen)
+      throws IOException, InterruptedException, TimeoutException {
+    conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, defaultBlockSize);
+    conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, defaultBlockSize);
+    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
+
+    final String fileName = "/" + UUID.randomUUID().toString();
+    final Path filePath = new Path(fileName);
+
+    Preconditions.checkNotNull(storageCapacities);
+    Preconditions.checkArgument(
+        storageCapacities.length == 2,
+        "need to specify capacities for two storages.");
+
+    // Write a file and restart the cluster
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(numDatanodes)
+        .storageCapacities(storageCapacities)
+        .storageTypes(new StorageType[]{StorageType.DISK, StorageType.DISK})
+        .storagesPerDatanode(2)
+        .build();
+    FsVolumeImpl source = null;
+    FsVolumeImpl dest = null;
+
+    cluster.waitActive();
+    Random r = new Random();
+    FileSystem fs = cluster.getFileSystem(0);
+    TestBalancer.createFile(cluster, filePath, fileLen, (short) 1, 0);
+
+    DFSTestUtil.waitReplication(fs, filePath, (short) 1);
+    cluster.restartDataNodes();
+    cluster.waitActive();
+
+    // Get the data node and move all data to one disk.
+    for (int i = 0; i < numDatanodes; i++) {
+      DataNode dnNode = cluster.getDataNodes().get(i);
+      try (FsDatasetSpi.FsVolumeReferences refs =
+               dnNode.getFSDataset().getFsVolumeReferences()) {
+        source = (FsVolumeImpl) refs.get(0);
+        dest = (FsVolumeImpl) refs.get(1);
+        assertTrue(DiskBalancerTestUtil.getBlockCount(source) > 0);
+        DiskBalancerTestUtil.moveAllDataToDestVolume(dnNode.getFSDataset(),
+            source, dest);
+        assertTrue(DiskBalancerTestUtil.getBlockCount(source) == 0);
+      }
+    }
+
+    cluster.restartDataNodes();
+    cluster.waitActive();
+
+    return cluster;
+  }
+
   /**
    * Moves all blocks to the destination volume.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/30bb1970/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java
index 9985210..5568032 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java
@@ -244,7 +244,9 @@ public class TestDiskBalancer {
     } catch (Exception e) {
       Assert.fail("Unexpected exception: " + e);
     } finally {
-      cluster.shutdown();
+      if (cluster != null) {
+        cluster.shutdown();
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/30bb1970/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
index 6697785..9f9c7b8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerTestUtil;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
@@ -409,14 +410,53 @@ public class TestDiskBalancerCommand {
     runCommand(cmdLine);
   }
 
-  private List<String> runCommandInternal(final String cmdLine) throws
-      Exception {
+  @Test
+  public void testPrintFullPathOfPlan()
+      throws Exception {
+    MiniDFSCluster miniCluster = null;
+    try {
+      Configuration hdfsConf = new HdfsConfiguration();
+      final int numDatanodes = 1;
+      final int defaultBlockSize = 1024;
+      final int fileLen = 200 * 1024;
+      final long capcacity = 300 * 1024;
+      final long[] capacities = new long[] {capcacity, capcacity};
+      List<String> outputs = null;
+
+      /* new cluster with imbalanced capacity */
+      miniCluster = DiskBalancerTestUtil.newImbalancedCluster(
+          hdfsConf,
+          numDatanodes,
+          capacities,
+          defaultBlockSize,
+          fileLen);
+
+      /* run plan command */
+      final String cmdLine = String.format(
+          "hdfs diskbalancer -%s %s",
+          PLAN,
+          miniCluster.getDataNodes().get(0).getDatanodeUuid());
+      outputs = runCommand(cmdLine, hdfsConf, miniCluster);
+
+      /* verify the path of plan */
+      assertThat(outputs.get(0), containsString("Writing plan to"));
+      assertThat(outputs.get(0), containsString("/system/diskbalancer"));
+    } finally {
+      if (miniCluster != null) {
+        miniCluster.shutdown();
+      }
+    }
+  }
+
+  private List<String> runCommandInternal(
+      final String cmdLine,
+      final Configuration clusterConf) throws Exception {
     String[] cmds = StringUtils.split(cmdLine, ' ');
     ByteArrayOutputStream bufOut = new ByteArrayOutputStream();
     PrintStream out = new PrintStream(bufOut);
 
-    Tool diskBalancerTool = new DiskBalancerCLI(conf, out);
-    ToolRunner.run(conf, diskBalancerTool, cmds);
+    Tool diskBalancerTool = new DiskBalancerCLI(clusterConf, out);
+    ToolRunner.run(clusterConf, diskBalancerTool, cmds);
 
     Scanner scanner = new Scanner(bufOut.toString());
     List<String> outputs = Lists.newArrayList();
@@ -426,6 +466,11 @@ public class TestDiskBalancerCommand {
     return outputs;
   }
 
+  private List<String> runCommandInternal(final String cmdLine)
+      throws Exception {
+    return runCommandInternal(cmdLine, conf);
+  }
+
   private List<String> runCommand(final String cmdLine) throws Exception {
     FileSystem.setDefaultUri(conf, clusterJson);
     return runCommandInternal(cmdLine);
@@ -437,6 +482,14 @@ public class TestDiskBalancerCommand {
     return runCommandInternal(cmdLine);
   }
 
+  private List<String> runCommand(
+      final String cmdLine,
+      Configuration clusterConf,
+      MiniDFSCluster miniCluster) throws Exception {
+    FileSystem.setDefaultUri(clusterConf, miniCluster.getURI());
+    return runCommandInternal(cmdLine, clusterConf);
+  }
+
   /**
    * Making sure that we can query the node without having done a submit.
    * @throws Exception


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


[45/50] hadoop git commit: HDFS-11013. Correct typos in native erasure coding dump code. Contributed by László Bence Nagy.

Posted by um...@apache.org.
HDFS-11013. Correct typos in native erasure coding dump code. Contributed by L�szl� Bence Nagy.


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

Branch: refs/heads/HDFS-10285
Commit: b671ee6846b79a6d106efed7cf7e1209b2cc408d
Parents: 987ee51
Author: Andrew Wang <wa...@apache.org>
Authored: Mon Oct 17 14:14:50 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Mon Oct 17 14:14:50 2016 -0700

----------------------------------------------------------------------
 .../main/native/src/org/apache/hadoop/io/erasurecode/dump.c  | 8 ++++----
 .../native/src/org/apache/hadoop/io/erasurecode/isal_load.h  | 2 +-
 2 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b671ee68/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.c
index 20bd189..e48032e 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.c
@@ -57,11 +57,11 @@ void dumpCodingMatrix(unsigned char* buf, int n1, int n2) {
 
 void dumpEncoder(IsalEncoder* pCoder) {
   int numDataUnits = pCoder->coder.numDataUnits;
-  int numParityUnits = pCoder->coder.numDataUnits;
+  int numParityUnits = pCoder->coder.numParityUnits;
   int numAllUnits = pCoder->coder.numAllUnits;
 
-  printf("Encoding (numAlnumParityUnitslUnits = %d, numDataUnits = %d)\n",
-                                    numParityUnits, numDataUnits);
+  printf("Encoding (numAllUnits = %d, numParityUnits = %d, numDataUnits = %d)\n",
+                                    numAllUnits, numParityUnits, numDataUnits);
 
   printf("\n\nEncodeMatrix:\n");
   dumpCodingMatrix((unsigned char*) pCoder->encodeMatrix,
@@ -91,7 +91,7 @@ void dumpDecoder(IsalDecoder* pCoder) {
 
   printf("InvertMatrix:\n");
   dumpCodingMatrix((unsigned char*) pCoder->invertMatrix,
-                                   numDataUnits, numDataUnits);
+                                   numDataUnits, numAllUnits);
 
   printf("DecodeMatrix:\n");
   dumpCodingMatrix((unsigned char*) pCoder->decodeMatrix,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b671ee68/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.h
index 7cb7a6a..c46a531 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.h
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.h
@@ -57,7 +57,7 @@ typedef void (*__d_ec_encode_data_update)(int, int, int, int, unsigned char*,
 #endif
 
 #ifdef WINDOWS
-// For erasure_code.h
+// For gf_util.h
 typedef unsigned char (__cdecl *__d_gf_mul)(unsigned char, unsigned char);
 typedef unsigned char (__cdecl *__d_gf_inv)(unsigned char);
 typedef void (__cdecl *__d_gf_gen_rs_matrix)(unsigned char *, int, int);


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


[30/50] hadoop git commit: HDFS-10735 Distcp using webhdfs on secure HA clusters fails with StandbyException

Posted by um...@apache.org.
HDFS-10735 Distcp using webhdfs on secure HA clusters fails with StandbyException


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

Branch: refs/heads/HDFS-10285
Commit: 701c27a7762294e1a5fb2b3ac81f5534aa37f667
Parents: 8a9f663
Author: Benoy Antony <be...@apache.org>
Authored: Fri Oct 14 10:26:39 2016 -0700
Committer: Benoy Antony <be...@apache.org>
Committed: Fri Oct 14 10:26:39 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java   | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/701c27a7/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 19de5b5..af43d56 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
@@ -91,6 +91,7 @@ import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryUtils;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.SecurityUtil;
@@ -471,6 +472,13 @@ public class WebHdfsFileSystem extends FileSystem
       }
 
       IOException re = JsonUtilClient.toRemoteException(m);
+
+      //check if exception is due to communication with a Standby name node
+      if (re.getMessage() != null && re.getMessage().endsWith(
+          StandbyException.class.getSimpleName())) {
+        LOG.trace("Detected StandbyException", re);
+        throw new IOException(re);
+      }
       // extract UGI-related exceptions and unwrap InvalidToken
       // the NN mangles these exceptions but the DN does not and may need
       // to re-fetch a token if either report the token is expired


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


[21/50] hadoop git commit: HDFS-10987. Make Decommission less expensive when lot of blocks present. Contributed by Brahma Reddy Battula.

Posted by um...@apache.org.
HDFS-10987. Make Decommission less expensive when lot of blocks present. Contributed by Brahma Reddy Battula.


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

Branch: refs/heads/HDFS-10285
Commit: 332a61fd74fd2a9874319232c583ab5d2c53ff03
Parents: fdce515
Author: Kihwal Lee <ki...@apache.org>
Authored: Thu Oct 13 13:52:49 2016 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Thu Oct 13 13:52:49 2016 -0500

----------------------------------------------------------------------
 .../blockmanagement/DecommissionManager.java    | 29 +++++++++++++++++++-
 1 file changed, 28 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/332a61fd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
----------------------------------------------------------------------
diff --git 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
index 6436fab..87b36da 100644
--- 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
@@ -389,6 +389,10 @@ public class DecommissionManager {
      */
     private int numBlocksChecked = 0;
     /**
+     * The number of blocks checked after (re)holding lock.
+     */
+    private int numBlocksCheckedPerLock = 0;
+    /**
      * The number of nodes that have been checked on this tick. Used for 
      * statistics.
      */
@@ -418,6 +422,7 @@ public class DecommissionManager {
       }
       // Reset the checked count at beginning of each iteration
       numBlocksChecked = 0;
+      numBlocksCheckedPerLock = 0;
       numNodesChecked = 0;
       // Check decom progress
       namesystem.writeLock();
@@ -451,7 +456,8 @@ public class DecommissionManager {
               iterkey).iterator();
       final LinkedList<DatanodeDescriptor> toRemove = new LinkedList<>();
 
-      while (it.hasNext() && !exceededNumBlocksPerCheck()) {
+      while (it.hasNext() && !exceededNumBlocksPerCheck() && namesystem
+          .isRunning()) {
         numNodesChecked++;
         final Map.Entry<DatanodeDescriptor, AbstractList<BlockInfo>>
             entry = it.next();
@@ -577,7 +583,28 @@ public class DecommissionManager {
       int decommissionOnlyReplicas = 0;
       int lowRedundancyInOpenFiles = 0;
       while (it.hasNext()) {
+        if (insufficientList == null
+            && numBlocksCheckedPerLock >= numBlocksPerCheck) {
+          // During fullscan insufficientlyReplicated will NOT be null, iterator
+          // will be DN's iterator. So should not yield lock, otherwise
+          // ConcurrentModificationException could occur.
+          // Once the fullscan done, iterator will be a copy. So can yield the
+          // lock.
+          // Yielding is required in case of block number is greater than the
+          // configured per-iteration-limit.
+          namesystem.writeUnlock();
+          try {
+            LOG.debug("Yielded lock during decommission check");
+            Thread.sleep(0, 500);
+          } catch (InterruptedException ignored) {
+            return;
+          }
+          // reset
+          numBlocksCheckedPerLock = 0;
+          namesystem.writeLock();
+        }
         numBlocksChecked++;
+        numBlocksCheckedPerLock++;
         final BlockInfo block = it.next();
         // Remove the block from the list if it's no longer in the block map,
         // e.g. the containing file has been deleted


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


[24/50] hadoop git commit: YARN-5717. Add tests for container-executor is_feature_enabled. Contributed by Sidharta Seethana

Posted by um...@apache.org.
YARN-5717. Add tests for container-executor is_feature_enabled. Contributed by Sidharta Seethana


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

Branch: refs/heads/HDFS-10285
Commit: cf3f43e95bf46030875137fc36da5c1fbe14250d
Parents: 0a85d07
Author: Chris Douglas <cd...@apache.org>
Authored: Thu Oct 13 20:47:49 2016 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Thu Oct 13 20:49:07 2016 -0700

----------------------------------------------------------------------
 .../impl/container-executor.c                   | 11 ++---
 .../impl/container-executor.h                   |  4 ++
 .../main/native/container-executor/impl/main.c  | 42 ++++++++--------
 .../test/test-container-executor.c              | 51 ++++++++++++++++++++
 4 files changed, 79 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf3f43e9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
index a9a7e96..8a995b1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
@@ -422,9 +422,9 @@ int change_user(uid_t user, gid_t group) {
   return 0;
 }
 
-
-static int is_feature_enabled(const char* feature_key, int default_value) {
-    char *enabled_str = get_value(feature_key, &executor_cfg);
+int is_feature_enabled(const char* feature_key, int default_value,
+                              struct configuration *cfg) {
+    char *enabled_str = get_value(feature_key, cfg);
     int enabled = default_value;
 
     if (enabled_str != NULL) {
@@ -448,15 +448,14 @@ static int is_feature_enabled(const char* feature_key, int default_value) {
     }
 }
 
-
 int is_docker_support_enabled() {
     return is_feature_enabled(DOCKER_SUPPORT_ENABLED_KEY,
-    DEFAULT_DOCKER_SUPPORT_ENABLED);
+    DEFAULT_DOCKER_SUPPORT_ENABLED, &executor_cfg);
 }
 
 int is_tc_support_enabled() {
     return is_feature_enabled(TC_SUPPORT_ENABLED_KEY,
-    DEFAULT_TC_SUPPORT_ENABLED);
+    DEFAULT_TC_SUPPORT_ENABLED, &executor_cfg);
 }
 
 char* check_docker_binary(char *docker_binary) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf3f43e9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
index 5c17b29..8ad5d47 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
@@ -264,6 +264,10 @@ int check_dir(const char* npath, mode_t st_mode, mode_t desired,
 int create_validate_dir(const char* npath, mode_t perm, const char* path,
    int finalComponent);
 
+/** Check if a feature is enabled in the specified configuration. */
+int is_feature_enabled(const char* feature_key, int default_value,
+                              struct configuration *cfg);
+
 /** Check if tc (traffic control) support is enabled in configuration. */
 int is_tc_support_enabled();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf3f43e9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
index 27a269e..47bb3b9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
@@ -43,73 +43,69 @@
 #endif
 
 static void display_usage(FILE *stream) {
-  char usage_template[4096];
-
-  usage_template[0] = '\0';
-  strcat(usage_template,
+  fprintf(stream,
     "Usage: container-executor --checksetup\n"
     "       container-executor --mount-cgroups <hierarchy> "
     "<controller=path>...\n" );
 
   if(is_tc_support_enabled()) {
-    strcat(usage_template,
+    fprintf(stream,
       "       container-executor --tc-modify-state <command-file>\n"
       "       container-executor --tc-read-state <command-file>\n"
       "       container-executor --tc-read-stats <command-file>\n" );
   } else {
-    strcat(usage_template,
+    fprintf(stream,
       "[DISABLED] container-executor --tc-modify-state <command-file>\n"
       "[DISABLED] container-executor --tc-read-state <command-file>\n"
       "[DISABLED] container-executor --tc-read-stats <command-file>\n");
   }
 
   if(is_docker_support_enabled()) {
-    strcat(usage_template,
+    fprintf(stream,
       "       container-executor --run-docker <command-file>\n");
   } else {
-    strcat(usage_template,
+    fprintf(stream,
       "[DISABLED] container-executor --run-docker <command-file>\n");
   }
 
-  strcat(usage_template,
+  fprintf(stream,
       "       container-executor <user> <yarn-user> <command> <command-args>\n"
       "       where command and command-args: \n" \
       "            initialize container:  %2d appid tokens nm-local-dirs "
       "nm-log-dirs cmd app...\n"
       "            launch container:      %2d appid containerid workdir "
-      "container-script tokens pidfile nm-local-dirs nm-log-dirs resources ");
+      "container-script tokens pidfile nm-local-dirs nm-log-dirs resources ",
+      INITIALIZE_CONTAINER, LAUNCH_CONTAINER);
 
   if(is_tc_support_enabled()) {
-    strcat(usage_template, "optional-tc-command-file\n");
+    fprintf(stream, "optional-tc-command-file\n");
   } else {
-    strcat(usage_template, "\n");
+    fprintf(stream, "\n");
   }
 
   if(is_docker_support_enabled()) {
-    strcat(usage_template,
+    fprintf(stream,
       "            launch docker container:      %2d appid containerid workdir "
       "container-script tokens pidfile nm-local-dirs nm-log-dirs "
-      "docker-command-file resources ");
+      "docker-command-file resources ", LAUNCH_DOCKER_CONTAINER);
   } else {
-    strcat(usage_template,
+    fprintf(stream,
       "[DISABLED]  launch docker container:      %2d appid containerid workdir "
       "container-script tokens pidfile nm-local-dirs nm-log-dirs "
-      "docker-command-file resources ");
+      "docker-command-file resources ", LAUNCH_DOCKER_CONTAINER);
   }
 
   if(is_tc_support_enabled()) {
-    strcat(usage_template, "optional-tc-command-file\n");
+    fprintf(stream, "optional-tc-command-file\n");
   } else {
-    strcat(usage_template, "\n");
+    fprintf(stream, "\n");
   }
 
-   strcat(usage_template,
+   fprintf(stream,
       "            signal container:      %2d container-pid signal\n"
       "            delete as user:        %2d relative-path\n"
-      "            list as user:          %2d relative-path\n");
-
-  fprintf(stream, usage_template, INITIALIZE_CONTAINER, LAUNCH_CONTAINER,
-    LAUNCH_DOCKER_CONTAINER, SIGNAL_CONTAINER, DELETE_AS_USER, LIST_AS_USER);
+      "            list as user:          %2d relative-path\n",
+      SIGNAL_CONTAINER, DELETE_AS_USER, LIST_AS_USER);
 }
 
 /* Sets up log files for normal/error logging */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf3f43e9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
index f174a9f..f7d4975 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
@@ -395,6 +395,54 @@ void test_delete_app() {
   free(dont_touch);
 }
 
+void validate_feature_enabled_value(int expected_value, const char* key,
+    int default_value, struct configuration *cfg) {
+  int value = is_feature_enabled(key, default_value, cfg);
+
+  if (value != expected_value) {
+    printf("FAIL: expected value %d for key %s but found %d\n",
+    expected_value, key, value);
+    exit(1);
+  }
+}
+
+void test_is_feature_enabled() {
+  char* filename = TEST_ROOT "/feature_flag_test.cfg";
+  FILE *file = fopen(filename, "w");
+  int disabled = 0;
+  int enabled = 1;
+  struct configuration cfg = {.size=0, .confdetails=NULL};
+
+  if (file == NULL) {
+    printf("FAIL: Could not open configuration file: %s\n", filename);
+    exit(1);
+  }
+
+  fprintf(file, "feature.name1.enabled=0\n");
+  fprintf(file, "feature.name2.enabled=1\n");
+  fprintf(file, "feature.name3.enabled=1klajdflkajdsflk\n");
+  fprintf(file, "feature.name4.enabled=asdkjfasdkljfklsdjf0\n");
+  fprintf(file, "feature.name5.enabled=-1\n");
+  fprintf(file, "feature.name6.enabled=2\n");
+  fclose(file);
+  read_config(filename, &cfg);
+
+  validate_feature_enabled_value(disabled, "feature.name1.enabled",
+      disabled, &cfg);
+  validate_feature_enabled_value(enabled, "feature.name2.enabled",
+          disabled, &cfg);
+  validate_feature_enabled_value(disabled, "feature.name3.enabled",
+          disabled, &cfg);
+  validate_feature_enabled_value(disabled, "feature.name4.enabled",
+          disabled, &cfg);
+  validate_feature_enabled_value(enabled, "feature.name5.enabled",
+          enabled, &cfg);
+  validate_feature_enabled_value(disabled, "feature.name6.enabled",
+          disabled, &cfg);
+
+
+  free_configurations(&cfg);
+}
 
 void test_delete_user() {
   printf("\nTesting delete_user\n");
@@ -1091,6 +1139,9 @@ int main(int argc, char **argv) {
   printf("\nTesting delete_app()\n");
   test_delete_app();
 
+  printf("\nTesting is_feature_enabled()\n");
+  test_is_feature_enabled();
+
   test_check_user(0);
 
 #ifdef __APPLE__


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


[32/50] hadoop git commit: HDFS-10827. When there are unrecoverable ec block groups, Namenode Web UI doesn't show the block names. Contributed by Takanobu Asanuma.

Posted by um...@apache.org.
HDFS-10827. When there are unrecoverable ec block groups, Namenode Web UI doesn't show the block names. Contributed by Takanobu Asanuma.


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

Branch: refs/heads/HDFS-10285
Commit: adb96e109f1ab4a2c3d469e716c084d0a891b951
Parents: 0007360
Author: Jing Zhao <ji...@apache.org>
Authored: Fri Oct 14 13:21:53 2016 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Fri Oct 14 13:21:53 2016 -0700

----------------------------------------------------------------------
 .../hdfs/server/namenode/FSNamesystem.java      |   2 +-
 .../server/namenode/TestNameNodeMXBean.java     | 105 +++++++++++++++++++
 2 files changed, 106 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/adb96e10/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 8c59186..563682f 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
@@ -4999,7 +4999,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         BlockInfo blk = blkIterator.next();
         final INodeFile inode = getBlockCollection(blk);
         skip++;
-        if (inode != null && blockManager.countNodes(blk).liveReplicas() == 0) {
+        if (inode != null) {
           String src = inode.getFullPathName();
           if (src.startsWith(path)){
             corruptFiles.add(new CorruptFileBlockInfo(src, blk));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/adb96e10/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 ac97a36..47f1c85 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
@@ -17,35 +17,48 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import com.google.common.base.Supplier;
 import com.google.common.util.concurrent.Uninterruptibles;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 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.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.hdfs.server.namenode.top.TopConf;
 import org.apache.hadoop.hdfs.util.HostsFileWriter;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.io.nativeio.NativeIO.POSIX.NoMlockCacheManipulator;
 import org.apache.hadoop.net.ServerSocketUtil;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.VersionInfo;
 import org.codehaus.jackson.map.ObjectMapper;
+import org.junit.Assert;
 import org.junit.Test;
 import org.mortbay.util.ajax.JSON;
 
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 import java.io.File;
+import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.net.BindException;
 import java.net.URI;
@@ -495,4 +508,96 @@ public class TestNameNodeMXBean {
           FileUtils.sizeOfDirectory(dir));
     }
   }
+
+  @Test
+  public void testVerifyMissingBlockGroupsMetrics() throws Exception {
+    MiniDFSCluster cluster = null;
+    DistributedFileSystem fs = null;
+    try {
+      Configuration conf = new HdfsConfiguration();
+      int dataBlocks = ErasureCodingPolicyManager
+          .getSystemDefaultPolicy().getNumDataUnits();
+      int parityBlocks = ErasureCodingPolicyManager
+          .getSystemDefaultPolicy().getNumParityUnits();
+      int cellSize = ErasureCodingPolicyManager
+          .getSystemDefaultPolicy().getCellSize();
+      int totalSize = dataBlocks + parityBlocks;
+      cluster = new MiniDFSCluster.Builder(conf)
+          .numDataNodes(totalSize).build();
+      fs = cluster.getFileSystem();
+
+      // create file
+      Path ecDirPath = new Path("/striped");
+      fs.mkdir(ecDirPath, FsPermission.getDirDefault());
+      fs.getClient().setErasureCodingPolicy(ecDirPath.toString(), null);
+      Path file = new Path(ecDirPath, "corrupted");
+      final int length = cellSize * dataBlocks;
+      final byte[] bytes = StripedFileTestUtil.generateBytes(length);
+      DFSTestUtil.writeFile(fs, file, bytes);
+
+      LocatedStripedBlock lsb = (LocatedStripedBlock)fs.getClient()
+          .getLocatedBlocks(file.toString(), 0, cellSize * dataBlocks).get(0);
+      final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(lsb,
+          cellSize, dataBlocks, parityBlocks);
+
+      // make an unrecoverable ec file with corrupted blocks
+      for(int i = 0; i < parityBlocks + 1; i++) {
+        int ipcPort = blks[i].getLocations()[0].getIpcPort();
+        cluster.corruptReplica(cluster.getDataNode(ipcPort),
+            blks[i].getBlock());
+      }
+
+      // disable the heart beat from DN so that the corrupted block record is
+      // kept in NameNode
+      for (DataNode dn : cluster.getDataNodes()) {
+        DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true);
+      }
+
+      // Read the file to trigger reportBadBlocks
+      try {
+        IOUtils.copyBytes(fs.open(file), new IOUtils.NullOutputStream(), conf,
+            true);
+      } catch (IOException ie) {
+        assertTrue(ie.getMessage().contains(
+            "missingChunksNum=" + (parityBlocks + 1)));
+      }
+
+      MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+      ObjectName mxbeanName = new ObjectName(
+          "Hadoop:service=NameNode,name=NameNodeInfo");
+
+      // Wait for the metrics to discover the unrecoverable block group
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          try {
+            Long numMissingBlocks =
+                (Long) mbs.getAttribute(mxbeanName, "NumberOfMissingBlocks");
+            if (numMissingBlocks == 1L) {
+              return true;
+            }
+          } catch (Exception e) {
+            Assert.fail("Caught unexpected exception.");
+          }
+          return false;
+        }
+      }, 1000, 60000);
+
+      String corruptFiles = (String) (mbs.getAttribute(mxbeanName,
+          "CorruptFiles"));
+      int numCorruptFiles = ((Object[]) JSON.parse(corruptFiles)).length;
+      assertEquals(1, numCorruptFiles);
+    } finally {
+      if (fs != null) {
+        try {
+          fs.close();
+        } catch (Exception e) {
+          throw e;
+        }
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }


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


[49/50] hadoop git commit: HDFS-10800: [SPS]: Daemon thread in Namenode to find blocks placed in other storage than what the policy specifies. Contributed by Uma Maheswara Rao G

Posted by um...@apache.org.
HDFS-10800: [SPS]: Daemon thread in Namenode to find blocks placed in other storage than what the policy specifies. Contributed by Uma Maheswara Rao G


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

Branch: refs/heads/HDFS-10285
Commit: 07299e2605067d1d9edc04b2cb8ad5cd42a78e56
Parents: 8aa5e5a
Author: Uma Maheswara Rao G <um...@intel.com>
Authored: Fri Sep 23 13:41:29 2016 -0700
Committer: Uma Maheswara Rao G <um...@intel.com>
Committed: Mon Oct 17 15:01:55 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |  41 ++
 .../server/blockmanagement/BlockManager.java    |  20 +
 .../blockmanagement/DatanodeDescriptor.java     |  38 ++
 .../server/blockmanagement/DatanodeManager.java |   7 +
 .../datanode/StoragePolicySatisfyWorker.java    |  29 +-
 .../namenode/BlockStorageMovementNeeded.java    |  53 +++
 .../server/namenode/StoragePolicySatisfier.java | 397 +++++++++++++++++++
 .../protocol/BlockStorageMovementCommand.java   |  11 +-
 .../TestStoragePolicySatisfyWorker.java         |  24 +-
 .../namenode/TestStoragePolicySatisfier.java    | 209 ++++++++++
 10 files changed, 797 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/07299e26/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 83870cf..39ee703 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
@@ -48,6 +48,7 @@ import java.util.Collection;
 import java.util.Comparator;
 import java.util.Date;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -68,6 +69,7 @@ import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -1574,4 +1576,43 @@ public class DFSUtil {
         .createKeyProviderCryptoExtension(keyProvider);
     return cryptoProvider;
   }
+
+  /**
+   * Remove the overlap between the expected types and the existing types.
+   *
+   * @param expected
+   *          - Expected storage types list.
+   * @param existing
+   *          - Existing storage types list.
+   * @param ignoreNonMovable
+   *          ignore non-movable storage types by removing them from both
+   *          expected and existing storage type list to prevent non-movable
+   *          storage from being moved.
+   * @returns if the existing types or the expected types is empty after
+   *          removing the overlap.
+   */
+  public static boolean removeOverlapBetweenStorageTypes(
+      List<StorageType> expected,
+      List<StorageType> existing, boolean ignoreNonMovable) {
+    for (Iterator<StorageType> i = existing.iterator(); i.hasNext();) {
+      final StorageType t = i.next();
+      if (expected.remove(t)) {
+        i.remove();
+      }
+    }
+    if (ignoreNonMovable) {
+      removeNonMovable(existing);
+      removeNonMovable(expected);
+    }
+    return expected.isEmpty() || existing.isEmpty();
+  }
+
+  private static void removeNonMovable(List<StorageType> types) {
+    for (Iterator<StorageType> i = types.iterator(); i.hasNext();) {
+      final StorageType t = i.next();
+      if (!t.isMovable()) {
+        i.remove();
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/07299e26/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 7b13add..ae21593 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -86,6 +86,8 @@ import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodesInPath;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
+import org.apache.hadoop.hdfs.server.namenode.BlockStorageMovementNeeded;
+import org.apache.hadoop.hdfs.server.namenode.StoragePolicySatisfier;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
@@ -341,6 +343,11 @@ public class BlockManager implements BlockStatsMXBean {
 
   private final BlockIdManager blockIdManager;
 
+  /** For satisfying block storage policies */
+  private final StoragePolicySatisfier sps;
+  private final BlockStorageMovementNeeded storageMovementNeeded =
+      new BlockStorageMovementNeeded();
+
   public BlockManager(final Namesystem namesystem, boolean haEnabled,
       final Configuration conf) throws IOException {
     this.namesystem = namesystem;
@@ -368,6 +375,7 @@ public class BlockManager implements BlockStatsMXBean {
         DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_DEFAULT)
         * 1000L);
 
+    sps = new StoragePolicySatisfier(namesystem, storageMovementNeeded, this);
     blockTokenSecretManager = createBlockTokenSecretManager(conf);
 
     this.maxCorruptFilesReturned = conf.getInt(
@@ -564,9 +572,11 @@ public class BlockManager implements BlockStatsMXBean {
     this.blockReportThread.start();
     mxBeanName = MBeans.register("NameNode", "BlockStats", this);
     bmSafeMode.activate(blockTotal);
+    sps.start();
   }
 
   public void close() {
+    sps.stop();
     bmSafeMode.close();
     try {
       replicationThread.interrupt();
@@ -4627,4 +4637,14 @@ public class BlockManager implements BlockStatsMXBean {
     }
     return i;
   }
+
+  /**
+   * Set file block collection for which storage movement needed for its blocks.
+   *
+   * @param id
+   *          - file block collection id.
+   */
+  public void satisfyStoragePolicy(long id) {
+    storageMovementNeeded.add(id);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/07299e26/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 6d163ec..47dcd74 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -41,6 +41,8 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
@@ -203,6 +205,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private final LightWeightHashSet<Block> invalidateBlocks =
       new LightWeightHashSet<>();
 
+  /** A queue of blocks for moving its storage placements by this datanode. */
+  private final Queue<List<BlockMovingInfo>> storageMovementBlocks =
+      new LinkedList<>();
+
   /* Variables for maintaining number of blocks scheduled to be written to
    * this storage. This count is approximate and might be slightly bigger
    * in case of errors (e.g. datanode does not report if an error occurs
@@ -916,5 +922,37 @@ public class DatanodeDescriptor extends DatanodeInfo {
   public boolean isRegistered() {
     return isAlive() && !forceRegistration;
   }
+
+  /**
+   * Add the block infos which needs to move its storage locations.
+   *
+   * @param storageMismatchedBlocks
+   *          - storage mismatched block infos
+   */
+  public void addBlocksToMoveStorage(
+      List<BlockMovingInfo> storageMismatchedBlocks) {
+    storageMovementBlocks.offer(storageMismatchedBlocks);
+  }
+
+  /**
+   * @return block infos which needs to move its storage locations.
+   */
+  public List<BlockMovingInfo> getBlocksToMoveStorages() {
+    return storageMovementBlocks.poll();
+  }
+
+  // TODO: we will remove this method once DN side handling integrated. We can
+  // convert the test to check real block movements instead of this ds.
+  @VisibleForTesting
+  public List<BlockMovingInfo> getStorageMovementPendingItems() {
+    List<BlockMovingInfo> flatList = new ArrayList<>();
+    Iterator<List<BlockMovingInfo>> iterator = storageMovementBlocks
+        .iterator();
+    while (iterator.hasNext()) {
+      List<BlockMovingInfo> next = iterator.next();
+      flatList.addAll(next);
+    }
+    return flatList;
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/07299e26/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 2d6547f..8aa1545 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.server.protocol.*;
 import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringStripedBlock;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.*;
 import org.apache.hadoop.net.NetworkTopology.InvalidTopologyException;
@@ -1599,6 +1600,12 @@ public class DatanodeManager {
       nodeinfo.setBalancerBandwidth(0);
     }
 
+    List<BlockMovingInfo> blocksToMoveStorages = nodeinfo
+        .getBlocksToMoveStorages();
+    if (blocksToMoveStorages != null) {
+      // TODO: create BlockStorageMovementCommand and add here in response.
+    }
+
     if (!cmds.isEmpty()) {
       return cmds.toArray(new DatanodeCommand[cmds.size()]);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/07299e26/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
index 6df4e81..fa408f6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
@@ -125,7 +126,7 @@ public class StoragePolicySatisfyWorker {
     return moverThreadPool;
   }
 
-  public void processBlockMovingTasks(long trackID,
+  public void processBlockMovingTasks(long trackID, String blockPoolID,
       List<BlockMovingInfo> blockMovingInfos) {
     Future<Void> moveCallable = null;
     for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
@@ -133,13 +134,11 @@ public class StoragePolicySatisfyWorker {
           .getSources().length == blkMovingInfo.getTargets().length;
 
       for (int i = 0; i < blkMovingInfo.getSources().length; i++) {
-        BlockMovingTask blockMovingTask =
-            new BlockMovingTask(blkMovingInfo.getBlock(),
-            blkMovingInfo.getSources()[i],
-            blkMovingInfo.getTargets()[i],
+        BlockMovingTask blockMovingTask = new BlockMovingTask(
+            blkMovingInfo.getBlock(), blockPoolID,
+            blkMovingInfo.getSources()[i], blkMovingInfo.getTargets()[i],
             blkMovingInfo.getTargetStorageTypes()[i]);
-        moveCallable = moverExecutorCompletionService
-            .submit(blockMovingTask);
+        moveCallable = moverExecutorCompletionService.submit(blockMovingTask);
         moverTaskFutures.add(moveCallable);
       }
     }
@@ -163,14 +162,16 @@ public class StoragePolicySatisfyWorker {
    * given target.
    */
   private class BlockMovingTask implements Callable<Void> {
-    private final ExtendedBlock block;
+    private final Block block;
     private final DatanodeInfo source;
     private final DatanodeInfo target;
     private final StorageType targetStorageType;
+    private String blockPoolID;
 
-    BlockMovingTask(ExtendedBlock block, DatanodeInfo source,
+    BlockMovingTask(Block block, String blockPoolID, DatanodeInfo source,
         DatanodeInfo target, StorageType targetStorageType) {
       this.block = block;
+      this.blockPoolID = blockPoolID;
       this.source = source;
       this.target = target;
       this.targetStorageType = targetStorageType;
@@ -201,12 +202,12 @@ public class StoragePolicySatisfyWorker {
 
         OutputStream unbufOut = sock.getOutputStream();
         InputStream unbufIn = sock.getInputStream();
-
+        ExtendedBlock extendedBlock = new ExtendedBlock(blockPoolID, block);
         Token<BlockTokenIdentifier> accessToken = datanode.getBlockAccessToken(
-            block, EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE));
+            extendedBlock, EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE));
 
         DataEncryptionKeyFactory keyFactory = datanode
-            .getDataEncryptionKeyFactoryForBlock(block);
+            .getDataEncryptionKeyFactoryForBlock(extendedBlock);
         IOStreamPair saslStreams = datanode.getSaslClient().socketSend(sock,
             unbufOut, unbufIn, keyFactory, accessToken, target);
         unbufOut = saslStreams.out;
@@ -215,10 +216,10 @@ public class StoragePolicySatisfyWorker {
             new BufferedOutputStream(unbufOut, ioFileBufferSize));
         in = new DataInputStream(
             new BufferedInputStream(unbufIn, ioFileBufferSize));
-        sendRequest(out, block, accessToken, source, targetStorageType);
+        sendRequest(out, extendedBlock, accessToken, source, targetStorageType);
         receiveResponse(in);
 
-        LOG.debug(
+        LOG.info(
             "Successfully moved block:{} from src:{} to destin:{} for"
                 + " satisfying storageType:{}",
             block, source, target, targetStorageType);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/07299e26/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
new file mode 100644
index 0000000..c916672
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockStorageMovementNeeded.java
@@ -0,0 +1,53 @@
+/**
+ * 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 java.util.LinkedList;
+import java.util.Queue;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * A Class to track the block collection IDs for which physical storage movement
+ * needed as per the Namespace and StorageReports from DN.
+ */
+@InterfaceAudience.Private
+public class BlockStorageMovementNeeded {
+  private final Queue<Long> storageMovementNeeded = new LinkedList<Long>();
+
+  /**
+   * Add the block collection id to tracking list for which storage movement
+   * expected if necessary.
+   *
+   * @param blockCollectionID
+   *          - block collection id, which is nothing but inode id.
+   */
+  public synchronized void add(Long blockCollectionID) {
+    storageMovementNeeded.add(blockCollectionID);
+  }
+
+  /**
+   * Gets the block collection id for which storage movements check necessary
+   * and make the movement if required.
+   *
+   * @return block collection ID
+   */
+  public synchronized Long get() {
+    return storageMovementNeeded.poll();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/07299e26/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/StoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/StoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/StoragePolicySatisfier.java
new file mode 100644
index 0000000..b5aed37
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/StoragePolicySatisfier.java
@@ -0,0 +1,397 @@
+/**
+ * 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 java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.server.balancer.Matcher;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.util.Daemon;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Setting storagePolicy on a file after the file write will only update the new
+ * storage policy type in Namespace, but physical block storage movement will
+ * not happen until user runs "Mover Tool" explicitly for such files. The
+ * StoragePolicySatisfier Daemon thread implemented for addressing the case
+ * where users may want to physically move the blocks by HDFS itself instead of
+ * running mover tool explicitly. Just calling client API to
+ * satisfyStoragePolicy on a file/dir will automatically trigger to move its
+ * physical storage locations as expected in asynchronous manner. Here Namenode
+ * will pick the file blocks which are expecting to change its storages, then it
+ * will build the mapping of source block location and expected storage type and
+ * location to move. After that this class will also prepare commands to send to
+ * Datanode for processing the physical block movements.
+ */
+@InterfaceAudience.Private
+public class StoragePolicySatisfier implements Runnable {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(StoragePolicySatisfier.class);
+  private Daemon storagePolicySatisfierThread;
+  private final Namesystem namesystem;
+  private final BlockManager blockManager;
+  private final BlockStorageMovementNeeded storageMovementNeeded;
+
+  public StoragePolicySatisfier(final Namesystem namesystem,
+      final BlockStorageMovementNeeded storageMovementNeeded,
+      final BlockManager blkManager) {
+    this.namesystem = namesystem;
+    this.storageMovementNeeded = storageMovementNeeded;
+    this.blockManager = blkManager;
+  }
+
+  /**
+   * Start storage policy satisfier demon thread.
+   */
+  public void start() {
+    storagePolicySatisfierThread = new Daemon(this);
+    storagePolicySatisfierThread.setName("StoragePolicySatisfier");
+    storagePolicySatisfierThread.start();
+  }
+
+  /**
+   * Stop storage policy satisfier demon thread.
+   */
+  public void stop() {
+    if (storagePolicySatisfierThread == null) {
+      return;
+    }
+    storagePolicySatisfierThread.interrupt();
+    try {
+      storagePolicySatisfierThread.join(3000);
+    } catch (InterruptedException ie) {
+    }
+  }
+
+  @Override
+  public void run() {
+    while (namesystem.isRunning()) {
+      try {
+        Long blockCollectionID = storageMovementNeeded.get();
+        if (blockCollectionID != null) {
+          computeAndAssignStorageMismatchedBlocksToDNs(blockCollectionID);
+        }
+        // TODO: We can think to make this as configurable later, how frequently
+        // we want to check block movements.
+        Thread.sleep(3000);
+      } catch (Throwable t) {
+        if (!namesystem.isRunning()) {
+          LOG.info("Stopping StoragePolicySatisfier.");
+          if (!(t instanceof InterruptedException)) {
+            LOG.info("StoragePolicySatisfier received an exception"
+                + " while shutting down.", t);
+          }
+          break;
+        }
+        LOG.error("StoragePolicySatisfier thread received runtime exception. "
+            + "Stopping Storage policy satisfier work", t);
+        // TODO: Just break for now. Once we implement dynamic start/stop
+        // option, we can add conditions here when to break/terminate.
+        break;
+      }
+    }
+  }
+
+  private void computeAndAssignStorageMismatchedBlocksToDNs(
+      long blockCollectionID) {
+    BlockCollection blockCollection =
+        namesystem.getBlockCollection(blockCollectionID);
+    if (blockCollection == null) {
+      return;
+    }
+    byte existingStoragePolicyID = blockCollection.getStoragePolicyID();
+    BlockStoragePolicy existingStoragePolicy =
+        blockManager.getStoragePolicy(existingStoragePolicyID);
+    if (!blockCollection.getLastBlock().isComplete()) {
+      // Postpone, currently file is under construction
+      // So, should we add back? or leave it to user
+      return;
+    }
+
+    // First datanode will be chosen as the co-ordinator node for storage
+    // movements. Later this can be optimized if needed.
+    DatanodeDescriptor coordinatorNode = null;
+    BlockInfo[] blocks = blockCollection.getBlocks();
+    List<BlockMovingInfo> blockMovingInfos = new ArrayList<BlockMovingInfo>();
+    for (int i = 0; i < blocks.length; i++) {
+      BlockInfo blockInfo = blocks[i];
+      List<StorageType> expectedStorageTypes =
+          existingStoragePolicy.chooseStorageTypes(blockInfo.getReplication());
+      DatanodeStorageInfo[] storages = blockManager.getStorages(blockInfo);
+      StorageType[] storageTypes = new StorageType[storages.length];
+      for (int j = 0; j < storages.length; j++) {
+        DatanodeStorageInfo datanodeStorageInfo = storages[j];
+        StorageType storageType = datanodeStorageInfo.getStorageType();
+        storageTypes[j] = storageType;
+      }
+      List<StorageType> existing =
+          new LinkedList<StorageType>(Arrays.asList(storageTypes));
+      if (!DFSUtil.removeOverlapBetweenStorageTypes(expectedStorageTypes,
+          existing, true)) {
+        List<StorageTypeNodePair> sourceWithStorageMap =
+            new ArrayList<StorageTypeNodePair>();
+        List<DatanodeStorageInfo> existingBlockStorages =
+            new ArrayList<DatanodeStorageInfo>(Arrays.asList(storages));
+        for (StorageType existingType : existing) {
+          Iterator<DatanodeStorageInfo> iterator =
+              existingBlockStorages.iterator();
+          while (iterator.hasNext()) {
+            DatanodeStorageInfo datanodeStorageInfo = iterator.next();
+            StorageType storageType = datanodeStorageInfo.getStorageType();
+            if (storageType == existingType) {
+              iterator.remove();
+              sourceWithStorageMap.add(new StorageTypeNodePair(storageType,
+                  datanodeStorageInfo.getDatanodeDescriptor()));
+              break;
+            }
+          }
+        }
+
+        StorageTypeNodeMap locsForExpectedStorageTypes =
+            findTargetsForExpectedStorageTypes(expectedStorageTypes);
+
+        BlockMovingInfo blockMovingInfo =
+            findSourceAndTargetToMove(blockInfo, existing, sourceWithStorageMap,
+                expectedStorageTypes, locsForExpectedStorageTypes);
+        if (coordinatorNode == null) {
+          // For now, first datanode will be chosen as the co-ordinator. Later
+          // this can be optimized if needed.
+          coordinatorNode =
+              (DatanodeDescriptor) blockMovingInfo.getSources()[0];
+        }
+        blockMovingInfos.add(blockMovingInfo);
+      }
+    }
+
+    if (blockMovingInfos.size() < 1) {
+      // TODO: Major: handle this case. I think we need retry cases to
+      // be implemented. Idea is, if some files are not getting storage movement
+      // chances, then we can just retry limited number of times and exit.
+      return;
+    }
+    coordinatorNode.addBlocksToMoveStorage(blockMovingInfos);
+  }
+
+  /**
+   * Find the good target node for each source node for which block storages was
+   * misplaced.
+   *
+   * @param blockInfo
+   *          - Block
+   * @param existing
+   *          - Existing storage types of block
+   * @param sourceWithStorageList
+   *          - Source Datanode with storages list
+   * @param expected
+   *          - Expecting storages to move
+   * @param locsForExpectedStorageTypes
+   *          - Available DNs for expected storage types
+   * @return list of block source and target node pair
+   */
+  private BlockMovingInfo findSourceAndTargetToMove(BlockInfo blockInfo,
+      List<StorageType> existing,
+      List<StorageTypeNodePair> sourceWithStorageList,
+      List<StorageType> expected,
+      StorageTypeNodeMap locsForExpectedStorageTypes) {
+    List<DatanodeInfo> sourceNodes = new ArrayList<>();
+    List<StorageType> sourceStorageTypes = new ArrayList<>();
+    List<DatanodeInfo> targetNodes = new ArrayList<>();
+    List<StorageType> targetStorageTypes = new ArrayList<>();
+    List<DatanodeDescriptor> chosenNodes = new ArrayList<>();
+    for (int i = 0; i < sourceWithStorageList.size(); i++) {
+      StorageTypeNodePair existingTypeNodePair = sourceWithStorageList.get(i);
+      StorageTypeNodePair chosenTarget =
+          chooseTargetTypeInSameNode(existingTypeNodePair.dn, expected,
+              locsForExpectedStorageTypes, chosenNodes);
+
+      if (chosenTarget == null && blockManager.getDatanodeManager()
+          .getNetworkTopology().isNodeGroupAware()) {
+        chosenTarget = chooseTarget(blockInfo, existingTypeNodePair.dn,
+            expected, Matcher.SAME_NODE_GROUP, locsForExpectedStorageTypes,
+            chosenNodes);
+      }
+
+      // Then, match nodes on the same rack
+      if (chosenTarget == null) {
+        chosenTarget =
+            chooseTarget(blockInfo, existingTypeNodePair.dn, expected,
+                Matcher.SAME_RACK, locsForExpectedStorageTypes, chosenNodes);
+      }
+
+      if (chosenTarget == null) {
+        chosenTarget =
+            chooseTarget(blockInfo, existingTypeNodePair.dn, expected,
+                Matcher.ANY_OTHER, locsForExpectedStorageTypes, chosenNodes);
+      }
+      if (null != chosenTarget) {
+        sourceNodes.add(existingTypeNodePair.dn);
+        sourceStorageTypes.add(existingTypeNodePair.storageType);
+        targetNodes.add(chosenTarget.dn);
+        targetStorageTypes.add(chosenTarget.storageType);
+        chosenNodes.add(chosenTarget.dn);
+        // TODO: We can increment scheduled block count for this node?
+      } else {
+        // TODO: Failed to ChooseTargetNodes...So let just retry. Shall we
+        // proceed without this targets? Then what should be final result?
+        // How about pack empty target, means target node could not be chosen ,
+        // so result should be RETRY_REQUIRED from DN always.
+        // Log..unable to choose target node for source datanodeDescriptor
+        sourceNodes.add(existingTypeNodePair.dn);
+        sourceStorageTypes.add(existingTypeNodePair.storageType);
+        targetNodes.add(null);
+        targetStorageTypes.add(null);
+      }
+    }
+    BlockMovingInfo blkMovingInfo = new BlockMovingInfo(blockInfo,
+        sourceNodes.toArray(new DatanodeInfo[sourceNodes.size()]),
+        targetNodes.toArray(new DatanodeInfo[targetNodes.size()]),
+        sourceStorageTypes.toArray(new StorageType[sourceStorageTypes.size()]),
+        targetStorageTypes.toArray(new StorageType[targetStorageTypes.size()]));
+    return blkMovingInfo;
+  }
+
+  /**
+   * Choose the target storage within same Datanode if possible.
+   *
+   * @param locsForExpectedStorageTypes
+   * @param chosenNodes
+   */
+  private StorageTypeNodePair chooseTargetTypeInSameNode(
+      DatanodeDescriptor source, List<StorageType> targetTypes,
+      StorageTypeNodeMap locsForExpectedStorageTypes,
+      List<DatanodeDescriptor> chosenNodes) {
+    for (StorageType t : targetTypes) {
+      DatanodeStorageInfo chooseStorage4Block =
+          source.chooseStorage4Block(t, 0);
+      if (chooseStorage4Block != null) {
+        return new StorageTypeNodePair(t, source);
+      }
+    }
+    return null;
+  }
+
+  private StorageTypeNodePair chooseTarget(Block block,
+      DatanodeDescriptor source, List<StorageType> targetTypes, Matcher matcher,
+      StorageTypeNodeMap locsForExpectedStorageTypes,
+      List<DatanodeDescriptor> chosenNodes) {
+    for (StorageType t : targetTypes) {
+      List<DatanodeDescriptor> nodesWithStorages =
+          locsForExpectedStorageTypes.getNodesWithStorages(t);
+      Collections.shuffle(nodesWithStorages);
+      for (DatanodeDescriptor target : nodesWithStorages) {
+        if (!chosenNodes.contains(target) && matcher.match(
+            blockManager.getDatanodeManager().getNetworkTopology(), source,
+            target)) {
+          if (null != target.chooseStorage4Block(t, block.getNumBytes())) {
+            return new StorageTypeNodePair(t, target);
+          }
+        }
+      }
+    }
+    return null;
+  }
+
+  private static class StorageTypeNodePair {
+    public StorageType storageType = null;
+    public DatanodeDescriptor dn = null;
+
+    public StorageTypeNodePair(StorageType storageType, DatanodeDescriptor dn) {
+      this.storageType = storageType;
+      this.dn = dn;
+    }
+  }
+
+  private StorageTypeNodeMap findTargetsForExpectedStorageTypes(
+      List<StorageType> expected) {
+    StorageTypeNodeMap targetMap = new StorageTypeNodeMap();
+    List<DatanodeDescriptor> reports = blockManager.getDatanodeManager()
+        .getDatanodeListForReport(DatanodeReportType.LIVE);
+    for (DatanodeDescriptor dn : reports) {
+      StorageReport[] storageReports = dn.getStorageReports();
+      for (StorageReport storageReport : storageReports) {
+        StorageType t = storageReport.getStorage().getStorageType();
+        if (expected.contains(t)) {
+          final long maxRemaining = getMaxRemaining(dn.getStorageReports(), t);
+          if (maxRemaining > 0L) {
+            targetMap.add(t, dn);
+          }
+        }
+      }
+    }
+    return targetMap;
+  }
+
+  private static long getMaxRemaining(StorageReport[] storageReports,
+      StorageType t) {
+    long max = 0L;
+    for (StorageReport r : storageReports) {
+      if (r.getStorage().getStorageType() == t) {
+        if (r.getRemaining() > max) {
+          max = r.getRemaining();
+        }
+      }
+    }
+    return max;
+  }
+
+  private static class StorageTypeNodeMap {
+    private final EnumMap<StorageType, List<DatanodeDescriptor>> typeNodeMap =
+        new EnumMap<StorageType, List<DatanodeDescriptor>>(StorageType.class);
+
+    private void add(StorageType t, DatanodeDescriptor dn) {
+      List<DatanodeDescriptor> nodesWithStorages = getNodesWithStorages(t);
+      LinkedList<DatanodeDescriptor> value = null;
+      if (nodesWithStorages == null) {
+        value = new LinkedList<DatanodeDescriptor>();
+        value.add(dn);
+        typeNodeMap.put(t, value);
+      } else {
+        nodesWithStorages.add(dn);
+      }
+    }
+
+    /**
+     * @param type
+     *          - Storage type
+     * @return datanodes which has the given storage type
+     */
+    private List<DatanodeDescriptor> getNodesWithStorages(StorageType type) {
+      return typeNodeMap.get(type);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/07299e26/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
index 42ba265..c1ab800 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
@@ -20,8 +20,8 @@ package org.apache.hadoop.hdfs.server.protocol;
 import java.util.Arrays;
 
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 
 /**
  * A BlockStorageMovementCommand is an instruction to a DataNode to move the
@@ -35,7 +35,6 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
  * NameNode about the movement status.
  */
 public class BlockStorageMovementCommand extends DatanodeCommand {
-
   // TODO: constructor needs to be refined based on the block movement data
   // structure.
   BlockStorageMovementCommand(int action) {
@@ -46,13 +45,13 @@ public class BlockStorageMovementCommand extends DatanodeCommand {
    * Stores block to storage info that can be used for block movement.
    */
   public static class BlockMovingInfo {
-    private ExtendedBlock blk;
+    private Block blk;
     private DatanodeInfo[] sourceNodes;
     private StorageType[] sourceStorageTypes;
     private DatanodeInfo[] targetNodes;
     private StorageType[] targetStorageTypes;
 
-    public BlockMovingInfo(ExtendedBlock block,
+    public BlockMovingInfo(Block block,
         DatanodeInfo[] sourceDnInfos, DatanodeInfo[] targetDnInfos,
         StorageType[] srcStorageTypes, StorageType[] targetStorageTypes) {
       this.blk = block;
@@ -62,11 +61,11 @@ public class BlockStorageMovementCommand extends DatanodeCommand {
       this.targetStorageTypes = targetStorageTypes;
     }
 
-    public void addBlock(ExtendedBlock block) {
+    public void addBlock(Block block) {
       this.blk = block;
     }
 
-    public ExtendedBlock getBlock() {
+    public Block getBlock() {
       return this.blk;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/07299e26/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStoragePolicySatisfyWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStoragePolicySatisfyWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStoragePolicySatisfyWorker.java
index 692847d..0575b9f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStoragePolicySatisfyWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStoragePolicySatisfyWorker.java
@@ -30,8 +30,8 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
@@ -70,14 +70,14 @@ public class TestStoragePolicySatisfyWorker {
   public void testMoveSingleBlockToAnotherDatanode() throws Exception {
     final Configuration conf = new HdfsConfiguration();
     initConf(conf);
-    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-        .numDataNodes(4)
-        .storageTypes(
-            new StorageType[][] {{StorageType.DISK, StorageType.ARCHIVE},
-                {StorageType.DISK, StorageType.ARCHIVE},
-                {StorageType.DISK, StorageType.ARCHIVE},
-                {StorageType.DISK, StorageType.ARCHIVE}})
-        .build();
+    final MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(4)
+            .storageTypes(
+                new StorageType[][]{{StorageType.DISK, StorageType.ARCHIVE},
+                    {StorageType.DISK, StorageType.ARCHIVE},
+                    {StorageType.ARCHIVE, StorageType.ARCHIVE},
+                    {StorageType.ARCHIVE, StorageType.ARCHIVE}})
+            .build();
     try {
       cluster.waitActive();
       final DistributedFileSystem dfs = cluster.getFileSystem();
@@ -107,12 +107,12 @@ public class TestStoragePolicySatisfyWorker {
           src);
       List<BlockMovingInfo> blockMovingInfos = new ArrayList<>();
       BlockMovingInfo blockMovingInfo = prepareBlockMovingInfo(
-          lb.getBlock(), lb.getLocations()[0], targetDnInfo,
+          lb.getBlock().getLocalBlock(), lb.getLocations()[0], targetDnInfo,
           lb.getStorageTypes()[0], StorageType.ARCHIVE);
       blockMovingInfos.add(blockMovingInfo);
       INode inode = cluster.getNamesystem().getFSDirectory().getINode(file);
       worker.processBlockMovingTasks(inode.getId(),
-          blockMovingInfos);
+          cluster.getNamesystem().getBlockPoolId(), blockMovingInfos);
       cluster.triggerHeartbeats();
 
       // Wait till NameNode notified about the block location details
@@ -149,7 +149,7 @@ public class TestStoragePolicySatisfyWorker {
     }, 100, timeout);
   }
 
-  BlockMovingInfo prepareBlockMovingInfo(ExtendedBlock block,
+  BlockMovingInfo prepareBlockMovingInfo(Block block,
       DatanodeInfo src, DatanodeInfo destin, StorageType storageType,
       StorageType targetStorageType) {
     return new BlockMovingInfo(block, new DatanodeInfo[] {src},

http://git-wip-us.apache.org/repos/asf/hadoop/blob/07299e26/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfier.java
new file mode 100644
index 0000000..b61814d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfier.java
@@ -0,0 +1,209 @@
+/**
+ * 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 java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Supplier;
+
+/**
+ * Tests that StoragePolicySatisfier daemon is able to check the blocks to be
+ * moved and finding its suggested target locations to move.
+ */
+public class TestStoragePolicySatisfier {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestStoragePolicySatisfier.class);
+  private final Configuration config = new HdfsConfiguration();
+  private StorageType[][] allDiskTypes =
+      new StorageType[][]{{StorageType.DISK, StorageType.DISK},
+          {StorageType.DISK, StorageType.DISK},
+          {StorageType.DISK, StorageType.DISK}};
+  private MiniDFSCluster hdfsCluster = null;
+  final private int numOfDatanodes = 3;
+  final private int storagesPerDatanode = 2;
+  final private long capacity = 2 * 256 * 1024 * 1024;
+  final private String file = "/testMoveWhenStoragePolicyNotSatisfying";
+  private DistributedFileSystem distributedFS = null;
+
+  @Before
+  public void setUp() throws IOException {
+    config.setLong("dfs.block.size", 1024);
+    hdfsCluster = startCluster(config, allDiskTypes, numOfDatanodes,
+        storagesPerDatanode, capacity);
+    distributedFS = hdfsCluster.getFileSystem();
+    writeContent(distributedFS, file);
+  }
+
+  @Test(timeout = 300000)
+  public void testWhenStoragePolicySetToCOLD()
+      throws Exception {
+
+    try {
+      // Change policy to ALL_SSD
+      distributedFS.setStoragePolicy(new Path(file), "COLD");
+      Set<DatanodeDescriptor> previousNodes =
+          hdfsCluster.getNameNode().getNamesystem().getBlockManager()
+              .getDatanodeManager().getDatanodes();
+      FSNamesystem namesystem = hdfsCluster.getNamesystem();
+      INode inode = namesystem.getFSDirectory().getINode(file);
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.ARCHIVE, StorageType.ARCHIVE},
+              {StorageType.ARCHIVE, StorageType.ARCHIVE},
+              {StorageType.ARCHIVE, StorageType.ARCHIVE}};
+      startAdditionalDNs(config, 3, numOfDatanodes, newtypes,
+          storagesPerDatanode, capacity, hdfsCluster);
+
+      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+
+      hdfsCluster.triggerHeartbeats();
+      // Wait till namenode notified about the block location details
+      waitExpectedStorageType(StorageType.ARCHIVE, distributedFS, previousNodes,
+          6, 30000);
+    } finally {
+      hdfsCluster.shutdown();
+    }
+  }
+
+  @Test(timeout = 300000)
+  public void testWhenStoragePolicySetToALLSSD()
+      throws Exception {
+    try {
+      // Change policy to ALL_SSD
+      distributedFS.setStoragePolicy(new Path(file), "ALL_SSD");
+      Set<DatanodeDescriptor> previousNodes =
+          hdfsCluster.getNameNode().getNamesystem().getBlockManager()
+              .getDatanodeManager().getDatanodes();
+      FSNamesystem namesystem = hdfsCluster.getNamesystem();
+      INode inode = namesystem.getFSDirectory().getINode(file);
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.SSD, StorageType.DISK},
+              {StorageType.SSD, StorageType.DISK},
+              {StorageType.SSD, StorageType.DISK}};
+
+      // Making sure SDD based nodes added to cluster. Adding SSD based
+      // datanodes.
+      startAdditionalDNs(config, 3, numOfDatanodes, newtypes,
+          storagesPerDatanode, capacity, hdfsCluster);
+      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+      hdfsCluster.triggerHeartbeats();
+      // Wait till StorgePolicySatisfier Identified that block to move to SSD
+      // areas
+      waitExpectedStorageType(StorageType.SSD, distributedFS, previousNodes, 6,
+          30000);
+    } finally {
+      hdfsCluster.shutdown();
+    }
+  }
+
+  private void writeContent(final DistributedFileSystem dfs,
+      final String fileName) throws IOException {
+    // write to DISK
+    final FSDataOutputStream out = dfs.create(new Path(fileName));
+    for (int i = 0; i < 1000; i++) {
+      out.writeChars("t");
+    }
+    out.close();
+  }
+
+  private void startAdditionalDNs(final Configuration conf,
+      int newNodesRequired, int existingNodesNum, StorageType[][] newTypes,
+      int storagesPerDatanode, long capacity, final MiniDFSCluster cluster)
+          throws IOException {
+    long[][] capacities;
+    existingNodesNum += newNodesRequired;
+    capacities = new long[newNodesRequired][storagesPerDatanode];
+    for (int i = 0; i < newNodesRequired; i++) {
+      for (int j = 0; j < storagesPerDatanode; j++) {
+        capacities[i][j] = capacity;
+      }
+    }
+
+    cluster.startDataNodes(conf, newNodesRequired, newTypes, true, null, null,
+        null, capacities, null, false, false, false, null);
+    cluster.triggerHeartbeats();
+  }
+
+  private MiniDFSCluster startCluster(final Configuration conf,
+      StorageType[][] storageTypes, int numberOfDatanodes, int storagesPerDn,
+      long nodeCapacity) throws IOException {
+    long[][] capacities = new long[numberOfDatanodes][storagesPerDn];
+    for (int i = 0; i < numberOfDatanodes; i++) {
+      for (int j = 0; j < storagesPerDn; j++) {
+        capacities[i][j] = nodeCapacity;
+      }
+    }
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(numberOfDatanodes).storagesPerDatanode(storagesPerDn)
+        .storageTypes(storageTypes).storageCapacities(capacities).build();
+    cluster.waitActive();
+    return cluster;
+  }
+
+  // TODO: this assertion can be changed to end to end based assertion later
+  // when DN side processing work integrated to this work.
+  private void waitExpectedStorageType(final StorageType expectedStorageType,
+      final DistributedFileSystem dfs,
+      final Set<DatanodeDescriptor> previousNodes, int expectedArchiveCount,
+      int timeout) throws Exception {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        Iterator<DatanodeDescriptor> iterator = previousNodes.iterator();
+        int archiveCount = 0;
+        while (iterator.hasNext()) {
+          DatanodeDescriptor dn = iterator.next();
+          List<BlockMovingInfo> pendingItemsToMove =
+              dn.getStorageMovementPendingItems();
+          for (BlockMovingInfo blkInfoToMoveStorage : pendingItemsToMove) {
+            StorageType[] targetStorageTypes =
+                blkInfoToMoveStorage.getTargetStorageTypes();
+            for (StorageType storageType : targetStorageTypes) {
+              if (storageType == expectedStorageType) {
+                archiveCount++;
+              }
+            }
+          }
+        }
+        LOG.info(
+            expectedStorageType + " replica count, expected={} and actual={}",
+            expectedArchiveCount, archiveCount);
+        return expectedArchiveCount == archiveCount;
+      }
+    }, 100, timeout);
+  }
+}


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


[34/50] hadoop git commit: HADOOP-13721. Remove stale method ViewFileSystem#getTrashCanLocation. Contributed by Manoj Govindassamy.

Posted by um...@apache.org.
HADOOP-13721. Remove stale method ViewFileSystem#getTrashCanLocation. Contributed by Manoj Govindassamy.


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

Branch: refs/heads/HDFS-10285
Commit: aee538be6c2ab324de4d7834cd3347959272de01
Parents: 8c520a2
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Oct 14 14:08:31 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Oct 14 14:08:31 2016 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java  | 6 ------
 1 file changed, 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aee538be/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
index edc59ab..f6947ff 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
@@ -220,12 +220,6 @@ public class ViewFileSystem extends FileSystem {
     this(FsConstants.VIEWFS_URI, conf);
   }
   
-  public Path getTrashCanLocation(final Path f) throws FileNotFoundException {
-    final InodeTree.ResolveResult<FileSystem> res = 
-      fsState.resolve(getUriPath(f), true);
-    return res.isInternalDir() ? null : res.targetFileSystem.getHomeDirectory();
-  }
-  
   @Override
   public URI getUri() {
     return myUri;


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


[03/50] hadoop git commit: HADOOP-13684. Snappy may complain Hadoop is built without snappy if libhadoop is not found. Contributed by Wei-Chiu Chuang.

Posted by um...@apache.org.
HADOOP-13684. Snappy may complain Hadoop is built without snappy if libhadoop is not found. Contributed by Wei-Chiu Chuang.


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

Branch: refs/heads/HDFS-10285
Commit: 4b32b1420d98ea23460d05ae94f2698109b3d6f7
Parents: 2fb392a
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Tue Oct 11 13:21:33 2016 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Tue Oct 11 13:21:33 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/io/compress/SnappyCodec.java  | 30 +++++++++++---------
 1 file changed, 16 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b32b142/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/SnappyCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/SnappyCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/SnappyCodec.java
index 2a9c5d0..20a4cd6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/SnappyCodec.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/SnappyCodec.java
@@ -60,20 +60,22 @@ public class SnappyCodec implements Configurable, CompressionCodec, DirectDecomp
    * Are the native snappy libraries loaded & initialized?
    */
   public static void checkNativeCodeLoaded() {
-      if (!NativeCodeLoader.isNativeCodeLoaded() ||
-          !NativeCodeLoader.buildSupportsSnappy()) {
-        throw new RuntimeException("native snappy library not available: " +
-            "this version of libhadoop was built without " +
-            "snappy support.");
-      }
-      if (!SnappyCompressor.isNativeCodeLoaded()) {
-        throw new RuntimeException("native snappy library not available: " +
-            "SnappyCompressor has not been loaded.");
-      }
-      if (!SnappyDecompressor.isNativeCodeLoaded()) {
-        throw new RuntimeException("native snappy library not available: " +
-            "SnappyDecompressor has not been loaded.");
-      }
+    if (!NativeCodeLoader.buildSupportsSnappy()) {
+      throw new RuntimeException("native snappy library not available: " +
+          "this version of libhadoop was built without " +
+          "snappy support.");
+    }
+    if (!NativeCodeLoader.isNativeCodeLoaded()) {
+      throw new RuntimeException("Failed to load libhadoop.");
+    }
+    if (!SnappyCompressor.isNativeCodeLoaded()) {
+      throw new RuntimeException("native snappy library not available: " +
+          "SnappyCompressor has not been loaded.");
+    }
+    if (!SnappyDecompressor.isNativeCodeLoaded()) {
+      throw new RuntimeException("native snappy library not available: " +
+          "SnappyDecompressor has not been loaded.");
+    }
   }
   
   public static boolean isNativeCodeLoaded() {


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


[50/50] hadoop git commit: HDFS-10801. [SPS]: Protocol buffer changes for sending storage movement commands from NN to DN. Contributed by Rakesh R

Posted by um...@apache.org.
HDFS-10801. [SPS]: Protocol buffer changes for sending storage movement commands from NN to DN. Contributed by Rakesh R


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

Branch: refs/heads/HDFS-10285
Commit: 13a8050350962d939fd808cf2e20604311b81b8e
Parents: 07299e2
Author: Rakesh Radhakrishnan <ra...@apache.org>
Authored: Tue Oct 11 11:44:06 2016 +0530
Committer: Uma Maheswara Rao G <um...@intel.com>
Committed: Mon Oct 17 16:05:26 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 90 ++++++++++++++++++++
 .../blockmanagement/DatanodeDescriptor.java     | 15 ----
 .../server/blockmanagement/DatanodeManager.java | 13 ++-
 .../hdfs/server/datanode/BPOfferService.java    |  8 ++
 .../hadoop/hdfs/server/datanode/DataNode.java   |  7 ++
 .../datanode/StoragePolicySatisfyWorker.java    | 22 ++++-
 .../protocol/BlockStorageMovementCommand.java   | 71 ++++++++++++++-
 .../hdfs/server/protocol/DatanodeProtocol.java  |  1 +
 .../src/main/proto/DatanodeProtocol.proto       | 22 +++++
 .../namenode/TestStoragePolicySatisfier.java    | 86 +++++++++++--------
 10 files changed, 273 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/13a80503/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 78371f5..28c7590 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
@@ -48,6 +48,8 @@ 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.DatanodeProtocolProtos.BlockStorageMovementCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockStorageMovementProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECReconstructionInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
@@ -89,6 +91,8 @@ import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.Block
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringStripedBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations;
@@ -457,6 +461,8 @@ public class PBHelper {
       return PBHelper.convert(proto.getBlkIdCmd());
     case BlockECReconstructionCommand:
       return PBHelper.convert(proto.getBlkECReconstructionCmd());
+    case BlockStorageMovementCommand:
+      return PBHelper.convert(proto.getBlkStorageMovementCmd());
     default:
       return null;
     }
@@ -591,6 +597,11 @@ public class PBHelper {
           .setBlkECReconstructionCmd(
               convert((BlockECReconstructionCommand) datanodeCommand));
       break;
+    case DatanodeProtocol.DNA_BLOCK_STORAGE_MOVEMENT:
+      builder.setCmdType(DatanodeCommandProto.Type.BlockStorageMovementCommand)
+          .setBlkStorageMovementCmd(
+              convert((BlockStorageMovementCommand) datanodeCommand));
+      break;
     case DatanodeProtocol.DNA_UNKNOWN: //Not expected
     default:
       builder.setCmdType(DatanodeCommandProto.Type.NullDatanodeCommand);
@@ -963,4 +974,83 @@ public class PBHelper {
         DatanodeProtocol.DNA_ERASURE_CODING_RECONSTRUCTION,
         blkECReconstructionInfos);
   }
+
+  private static BlockStorageMovementCommandProto convert(
+      BlockStorageMovementCommand blkStorageMovementCmd) {
+    BlockStorageMovementCommandProto.Builder builder =
+        BlockStorageMovementCommandProto.newBuilder();
+
+    builder.setTrackID(blkStorageMovementCmd.getTrackID());
+    builder.setBlockPoolId(blkStorageMovementCmd.getBlockPoolId());
+    Collection<BlockMovingInfo> blockMovingInfos = blkStorageMovementCmd
+        .getBlockMovingTasks();
+    for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
+      builder.addBlockStorageMovement(
+          convertBlockMovingInfo(blkMovingInfo));
+    }
+    return builder.build();
+  }
+
+  private static BlockStorageMovementProto convertBlockMovingInfo(
+      BlockMovingInfo blkMovingInfo) {
+    BlockStorageMovementProto.Builder builder = BlockStorageMovementProto
+        .newBuilder();
+    builder.setBlock(PBHelperClient.convert(blkMovingInfo.getBlock()));
+
+    DatanodeInfo[] sourceDnInfos = blkMovingInfo.getSources();
+    builder.setSourceDnInfos(convertToDnInfosProto(sourceDnInfos));
+
+    DatanodeInfo[] targetDnInfos = blkMovingInfo.getTargets();
+    builder.setTargetDnInfos(convertToDnInfosProto(targetDnInfos));
+
+    StorageType[] sourceStorageTypes = blkMovingInfo.getSourceStorageTypes();
+    builder.setSourceStorageTypes(convertStorageTypesProto(sourceStorageTypes));
+
+    StorageType[] targetStorageTypes = blkMovingInfo.getTargetStorageTypes();
+    builder.setTargetStorageTypes(convertStorageTypesProto(targetStorageTypes));
+
+    return builder.build();
+  }
+
+  private static DatanodeCommand convert(
+      BlockStorageMovementCommandProto blkStorageMovementCmdProto) {
+    Collection<BlockMovingInfo> blockMovingInfos = new ArrayList<>();
+    List<BlockStorageMovementProto> blkSPSatisfyList =
+        blkStorageMovementCmdProto.getBlockStorageMovementList();
+    for (BlockStorageMovementProto blkSPSatisfy : blkSPSatisfyList) {
+      blockMovingInfos.add(convertBlockMovingInfo(blkSPSatisfy));
+    }
+    return new BlockStorageMovementCommand(
+        DatanodeProtocol.DNA_BLOCK_STORAGE_MOVEMENT,
+        blkStorageMovementCmdProto.getTrackID(),
+        blkStorageMovementCmdProto.getBlockPoolId(), blockMovingInfos);
+  }
+
+  private static BlockMovingInfo convertBlockMovingInfo(
+      BlockStorageMovementProto blockStoragePolicySatisfyProto) {
+    BlockProto blockProto = blockStoragePolicySatisfyProto.getBlock();
+    Block block = PBHelperClient.convert(blockProto);
+
+    DatanodeInfosProto sourceDnInfosProto = blockStoragePolicySatisfyProto
+        .getSourceDnInfos();
+    DatanodeInfo[] sourceDnInfos = PBHelperClient.convert(sourceDnInfosProto);
+
+    DatanodeInfosProto targetDnInfosProto = blockStoragePolicySatisfyProto
+        .getTargetDnInfos();
+    DatanodeInfo[] targetDnInfos = PBHelperClient.convert(targetDnInfosProto);
+
+    StorageTypesProto srcStorageTypesProto = blockStoragePolicySatisfyProto
+        .getSourceStorageTypes();
+    StorageType[] srcStorageTypes = PBHelperClient.convertStorageTypes(
+        srcStorageTypesProto.getStorageTypesList(),
+        srcStorageTypesProto.getStorageTypesList().size());
+
+    StorageTypesProto targetStorageTypesProto = blockStoragePolicySatisfyProto
+        .getTargetStorageTypes();
+    StorageType[] targetStorageTypes = PBHelperClient.convertStorageTypes(
+        targetStorageTypesProto.getStorageTypesList(),
+        targetStorageTypesProto.getStorageTypesList().size());
+    return new BlockMovingInfo(block, sourceDnInfos, targetDnInfos,
+        srcStorageTypes, targetStorageTypes);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13a80503/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 47dcd74..8bbc98a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -940,19 +940,4 @@ public class DatanodeDescriptor extends DatanodeInfo {
   public List<BlockMovingInfo> getBlocksToMoveStorages() {
     return storageMovementBlocks.poll();
   }
-
-  // TODO: we will remove this method once DN side handling integrated. We can
-  // convert the test to check real block movements instead of this ds.
-  @VisibleForTesting
-  public List<BlockMovingInfo> getStorageMovementPendingItems() {
-    List<BlockMovingInfo> flatList = new ArrayList<>();
-    Iterator<List<BlockMovingInfo>> iterator = storageMovementBlocks
-        .iterator();
-    while (iterator.hasNext()) {
-      List<BlockMovingInfo> next = iterator.next();
-      flatList.addAll(next);
-    }
-    return flatList;
-  }
 }
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13a80503/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 8aa1545..026ad72 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -1600,10 +1600,17 @@ public class DatanodeManager {
       nodeinfo.setBalancerBandwidth(0);
     }
 
-    List<BlockMovingInfo> blocksToMoveStorages = nodeinfo
+    // check pending block storage movement tasks
+    List<BlockMovingInfo> pendingBlockMovementList = nodeinfo
         .getBlocksToMoveStorages();
-    if (blocksToMoveStorages != null) {
-      // TODO: create BlockStorageMovementCommand and add here in response.
+    if (pendingBlockMovementList != null) {
+      // TODO: trackID is used to track the block movement sends to coordinator
+      // datanode. Need to implement tracking logic. Temporarily, using a
+      // constant value -1.
+      long trackID = -1;
+      cmds.add(new BlockStorageMovementCommand(
+          DatanodeProtocol.DNA_BLOCK_STORAGE_MOVEMENT, trackID, blockPoolId,
+          pendingBlockMovementList));
     }
 
     if (!cmds.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13a80503/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
index 00102eb..f5f6738 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
@@ -732,6 +732,13 @@ class BPOfferService {
           ((BlockECReconstructionCommand) cmd).getECTasks();
       dn.getErasureCodingWorker().processErasureCodingTasks(ecTasks);
       break;
+    case DatanodeProtocol.DNA_BLOCK_STORAGE_MOVEMENT:
+      LOG.info("DatanodeCommand action: DNA_BLOCK_STORAGE_MOVEMENT");
+      BlockStorageMovementCommand blkSPSCmd = (BlockStorageMovementCommand) cmd;
+      dn.getStoragePolicySatisfyWorker().processBlockMovingTasks(
+          blkSPSCmd.getTrackID(), blkSPSCmd.getBlockPoolId(),
+          blkSPSCmd.getBlockMovingTasks());
+      break;
     default:
       LOG.warn("Unknown DatanodeCommand action: " + cmd.getAction());
     }
@@ -762,6 +769,7 @@ class BPOfferService {
     case DatanodeProtocol.DNA_CACHE:
     case DatanodeProtocol.DNA_UNCACHE:
     case DatanodeProtocol.DNA_ERASURE_CODING_RECONSTRUCTION:
+    case DatanodeProtocol.DNA_BLOCK_STORAGE_MOVEMENT:
       LOG.warn("Got a command from standby NN - ignoring command:" + cmd.getAction());
       break;
     default:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13a80503/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 8f65efe..f099304 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -381,6 +381,7 @@ public class DataNode extends ReconfigurableBase
   private String dnUserName = null;
   private BlockRecoveryWorker blockRecoveryWorker;
   private ErasureCodingWorker ecWorker;
+  private StoragePolicySatisfyWorker storagePolicySatisfyWorker;
   private final Tracer tracer;
   private final TracerConfigurationManager tracerConfigurationManager;
   private static final int NUM_CORES = Runtime.getRuntime()
@@ -1359,6 +1360,8 @@ public class DataNode extends ReconfigurableBase
 
     ecWorker = new ErasureCodingWorker(getConf(), this);
     blockRecoveryWorker = new BlockRecoveryWorker(this);
+    storagePolicySatisfyWorker =
+        new StoragePolicySatisfyWorker(getConf(), this);
 
     blockPoolManager = new BlockPoolManager(this);
     blockPoolManager.refreshNamenodes(getConf());
@@ -3471,4 +3474,8 @@ public class DataNode extends ReconfigurableBase
   void setBlockScanner(BlockScanner blockScanner) {
     this.blockScanner = blockScanner;
   }
+
+  StoragePolicySatisfyWorker getStoragePolicySatisfyWorker() {
+    return storagePolicySatisfyWorker;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13a80503/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
index fa408f6..2c99963 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StoragePolicySatisfyWorker.java
@@ -28,6 +28,7 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.net.Socket;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.concurrent.Callable;
@@ -126,8 +127,25 @@ public class StoragePolicySatisfyWorker {
     return moverThreadPool;
   }
 
+  /**
+   * Handles the given set of block movement tasks. This will iterate over the
+   * block movement list and submit each block movement task asynchronously in a
+   * separate thread. Each task will move the block replica to the target node &
+   * wait for the completion.
+   *
+   * TODO: Presently this function is a blocking call, this has to be refined by
+   * moving the tracking logic to another tracker thread. HDFS-10884 jira
+   * addresses the same.
+   *
+   * @param trackID
+   *          unique tracking identifier
+   * @param blockPoolID
+   *          block pool ID
+   * @param blockMovingInfos
+   *          list of blocks to be moved
+   */
   public void processBlockMovingTasks(long trackID, String blockPoolID,
-      List<BlockMovingInfo> blockMovingInfos) {
+      Collection<BlockMovingInfo> blockMovingInfos) {
     Future<Void> moveCallable = null;
     for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
       assert blkMovingInfo
@@ -143,8 +161,6 @@ public class StoragePolicySatisfyWorker {
       }
     }
 
-    // TODO: Presently this function act as a blocking call, this has to be
-    // refined by moving the tracking logic to another tracker thread.
     for (int i = 0; i < moverTaskFutures.size(); i++) {
       try {
         moveCallable = moverExecutorCompletionService.take();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13a80503/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
index c1ab800..7c97f1a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.protocol;
 
 import java.util.Arrays;
+import java.util.Collection;
 
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -33,12 +34,60 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
  * {@link org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker}
  * service. After the block movement this DataNode sends response back to the
  * NameNode about the movement status.
+ *
+ * The coordinator datanode will use 'trackId' identifier to coordinate the block
+ * movement of the given set of blocks. TrackId is a unique identifier that
+ * represents a group of blocks. Namenode will generate this unique value and
+ * send it to the coordinator datanode along with the
+ * BlockStorageMovementCommand. Datanode will monitor the completion of the
+ * block movements that grouped under this trackId and notifies Namenode about
+ * the completion status.
  */
 public class BlockStorageMovementCommand extends DatanodeCommand {
-  // TODO: constructor needs to be refined based on the block movement data
-  // structure.
-  BlockStorageMovementCommand(int action) {
+  private final long trackID;
+  private final String blockPoolId;
+  private final Collection<BlockMovingInfo> blockMovingTasks;
+
+  /**
+   * Block storage movement command constructor.
+   *
+   * @param action
+   *          protocol specific action
+   * @param trackID
+   *          unique identifier to monitor the given set of block movements
+   * @param blockPoolId
+   *          block pool ID
+   * @param blockMovingInfos
+   *          block to storage info that will be used for movement
+   */
+  public BlockStorageMovementCommand(int action, long trackID,
+      String blockPoolId, Collection<BlockMovingInfo> blockMovingInfos) {
     super(action);
+    this.trackID = trackID;
+    this.blockPoolId = blockPoolId;
+    this.blockMovingTasks = blockMovingInfos;
+  }
+
+  /**
+   * Returns trackID, which will be used to monitor the block movement assigned
+   * to this coordinator datanode.
+   */
+  public long getTrackID() {
+    return trackID;
+  }
+
+  /**
+   * Returns block pool ID.
+   */
+  public String getBlockPoolId() {
+    return blockPoolId;
+  }
+
+  /**
+   * Returns the list of blocks to be moved.
+   */
+  public Collection<BlockMovingInfo> getBlockMovingTasks() {
+    return blockMovingTasks;
   }
 
   /**
@@ -47,10 +96,24 @@ public class BlockStorageMovementCommand extends DatanodeCommand {
   public static class BlockMovingInfo {
     private Block blk;
     private DatanodeInfo[] sourceNodes;
-    private StorageType[] sourceStorageTypes;
     private DatanodeInfo[] targetNodes;
+    private StorageType[] sourceStorageTypes;
     private StorageType[] targetStorageTypes;
 
+    /**
+     * Block to storage info constructor.
+     *
+     * @param block
+     *          block
+     * @param sourceDnInfos
+     *          node that can be the sources of a block move
+     * @param targetDnInfos
+     *          target datanode info
+     * @param srcStorageTypes
+     *          type of source storage media
+     * @param targetStorageTypes
+     *          type of destin storage media
+     */
     public BlockMovingInfo(Block block,
         DatanodeInfo[] sourceDnInfos, DatanodeInfo[] targetDnInfos,
         StorageType[] srcStorageTypes, StorageType[] targetStorageTypes) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13a80503/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
index 8c4359f..f8b4934 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
@@ -77,6 +77,7 @@ public interface DatanodeProtocol {
   final static int DNA_CACHE = 9;      // cache blocks
   final static int DNA_UNCACHE = 10;   // uncache blocks
   final static int DNA_ERASURE_CODING_RECONSTRUCTION = 11; // erasure coding reconstruction command
+  final static int DNA_BLOCK_STORAGE_MOVEMENT = 12; // block storage movement command
 
   /** 
    * Register Datanode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13a80503/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
index 9bae4c3..fc64838 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -61,6 +61,7 @@ message DatanodeCommandProto {
     NullDatanodeCommand = 7;
     BlockIdCommand = 8;
     BlockECReconstructionCommand = 9;
+    BlockStorageMovementCommand = 10;
   }
 
   required Type cmdType = 1;    // Type of the command
@@ -75,6 +76,7 @@ message DatanodeCommandProto {
   optional RegisterCommandProto registerCmd = 7;
   optional BlockIdCommandProto blkIdCmd = 8;
   optional BlockECReconstructionCommandProto blkECReconstructionCmd = 9;
+  optional BlockStorageMovementCommandProto blkStorageMovementCmd = 10;
 }
 
 /**
@@ -155,6 +157,26 @@ message BlockECReconstructionCommandProto {
   repeated BlockECReconstructionInfoProto blockECReconstructioninfo = 1;
 }
 
+ /**
+ * Block storage movement command
+ */
+message BlockStorageMovementCommandProto {
+  required uint64 trackID = 1;
+  required string blockPoolId = 2;
+  repeated BlockStorageMovementProto blockStorageMovement = 3;
+}
+
+/**
+ * Block storage movement information
+ */
+message BlockStorageMovementProto {
+  required BlockProto block = 1;
+  required DatanodeInfosProto sourceDnInfos = 2;
+  required DatanodeInfosProto targetDnInfos = 3;
+  required StorageTypesProto sourceStorageTypes = 4;
+  required StorageTypesProto targetStorageTypes = 5;
+}
+
 /**
  * registration - Information of the datanode registering with the namenode
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13a80503/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfier.java
index b61814d..37664b5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStoragePolicySatisfier.java
@@ -18,9 +18,6 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -29,8 +26,7 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Before;
 import org.junit.Test;
@@ -74,9 +70,6 @@ public class TestStoragePolicySatisfier {
     try {
       // Change policy to ALL_SSD
       distributedFS.setStoragePolicy(new Path(file), "COLD");
-      Set<DatanodeDescriptor> previousNodes =
-          hdfsCluster.getNameNode().getNamesystem().getBlockManager()
-              .getDatanodeManager().getDatanodes();
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
       INode inode = namesystem.getFSDirectory().getINode(file);
 
@@ -91,8 +84,8 @@ public class TestStoragePolicySatisfier {
 
       hdfsCluster.triggerHeartbeats();
       // Wait till namenode notified about the block location details
-      waitExpectedStorageType(StorageType.ARCHIVE, distributedFS, previousNodes,
-          6, 30000);
+      waitExpectedStorageType(file, StorageType.ARCHIVE, distributedFS, 3,
+          30000);
     } finally {
       hdfsCluster.shutdown();
     }
@@ -104,9 +97,6 @@ public class TestStoragePolicySatisfier {
     try {
       // Change policy to ALL_SSD
       distributedFS.setStoragePolicy(new Path(file), "ALL_SSD");
-      Set<DatanodeDescriptor> previousNodes =
-          hdfsCluster.getNameNode().getNamesystem().getBlockManager()
-              .getDatanodeManager().getDatanodes();
       FSNamesystem namesystem = hdfsCluster.getNamesystem();
       INode inode = namesystem.getFSDirectory().getINode(file);
 
@@ -123,8 +113,34 @@ public class TestStoragePolicySatisfier {
       hdfsCluster.triggerHeartbeats();
       // Wait till StorgePolicySatisfier Identified that block to move to SSD
       // areas
-      waitExpectedStorageType(StorageType.SSD, distributedFS, previousNodes, 6,
-          30000);
+      waitExpectedStorageType(file, StorageType.SSD, distributedFS, 3, 30000);
+    } finally {
+      hdfsCluster.shutdown();
+    }
+  }
+
+  @Test(timeout = 300000)
+  public void testWhenStoragePolicySetToONESSD()
+      throws Exception {
+    try {
+      // Change policy to ONE_SSD
+      distributedFS.setStoragePolicy(new Path(file), "ONE_SSD");
+      FSNamesystem namesystem = hdfsCluster.getNamesystem();
+      INode inode = namesystem.getFSDirectory().getINode(file);
+
+      StorageType[][] newtypes =
+          new StorageType[][]{{StorageType.SSD, StorageType.DISK}};
+
+      // Making sure SDD based nodes added to cluster. Adding SSD based
+      // datanodes.
+      startAdditionalDNs(config, 1, numOfDatanodes, newtypes,
+          storagesPerDatanode, capacity, hdfsCluster);
+      namesystem.getBlockManager().satisfyStoragePolicy(inode.getId());
+      hdfsCluster.triggerHeartbeats();
+      // Wait till StorgePolicySatisfier Identified that block to move to SSD
+      // areas
+      waitExpectedStorageType(file, StorageType.SSD, distributedFS, 1, 30000);
+      waitExpectedStorageType(file, StorageType.DISK, distributedFS, 2, 30000);
     } finally {
       hdfsCluster.shutdown();
     }
@@ -174,35 +190,31 @@ public class TestStoragePolicySatisfier {
     return cluster;
   }
 
-  // TODO: this assertion can be changed to end to end based assertion later
-  // when DN side processing work integrated to this work.
-  private void waitExpectedStorageType(final StorageType expectedStorageType,
-      final DistributedFileSystem dfs,
-      final Set<DatanodeDescriptor> previousNodes, int expectedArchiveCount,
-      int timeout) throws Exception {
+  // Check whether the Block movement has been successfully completed to satisfy
+  // the storage policy for the given file.
+  private void waitExpectedStorageType(final String fileName,
+      final StorageType expectedStorageType, final DistributedFileSystem dfs,
+      int expectedStorageCount, int timeout) throws Exception {
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
-        Iterator<DatanodeDescriptor> iterator = previousNodes.iterator();
-        int archiveCount = 0;
-        while (iterator.hasNext()) {
-          DatanodeDescriptor dn = iterator.next();
-          List<BlockMovingInfo> pendingItemsToMove =
-              dn.getStorageMovementPendingItems();
-          for (BlockMovingInfo blkInfoToMoveStorage : pendingItemsToMove) {
-            StorageType[] targetStorageTypes =
-                blkInfoToMoveStorage.getTargetStorageTypes();
-            for (StorageType storageType : targetStorageTypes) {
-              if (storageType == expectedStorageType) {
-                archiveCount++;
-              }
-            }
+        LocatedBlock lb = null;
+        try {
+          lb = dfs.getClient().getLocatedBlocks(fileName, 0).get(0);
+        } catch (IOException e) {
+          LOG.error("Exception while getting located blocks", e);
+          return false;
+        }
+        int actualStorageCount = 0;
+        for (StorageType storageType : lb.getStorageTypes()) {
+          if (expectedStorageType == storageType) {
+            actualStorageCount++;
           }
         }
         LOG.info(
             expectedStorageType + " replica count, expected={} and actual={}",
-            expectedArchiveCount, archiveCount);
-        return expectedArchiveCount == archiveCount;
+            expectedStorageType, actualStorageCount);
+        return expectedStorageCount == actualStorageCount;
       }
     }, 100, timeout);
   }


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


[26/50] hadoop git commit: HADOOP-13723. AliyunOSSInputStream#read() should update read bytes stat correctly. Contributed by Mingliang Liu

Posted by um...@apache.org.
HADOOP-13723. AliyunOSSInputStream#read() should update read bytes stat correctly. 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/d9f73f1b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d9f73f1b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d9f73f1b

Branch: refs/heads/HDFS-10285
Commit: d9f73f1b7cd893a7d88baa9bfd1b809a5dec9e59
Parents: ae51b11
Author: Mingliang Liu <li...@apache.org>
Authored: Thu Oct 13 17:05:28 2016 -0700
Committer: Mingliang Liu <li...@apache.org>
Committed: Thu Oct 13 22:33:55 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d9f73f1b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
index b87a3a7..a3af7ce 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
@@ -123,7 +123,7 @@ public class AliyunOSSInputStream extends FSInputStream {
     }
 
     if (statistics != null && byteRead >= 0) {
-      statistics.incrementBytesRead(1);
+      statistics.incrementBytesRead(byteRead);
     }
     return byteRead;
   }


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


[38/50] hadoop git commit: HDFS-11012. Unnecessary INFO logging on DFSClients for InvalidToken. Contributed by Harsh J.

Posted by um...@apache.org.
HDFS-11012. Unnecessary INFO logging on DFSClients for InvalidToken. Contributed by Harsh J.

This closes #142


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

Branch: refs/heads/HDFS-10285
Commit: 5ad037df25ab3206509083276b7ef4ef001be48b
Parents: 391ce53
Author: Akira Ajisaka <aa...@apache.org>
Authored: Sat Oct 15 22:14:24 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Sat Oct 15 22:14:24 2016 +0900

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java    | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ad037df/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index dbffc64..5783f90 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -1261,8 +1261,9 @@ public class DFSInputStream extends FSInputStream
      */
     if (ex instanceof InvalidBlockTokenException ||
         ex instanceof InvalidToken) {
-      DFSClient.LOG.info("Access token was invalid when connecting to "
-          + targetAddr + " : " + ex);
+      DFSClient.LOG.debug(
+          "Access token was invalid when connecting to {}: {}",
+          targetAddr, ex);
       return true;
     }
     return false;


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


[29/50] hadoop git commit: HDFS-10990. TestPendingInvalidateBlock should wait for IBRs. Contributed by Yiqun Lin.

Posted by um...@apache.org.
HDFS-10990. TestPendingInvalidateBlock should wait for IBRs. Contributed by Yiqun Lin.


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

Branch: refs/heads/HDFS-10285
Commit: 8a9f6635a33e9648e9396e9ec5571fa34aa0c773
Parents: dbe663d
Author: Kihwal Lee <ki...@apache.org>
Authored: Fri Oct 14 11:38:48 2016 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Fri Oct 14 11:38:48 2016 -0500

----------------------------------------------------------------------
 .../blockmanagement/TestPendingInvalidateBlock.java    | 13 +++++++++----
 1 file changed, 9 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a9f6635/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
index 696b2aa..d856065 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
@@ -86,6 +86,8 @@ public class TestPendingInvalidateBlock {
   public void testPendingDeletion() throws Exception {
     final Path foo = new Path("/foo");
     DFSTestUtil.createFile(dfs, foo, BLOCKSIZE, REPLICATION, 0);
+    DFSTestUtil.waitForReplication(dfs, foo, REPLICATION, 10000);
+
     // restart NN
     cluster.restartNameNode(true);
     InvalidateBlocks invalidateBlocks =
@@ -98,6 +100,7 @@ public class TestPendingInvalidateBlock {
         "invalidateBlocks", mockIb);
     dfs.delete(foo, true);
 
+    waitForNumPendingDeletionBlocks(REPLICATION);
     Assert.assertEquals(0, cluster.getNamesystem().getBlocksTotal());
     Assert.assertEquals(REPLICATION, cluster.getNamesystem()
         .getPendingDeletionBlocks());
@@ -105,7 +108,7 @@ public class TestPendingInvalidateBlock {
         dfs.getPendingDeletionBlocksCount());
     Mockito.doReturn(0L).when(mockIb).getInvalidationDelay();
 
-    waitForBlocksToDelete();
+    waitForNumPendingDeletionBlocks(0);
     Assert.assertEquals(0, cluster.getNamesystem().getBlocksTotal());
     Assert.assertEquals(0, cluster.getNamesystem().getPendingDeletionBlocks());
     Assert.assertEquals(0, dfs.getPendingDeletionBlocksCount());
@@ -182,7 +185,7 @@ public class TestPendingInvalidateBlock {
     Assert.assertEquals(4, cluster.getNamesystem().getPendingDeletionBlocks());
 
     cluster.restartNameNode(true);
-    waitForBlocksToDelete();
+    waitForNumPendingDeletionBlocks(0);
     Assert.assertEquals(3, cluster.getNamesystem().getBlocksTotal());
     Assert.assertEquals(0, cluster.getNamesystem().getPendingDeletionBlocks());
   }
@@ -199,7 +202,8 @@ public class TestPendingInvalidateBlock {
     return cluster.getNamesystem().getUnderReplicatedBlocks();
   }
 
-  private void waitForBlocksToDelete() throws Exception {
+  private void waitForNumPendingDeletionBlocks(final int numBlocks)
+      throws Exception {
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
 
       @Override
@@ -207,7 +211,8 @@ public class TestPendingInvalidateBlock {
         try {
           cluster.triggerBlockReports();
 
-          if (cluster.getNamesystem().getPendingDeletionBlocks() == 0) {
+          if (cluster.getNamesystem().getPendingDeletionBlocks()
+              == numBlocks) {
             return true;
           }
         } catch (Exception e) {


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


[02/50] hadoop git commit: HADOOP-13697. LogLevel#main should not throw exception if no arguments. Contributed by Mingliang Liu

Posted by um...@apache.org.
HADOOP-13697. LogLevel#main should not throw exception if no arguments. 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/2fb392a5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2fb392a5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2fb392a5

Branch: refs/heads/HDFS-10285
Commit: 2fb392a587d288b628936ca6d18fabad04afc585
Parents: 809cfd2
Author: Mingliang Liu <li...@apache.org>
Authored: Fri Oct 7 14:05:40 2016 -0700
Committer: Mingliang Liu <li...@apache.org>
Committed: Tue Oct 11 10:57:08 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/log/LogLevel.java       | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2fb392a5/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java
index 4fa839f..79eae12 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java
@@ -47,15 +47,17 @@ import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.KerberosAuthenticator;
 import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.ServletUtil;
 import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
 
 /**
  * Change log level in runtime.
  */
 @InterfaceStability.Evolving
 public class LogLevel {
-  public static final String USAGES = "\nUsage: General options are:\n"
+  public static final String USAGES = "\nUsage: Command options are:\n"
       + "\t[-getlevel <host:port> <classname> [-protocol (http|https)]\n"
       + "\t[-setlevel <host:port> <classname> <level> "
       + "[-protocol (http|https)]\n";
@@ -67,7 +69,7 @@ public class LogLevel {
    */
   public static void main(String[] args) throws Exception {
     CLI cli = new CLI(new Configuration());
-    System.exit(cli.run(args));
+    System.exit(ToolRunner.run(cli, args));
   }
 
   /**
@@ -81,6 +83,7 @@ public class LogLevel {
 
   private static void printUsage() {
     System.err.println(USAGES);
+    GenericOptionsParser.printGenericCommandUsage(System.err);
   }
 
   public static boolean isValidProtocol(String protocol) {
@@ -107,7 +110,7 @@ public class LogLevel {
         sendLogLevelRequest();
       } catch (HadoopIllegalArgumentException e) {
         printUsage();
-        throw e;
+        return -1;
       }
       return 0;
     }


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


[15/50] hadoop git commit: HDFS-11000. webhdfs PUT does not work if requests are routed to call queue. Contributed by Kihwal Lee.

Posted by um...@apache.org.
HDFS-11000. webhdfs PUT does not work if requests are routed to call queue. Contributed by Kihwal Lee.


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

Branch: refs/heads/HDFS-10285
Commit: 9454dc5e8091354cd0a4b8c8aa5f4004529db5d5
Parents: 901eca0
Author: Kihwal Lee <ki...@apache.org>
Authored: Thu Oct 13 08:47:15 2016 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Thu Oct 13 08:47:15 2016 -0500

----------------------------------------------------------------------
 .../hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9454dc5e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
index 4887e35..4247a67 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
@@ -332,7 +332,7 @@ public class NamenodeWebHdfsMethods {
     } else {
       //generate a token
       final Token<? extends TokenIdentifier> t = generateDelegationToken(
-          namenode, ugi, userPrincipal.getName());
+          namenode, ugi, null);
       delegationQuery = "&" + new DelegationParam(t.encodeToUrlString());
     }
     final String query = op.toQueryString() + delegationQuery


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


[39/50] hadoop git commit: HDFS-11003. Expose XmitsInProgress through DataNodeMXBean. Contributed By Brahma Reddy Battula

Posted by um...@apache.org.
HDFS-11003. Expose XmitsInProgress through DataNodeMXBean. Contributed By Brahma Reddy Battula


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

Branch: refs/heads/HDFS-10285
Commit: 5f4ae85bd8a20510948696467873498723b06477
Parents: 5ad037d
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Sat Oct 15 22:28:33 2016 +0530
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Sat Oct 15 22:28:33 2016 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/server/datanode/DataNode.java  | 5 +++--
 .../org/apache/hadoop/hdfs/server/datanode/DataNodeMXBean.java | 6 ++++++
 .../apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java | 6 +++++-
 3 files changed, 14 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f4ae85b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index cb8e308..8f65efe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -2101,8 +2101,9 @@ public class DataNode extends ReconfigurableBase
       }
     }
   }
-  
-  int getXmitsInProgress() {
+
+  @Override //DataNodeMXBean
+  public int getXmitsInProgress() {
     return xmitsInProgress.get();
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f4ae85b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeMXBean.java
index 5ec4cda..5d4c218 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeMXBean.java
@@ -101,6 +101,12 @@ public interface DataNodeMXBean {
   public int getXceiverCount();
 
   /**
+   * Returns an estimate of the number of data replication/reconstruction tasks
+   * running currently.
+   */
+  public int getXmitsInProgress();
+
+  /**
    * Gets the network error counts on a per-Datanode basis.
    */
   public Map<String, Map<String, Long>> getDatanodeNetworkCounts();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f4ae85b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java
index 8b0d5cb..a77c943 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java
@@ -96,7 +96,11 @@ public class TestDataNodeMXBean {
       int xceiverCount = (Integer)mbs.getAttribute(mxbeanName,
           "XceiverCount");
       Assert.assertEquals(datanode.getXceiverCount(), xceiverCount);
-
+      // Ensure mxbean's XmitsInProgress is same as the DataNode's
+      // live value.
+      int xmitsInProgress =
+          (Integer) mbs.getAttribute(mxbeanName, "XmitsInProgress");
+      Assert.assertEquals(datanode.getXmitsInProgress(), xmitsInProgress);
       String bpActorInfo = (String)mbs.getAttribute(mxbeanName,
           "BPServiceActorInfo");
       Assert.assertEquals(datanode.getBPServiceActorInfo(), bpActorInfo);


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


[47/50] hadoop git commit: YARN-5466. DefaultContainerExecutor needs JavaDocs (templedf via rkanter)

Posted by um...@apache.org.
YARN-5466. DefaultContainerExecutor needs JavaDocs (templedf via rkanter)


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

Branch: refs/heads/HDFS-10285
Commit: f5d92359145dfb820a9521e00e2d44c4ee96e67e
Parents: 8fd4c37
Author: Robert Kanter <rk...@apache.org>
Authored: Mon Oct 17 14:29:09 2016 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Mon Oct 17 14:29:09 2016 -0700

----------------------------------------------------------------------
 .../nodemanager/DefaultContainerExecutor.java   | 272 ++++++++++++++++---
 .../WindowsSecureContainerExecutor.java         |   2 +-
 2 files changed, 231 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5d92359/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.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/DefaultContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java
index 59b69ac..568c80b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java
@@ -65,6 +65,11 @@ import org.apache.hadoop.yarn.server.nodemanager.executor.LocalizerStartContext;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 
+/**
+ * The {@code DefaultContainerExecuter} class offers generic container
+ * execution services. Process execution is handled in a platform-independent
+ * way via {@link ProcessBuilder}.
+ */
 public class DefaultContainerExecutor extends ContainerExecutor {
 
   private static final Log LOG = LogFactory
@@ -72,10 +77,17 @@ public class DefaultContainerExecutor extends ContainerExecutor {
 
   private static final int WIN_MAX_PATH = 260;
 
+  /**
+   * A {@link FileContext} for the local file system.
+   */
   protected final FileContext lfs;
 
   private String logDirPermissions = null;
 
+  /**
+   * Default constructor for use in testing.
+   */
+  @VisibleForTesting
   public DefaultContainerExecutor() {
     try {
       this.lfs = FileContext.getLocalFSFileContext();
@@ -84,15 +96,40 @@ public class DefaultContainerExecutor extends ContainerExecutor {
     }
   }
 
+  /**
+   * Create an instance with a given {@link FileContext}.
+   *
+   * @param lfs the given {@link FileContext}
+   */
   DefaultContainerExecutor(FileContext lfs) {
     this.lfs = lfs;
   }
 
+  /**
+   * Copy a file using the {@link #lfs} {@link FileContext}.
+   *
+   * @param src the file to copy
+   * @param dst where to copy the file
+   * @param owner the owner of the new copy. Used only in secure Windows
+   * clusters
+   * @throws IOException when the copy fails
+   * @see WindowsSecureContainerExecutor
+   */
   protected void copyFile(Path src, Path dst, String owner) throws IOException {
     lfs.util().copy(src, dst, false, true);
   }
   
-  protected void setScriptExecutable(Path script, String owner) throws IOException {
+  /**
+   * Make a file executable using the {@link #lfs} {@link FileContext}.
+   *
+   * @param script the path to make executable
+   * @param owner the new owner for the file. Used only in secure Windows
+   * clusters
+   * @throws IOException when the change mode operation fails
+   * @see WindowsSecureContainerExecutor
+   */
+  protected void setScriptExecutable(Path script, String owner)
+      throws IOException {
     lfs.setPermission(script, ContainerExecutor.TASK_LAUNCH_SCRIPT_PERMISSION);
   }
 
@@ -122,14 +159,16 @@ public class DefaultContainerExecutor extends ContainerExecutor {
     // randomly choose the local directory
     Path appStorageDir = getWorkingDir(localDirs, user, appId);
 
-    String tokenFn = String.format(ContainerLocalizer.TOKEN_FILE_NAME_FMT, locId);
+    String tokenFn =
+        String.format(ContainerLocalizer.TOKEN_FILE_NAME_FMT, locId);
     Path tokenDst = new Path(appStorageDir, tokenFn);
     copyFile(nmPrivateContainerTokensPath, tokenDst, user);
-    LOG.info("Copying from " + nmPrivateContainerTokensPath + " to " + tokenDst);
+    LOG.info("Copying from " + nmPrivateContainerTokensPath
+        + " to " + tokenDst);
 
 
-    FileContext localizerFc = FileContext.getFileContext(
-        lfs.getDefaultFileSystem(), getConf());
+    FileContext localizerFc =
+        FileContext.getFileContext(lfs.getDefaultFileSystem(), getConf());
     localizerFc.setUMask(lfs.getUMask());
     localizerFc.setWorkingDirectory(appStorageDir);
     LOG.info("Localizer CWD set to " + appStorageDir + " = " 
@@ -141,6 +180,22 @@ public class DefaultContainerExecutor extends ContainerExecutor {
     localizer.runLocalization(nmAddr);
   }
 
+  /**
+   * Create a new {@link ContainerLocalizer} instance.
+   *
+   * @param user the user who owns the job for which the localization is being
+   * run
+   * @param appId the ID of the application for which the localization is being
+   * run
+   * @param locId the ID of the container for which the localization is being
+   * run
+   * @param localDirs a list of directories to use as destinations for the
+   * localization
+   * @param localizerFc the {@link FileContext} to use when localizing files
+   * @return the new {@link ContainerLocalizer} instance
+   * @throws IOException if {@code user} or {@code locId} is {@code null} or if
+   * the container localizer has an initialization failure
+   */
   @Private
   @VisibleForTesting
   protected ContainerLocalizer createContainerLocalizer(String user,
@@ -258,15 +313,17 @@ public class DefaultContainerExecutor extends ContainerExecutor {
 
         StringBuilder builder = new StringBuilder();
         builder.append("Exception from container-launch.\n");
-        builder.append("Container id: " + containerId + "\n");
-        builder.append("Exit code: " + exitCode + "\n");
+        builder.append("Container id: ").append(containerId).append("\n");
+        builder.append("Exit code: ").append(exitCode).append("\n");
         if (!Optional.fromNullable(e.getMessage()).or("").isEmpty()) {
-          builder.append("Exception message: " + e.getMessage() + "\n");
+          builder.append("Exception message: ");
+          builder.append(e.getMessage()).append("\n");
         }
-        builder.append("Stack trace: "
-            + StringUtils.stringifyException(e) + "\n");
+        builder.append("Stack trace: ");
+        builder.append(StringUtils.stringifyException(e)).append("\n");
         if (!shExec.getOutput().isEmpty()) {
-          builder.append("Shell output: " + shExec.getOutput() + "\n");
+          builder.append("Shell output: ");
+          builder.append(shExec.getOutput()).append("\n");
         }
         String diagnostics = builder.toString();
         logOutput(diagnostics);
@@ -283,10 +340,24 @@ public class DefaultContainerExecutor extends ContainerExecutor {
     return 0;
   }
 
+  /**
+   * Create a new {@link ShellCommandExecutor} using the parameters.
+   *
+   * @param wrapperScriptPath the path to the script to execute
+   * @param containerIdStr the container ID
+   * @param user the application owner's username
+   * @param pidFile the path to the container's PID file
+   * @param resource this parameter controls memory and CPU limits.
+   * @param workDir If not-null, specifies the directory which should be set
+   * as the current working directory for the command. If null,
+   * the current working directory is not modified.
+   * @param environment the container environment
+   * @return the new {@link ShellCommandExecutor}
+   * @see ShellCommandExecutor
+   */
   protected CommandExecutor buildCommandExecutor(String wrapperScriptPath, 
       String containerIdStr, String user, Path pidFile, Resource resource,
-      File wordDir, Map<String, String> environment)
-          throws IOException {
+      File workDir, Map<String, String> environment) {
     
     String[] command = getRunCommand(wrapperScriptPath,
         containerIdStr, user, pidFile, this.getConf(), resource);
@@ -294,12 +365,20 @@ public class DefaultContainerExecutor extends ContainerExecutor {
       LOG.info("launchContainer: " + Arrays.toString(command));
       return new ShellCommandExecutor(
           command,
-          wordDir,
+          workDir,
           environment,
           0L,
           false);
   }
 
+  /**
+   * Create a {@link LocalWrapperScriptBuilder} for the given container ID
+   * and path that is appropriate to the current platform.
+   *
+   * @param containerIdStr the container ID
+   * @param containerWorkDir the container's working directory
+   * @return a new {@link LocalWrapperScriptBuilder}
+   */
   protected LocalWrapperScriptBuilder getLocalWrapperScriptBuilder(
       String containerIdStr, Path containerWorkDir) {
    return  Shell.WINDOWS ?
@@ -307,15 +386,34 @@ public class DefaultContainerExecutor extends ContainerExecutor {
        new UnixLocalWrapperScriptBuilder(containerWorkDir);
   }
 
+  /**
+   * This class is a utility to create a wrapper script that is platform
+   * appropriate.
+   */
   protected abstract class LocalWrapperScriptBuilder {
 
     private final Path wrapperScriptPath;
 
+    /**
+     * Return the path for the wrapper script.
+     *
+     * @return the path for the wrapper script
+     */
     public Path getWrapperScriptPath() {
       return wrapperScriptPath;
     }
 
-    public void writeLocalWrapperScript(Path launchDst, Path pidFile) throws IOException {
+    /**
+     * Write out the wrapper script for the container launch script. This method
+     * will create the script at the configured wrapper script path.
+     *
+     * @param launchDst the script to launch
+     * @param pidFile the file that will hold the PID
+     * @throws IOException if the wrapper script cannot be created
+     * @see #getWrapperScriptPath
+     */
+    public void writeLocalWrapperScript(Path launchDst, Path pidFile)
+        throws IOException {
       DataOutputStream out = null;
       PrintStream pout = null;
 
@@ -328,19 +426,40 @@ public class DefaultContainerExecutor extends ContainerExecutor {
       }
     }
 
-    protected abstract void writeLocalWrapperScript(Path launchDst, Path pidFile,
-        PrintStream pout);
-
+    /**
+     * Write out the wrapper script for the container launch script.
+     *
+     * @param launchDst the script to launch
+     * @param pidFile the file that will hold the PID
+     * @param pout the stream to use to write out the wrapper script
+     */
+    protected abstract void writeLocalWrapperScript(Path launchDst,
+        Path pidFile, PrintStream pout);
+
+    /**
+     * Create an instance for the given container working directory.
+     *
+     * @param containerWorkDir the working directory for the container
+     */
     protected LocalWrapperScriptBuilder(Path containerWorkDir) {
       this.wrapperScriptPath = new Path(containerWorkDir,
         Shell.appendScriptExtension("default_container_executor"));
     }
   }
 
+  /**
+   * This class is an instance of {@link LocalWrapperScriptBuilder} for
+   * non-Windows hosts.
+   */
   private final class UnixLocalWrapperScriptBuilder
       extends LocalWrapperScriptBuilder {
     private final Path sessionScriptPath;
 
+    /**
+     * Create an instance for the given container path.
+     *
+     * @param containerWorkDir the container's working directory
+     */
     public UnixLocalWrapperScriptBuilder(Path containerWorkDir) {
       super(containerWorkDir);
       this.sessionScriptPath = new Path(containerWorkDir,
@@ -383,8 +502,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
         pout.println("echo $$ > " + pidFile.toString() + ".tmp");
         pout.println("/bin/mv -f " + pidFile.toString() + ".tmp " + pidFile);
         String exec = Shell.isSetsidAvailable? "exec setsid" : "exec";
-        pout.println(exec + " /bin/bash \"" +
-            launchDst.toUri().getPath().toString() + "\"");
+        pout.printf("%s /bin/bash \"%s\"", exec, launchDst.toUri().getPath());
       } finally {
         IOUtils.cleanup(LOG, pout, out);
       }
@@ -393,11 +511,21 @@ public class DefaultContainerExecutor extends ContainerExecutor {
     }
   }
 
+  /**
+   * This class is an instance of {@link LocalWrapperScriptBuilder} for
+   * Windows hosts.
+   */
   private final class WindowsLocalWrapperScriptBuilder
       extends LocalWrapperScriptBuilder {
 
     private final String containerIdStr;
 
+    /**
+     * Create an instance for the given container and working directory.
+     *
+     * @param containerIdStr the container ID
+     * @param containerWorkDir the container's working directory
+     */
     public WindowsLocalWrapperScriptBuilder(String containerIdStr,
         Path containerWorkDir) {
 
@@ -458,6 +586,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
    * 
    * @param pid String pid
    * @return boolean true if the process is alive
+   * @throws IOException if the command to test process liveliness fails
    */
   @VisibleForTesting
   public static boolean containerIsAlive(String pid) throws IOException {
@@ -478,7 +607,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
    *
    * @param pid the pid of the process [group] to signal.
    * @param signal signal to send
-   * (for logging).
+   * @throws IOException if the command to kill the process fails
    */
   protected void killContainer(String pid, Signal signal) throws IOException {
     new ShellCommandExecutor(Shell.getSignalKillCommand(signal.getValue(), pid))
@@ -517,17 +646,25 @@ public class DefaultContainerExecutor extends ContainerExecutor {
     FileUtil.symLink(target, symlink);
   }
 
-  /** Permissions for user dir.
-   * $local.dir/usercache/$user */
+  /**
+   * Permissions for user dir.
+   * $local.dir/usercache/$user
+   */
   static final short USER_PERM = (short)0750;
-  /** Permissions for user appcache dir.
-   * $local.dir/usercache/$user/appcache */
+  /**
+   * Permissions for user appcache dir.
+   * $local.dir/usercache/$user/appcache
+   */
   static final short APPCACHE_PERM = (short)0710;
-  /** Permissions for user filecache dir.
-   * $local.dir/usercache/$user/filecache */
+  /**
+   * Permissions for user filecache dir.
+   * $local.dir/usercache/$user/filecache
+   */
   static final short FILECACHE_PERM = (short)0710;
-  /** Permissions for user app dir.
-   * $local.dir/usercache/$user/appcache/$appId */
+  /**
+   * Permissions for user app dir.
+   * $local.dir/usercache/$user/appcache/$appId
+   */
   static final short APPDIR_PERM = (short)0710;
 
   private long getDiskFreeSpace(Path base) throws IOException {
@@ -552,9 +689,20 @@ public class DefaultContainerExecutor extends ContainerExecutor {
         ContainerLocalizer.FILECACHE);
   }
 
+  /**
+   * Return a randomly chosen application directory from a list of local storage
+   * directories. The probability of selecting a directory is proportional to
+   * its size.
+   *
+   * @param localDirs the target directories from which to select
+   * @param user the user who owns the application
+   * @param appId the application ID
+   * @return the selected directory
+   * @throws IOException if no application directories for the user can be
+   * found
+   */
   protected Path getWorkingDir(List<String> localDirs, String user,
       String appId) throws IOException {
-    Path appStorageDir = null;
     long totalAvailable = 0L;
     long[] availableOnDisk = new long[localDirs.size()];
     int i = 0;
@@ -563,8 +711,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
     // the available space on the directory.
     // firstly calculate the sum of all available space on these directories
     for (String localDir : localDirs) {
-      Path curBase = getApplicationDir(new Path(localDir),
-          user, appId);
+      Path curBase = getApplicationDir(new Path(localDir), user, appId);
       long space = 0L;
       try {
         space = getDiskFreeSpace(curBase);
@@ -577,8 +724,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
 
     // throw an IOException if totalAvailable is 0.
     if (totalAvailable <= 0L) {
-      throw new IOException("Not able to find a working directory for "
-          + user);
+      throw new IOException("Not able to find a working directory for " + user);
     }
 
     // make probability to pick a directory proportional to
@@ -595,12 +741,21 @@ public class DefaultContainerExecutor extends ContainerExecutor {
     while (randomPosition > availableOnDisk[dir]) {
       randomPosition -= availableOnDisk[dir++];
     }
-    appStorageDir = getApplicationDir(new Path(localDirs.get(dir)),
-        user, appId);
 
-    return appStorageDir;
+    return getApplicationDir(new Path(localDirs.get(dir)), user, appId);
   }
 
+  /**
+   * Use the {@link #lfs} {@link FileContext} to create the target directory.
+   *
+   * @param dirPath the target directory
+   * @param perms the target permissions for the target directory
+   * @param createParent whether the parent directories should also be created
+   * @param user the user as whom the target directory should be created.
+   * Used only on secure Windows hosts.
+   * @throws IOException if there's a failure performing a file operation
+   * @see WindowsSecureContainerExecutor
+   */
   protected void createDir(Path dirPath, FsPermission perms,
       boolean createParent, String user) throws IOException {
     lfs.mkdir(dirPath, perms, createParent);
@@ -614,6 +769,11 @@ public class DefaultContainerExecutor extends ContainerExecutor {
    * <ul>.mkdir
    * <li>$local.dir/usercache/$user</li>
    * </ul>
+   *
+   * @param localDirs the target directories to create
+   * @param user the user whose local cache directories should be initialized
+   * @throws IOException if there's an issue initializing the user local
+   * directories
    */
   void createUserLocalDirs(List<String> localDirs, String user)
       throws IOException {
@@ -622,7 +782,8 @@ public class DefaultContainerExecutor extends ContainerExecutor {
     for (String localDir : localDirs) {
       // create $local.dir/usercache/$user and its immediate parent
       try {
-        createDir(getUserCacheDir(new Path(localDir), user), userperms, true, user);
+        createDir(getUserCacheDir(new Path(localDir), user), userperms, true,
+            user);
       } catch (IOException e) {
         LOG.warn("Unable to create the user directory : " + localDir, e);
         continue;
@@ -643,6 +804,11 @@ public class DefaultContainerExecutor extends ContainerExecutor {
    * <li>$local.dir/usercache/$user/appcache</li>
    * <li>$local.dir/usercache/$user/filecache</li>
    * </ul>
+   *
+   * @param localDirs the target directories to create
+   * @param user the user whose local cache directories should be initialized
+   * @throws IOException if there's an issue initializing the cache
+   * directories
    */
   void createUserCacheDirs(List<String> localDirs, String user)
       throws IOException {
@@ -689,7 +855,12 @@ public class DefaultContainerExecutor extends ContainerExecutor {
    * <ul>
    * <li>$local.dir/usercache/$user/appcache/$appid</li>
    * </ul>
-   * @param localDirs 
+   *
+   * @param localDirs the target directories to create
+   * @param user the user whose local cache directories should be initialized
+   * @param appId the application ID
+   * @throws IOException if there's an issue initializing the application
+   * directories
    */
   void createAppDirs(List<String> localDirs, String user, String appId)
       throws IOException {
@@ -714,6 +885,12 @@ public class DefaultContainerExecutor extends ContainerExecutor {
 
   /**
    * Create application log directories on all disks.
+   *
+   * @param appId the application ID
+   * @param logDirs the target directories to create
+   * @param user the user whose local cache directories should be initialized
+   * @throws IOException if there's an issue initializing the application log
+   * directories
    */
   void createAppLogDirs(String appId, List<String> logDirs, String user)
       throws IOException {
@@ -740,10 +917,17 @@ public class DefaultContainerExecutor extends ContainerExecutor {
 
   /**
    * Create application log directories on all disks.
+   *
+   * @param appId the application ID
+   * @param containerId the container ID
+   * @param logDirs the target directories to create
+   * @param user the user as whom the directories should be created.
+   * Used only on secure Windows hosts.
+   * @throws IOException if there's an issue initializing the container log
+   * directories
    */
   void createContainerLogDirs(String appId, String containerId,
       List<String> logDirs, String user) throws IOException {
-
     boolean containerLogDirStatus = false;
     FsPermission containerLogDirPerms = new
         FsPermission(getLogDirPermissions());
@@ -769,7 +953,9 @@ public class DefaultContainerExecutor extends ContainerExecutor {
   }
 
   /**
-   * Return default container log directory permissions.
+   * Return the default container log directory permissions.
+   *
+   * @return the default container log directory permissions
    */
   @VisibleForTesting
   public String getLogDirPermissions() {
@@ -790,10 +976,12 @@ public class DefaultContainerExecutor extends ContainerExecutor {
   }
 
   /**
+   * Return the list of paths of given local directories.
+   *
    * @return the list of paths of given local directories
    */
   private static List<Path> getPaths(List<String> dirs) {
-    List<Path> paths = new ArrayList<Path>(dirs.size());
+    List<Path> paths = new ArrayList<>(dirs.size());
     for (int i = 0; i < dirs.size(); i++) {
       paths.add(new Path(dirs.get(i)));
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5d92359/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.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/WindowsSecureContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java
index c75ecb1..9e38151 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java
@@ -732,7 +732,7 @@ public class WindowsSecureContainerExecutor extends DefaultContainerExecutor {
   @Override
   protected CommandExecutor buildCommandExecutor(String wrapperScriptPath,
       String containerIdStr, String userName, Path pidFile, Resource resource,
-      File wordDir, Map<String, String> environment) throws IOException {
+      File wordDir, Map<String, String> environment) {
      return new WintuilsProcessStubExecutor(
          wordDir.toString(),
          containerIdStr, userName, pidFile.toString(), 


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


[20/50] hadoop git commit: HDFS-10990. TestPendingInvalidateBlock should wait for IBRs. Contributed by Yiqun Lin.

Posted by um...@apache.org.
HDFS-10990. TestPendingInvalidateBlock should wait for IBRs. Contributed by Yiqun Lin.


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

Branch: refs/heads/HDFS-10285
Commit: fdce515091f0a61ffd6c9ae464a68447dedf1124
Parents: 008122b
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Oct 13 11:41:37 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Oct 13 11:41:37 2016 -0700

----------------------------------------------------------------------
 .../blockmanagement/TestPendingInvalidateBlock.java    | 13 +++++++++----
 1 file changed, 9 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdce5150/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
index 696b2aa..19f3178 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
@@ -86,6 +86,8 @@ public class TestPendingInvalidateBlock {
   public void testPendingDeletion() throws Exception {
     final Path foo = new Path("/foo");
     DFSTestUtil.createFile(dfs, foo, BLOCKSIZE, REPLICATION, 0);
+    DFSTestUtil.waitForReplication(dfs, foo, REPLICATION, 10000);
+
     // restart NN
     cluster.restartNameNode(true);
     InvalidateBlocks invalidateBlocks =
@@ -98,6 +100,7 @@ public class TestPendingInvalidateBlock {
         "invalidateBlocks", mockIb);
     dfs.delete(foo, true);
 
+    waitForNumPendingDeletionBlocks(REPLICATION);
     Assert.assertEquals(0, cluster.getNamesystem().getBlocksTotal());
     Assert.assertEquals(REPLICATION, cluster.getNamesystem()
         .getPendingDeletionBlocks());
@@ -105,7 +108,7 @@ public class TestPendingInvalidateBlock {
         dfs.getPendingDeletionBlocksCount());
     Mockito.doReturn(0L).when(mockIb).getInvalidationDelay();
 
-    waitForBlocksToDelete();
+    waitForNumPendingDeletionBlocks(0);
     Assert.assertEquals(0, cluster.getNamesystem().getBlocksTotal());
     Assert.assertEquals(0, cluster.getNamesystem().getPendingDeletionBlocks());
     Assert.assertEquals(0, dfs.getPendingDeletionBlocksCount());
@@ -182,7 +185,7 @@ public class TestPendingInvalidateBlock {
     Assert.assertEquals(4, cluster.getNamesystem().getPendingDeletionBlocks());
 
     cluster.restartNameNode(true);
-    waitForBlocksToDelete();
+    waitForNumPendingDeletionBlocks(0);
     Assert.assertEquals(3, cluster.getNamesystem().getBlocksTotal());
     Assert.assertEquals(0, cluster.getNamesystem().getPendingDeletionBlocks());
   }
@@ -199,7 +202,8 @@ public class TestPendingInvalidateBlock {
     return cluster.getNamesystem().getUnderReplicatedBlocks();
   }
 
-  private void waitForBlocksToDelete() throws Exception {
+  private void waitForNumPendingDeletionBlocks(int numBlocks)
+      throws Exception {
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
 
       @Override
@@ -207,7 +211,8 @@ public class TestPendingInvalidateBlock {
         try {
           cluster.triggerBlockReports();
 
-          if (cluster.getNamesystem().getPendingDeletionBlocks() == 0) {
+          if (cluster.getNamesystem().getPendingDeletionBlocks()
+              == numBlocks) {
             return true;
           }
         } catch (Exception e) {


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