You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ar...@apache.org on 2015/06/20 23:25:06 UTC

[01/50] [abbrv] hadoop git commit: HDFS-8576. Lease recovery should return true if the lease can be released and the file can be closed. Contributed by J.Andreina

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 197b8fb0f -> f827d3b25


HDFS-8576.  Lease recovery should return true if the lease can be released and the file can be closed.  Contributed by J.Andreina


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

Branch: refs/heads/HDFS-7240
Commit: 2cb09e98e392feb5732d0754b539240094edc37a
Parents: 85cc644
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jun 15 16:07:38 2015 -0700
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Mon Jun 15 16:07:38 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../hdfs/server/namenode/FSNamesystem.java      | 22 ++++++----
 .../apache/hadoop/hdfs/TestLeaseRecovery.java   | 46 ++++++++++++++++++++
 3 files changed, 62 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2cb09e98/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index c98d918..21acf98 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1011,6 +1011,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8595. TestCommitBlockSynchronization fails in branch-2.7. (Patch
     applies to all branches). (Arpit Agarwal)
 
+    HDFS-8576.  Lease recovery should return true if the lease can be released
+    and the file can be closed.  (J.Andreina via szetszwo)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2cb09e98/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 f962373..518adb4 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
@@ -2616,7 +2616,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @param src the path of the file to start lease recovery
    * @param holder the lease holder's name
    * @param clientMachine the client machine's name
-   * @return true if the file is already closed
+   * @return true if the file is already closed or
+   *         if the lease can be released and the file can be closed.
    * @throws IOException
    */
   boolean recoverLease(String src, String holder, String clientMachine)
@@ -2643,7 +2644,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         dir.checkPathAccess(pc, iip, FsAction.WRITE);
       }
   
-      recoverLeaseInternal(RecoverLeaseOp.RECOVER_LEASE,
+      return recoverLeaseInternal(RecoverLeaseOp.RECOVER_LEASE,
           iip, src, holder, clientMachine, true);
     } catch (StandbyException se) {
       skipSync = true;
@@ -2656,7 +2657,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         getEditLog().logSync();
       }
     }
-    return false;
   }
 
   enum RecoverLeaseOp {
@@ -2672,12 +2672,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
   }
 
-  void recoverLeaseInternal(RecoverLeaseOp op, INodesInPath iip,
+  boolean recoverLeaseInternal(RecoverLeaseOp op, INodesInPath iip,
       String src, String holder, String clientMachine, boolean force)
       throws IOException {
     assert hasWriteLock();
     INodeFile file = iip.getLastINode().asFile();
-    if (file != null && file.isUnderConstruction()) {
+    if (file.isUnderConstruction()) {
       //
       // If the file is under construction , then it must be in our
       // leases. Find the appropriate lease record.
@@ -2710,7 +2710,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         // close only the file src
         LOG.info("recoverLease: " + lease + ", src=" + src +
           " from client " + clientName);
-        internalReleaseLease(lease, src, iip, holder);
+        return internalReleaseLease(lease, src, iip, holder);
       } else {
         assert lease.getHolder().equals(clientName) :
           "Current lease holder " + lease.getHolder() +
@@ -2722,11 +2722,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         if (lease.expiredSoftLimit()) {
           LOG.info("startFile: recover " + lease + ", src=" + src + " client "
               + clientName);
-          boolean isClosed = internalReleaseLease(lease, src, iip, null);
-          if(!isClosed)
+          if (internalReleaseLease(lease, src, iip, null)) {
+            return true;
+          } else {
             throw new RecoveryInProgressException(
                 op.getExceptionMessage(src, holder, clientMachine,
                     "lease recovery is in progress. Try again later."));
+          }
         } else {
           final BlockInfo lastBlock = file.getLastBlock();
           if (lastBlock != null
@@ -2743,7 +2745,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           }
         }
       }
-    }
+    } else {
+      return true;
+     }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2cb09e98/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java
index 15580a5..c9f3842 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.IOException;
@@ -43,6 +44,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.TestInterDatanodePr
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.After;
 import org.junit.Test;
@@ -212,4 +214,48 @@ public class TestLeaseRecovery {
     assertTrue("File should be closed", newdfs.recoverLease(file));
 
   }
+
+  /**
+   * Recover the lease on a file and append file from another client.
+   */
+  @Test
+  public void testLeaseRecoveryAndAppend() throws Exception {
+    Configuration conf = new Configuration();
+    try{
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    Path file = new Path("/testLeaseRecovery");
+    DistributedFileSystem dfs = cluster.getFileSystem();
+
+    // create a file with 0 bytes
+    FSDataOutputStream out = dfs.create(file);
+    out.hflush();
+    out.hsync();
+
+    // abort the original stream
+    ((DFSOutputStream) out.getWrappedStream()).abort();
+    DistributedFileSystem newdfs =
+        (DistributedFileSystem) FileSystem.newInstance
+        (cluster.getConfiguration(0));
+
+    // Append to a file , whose lease is held by another client should fail
+    try {
+        newdfs.append(file);
+        fail("Append to a file(lease is held by another client) should fail");
+    } catch (RemoteException e) {
+      assertTrue(e.getMessage().contains("file lease is currently owned"));
+    }
+
+    // Lease recovery on first try should be successful
+    boolean recoverLease = newdfs.recoverLease(file);
+    assertTrue(recoverLease);
+    FSDataOutputStream append = newdfs.append(file);
+    append.write("test".getBytes());
+    append.close();
+    }finally{
+      if (cluster != null) {
+        cluster.shutdown();
+        cluster = null;
+      }
+    }
+  }
 }


[10/50] [abbrv] hadoop git commit: HADOOP-12095. org.apache.hadoop.fs.shell.TestCount fails. Contributed by Brahma Reddy Battula.

Posted by ar...@apache.org.
HADOOP-12095. org.apache.hadoop.fs.shell.TestCount fails. Contributed by Brahma Reddy Battula.


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

Branch: refs/heads/HDFS-7240
Commit: b578807b9903f3f2056c7064bfe5af8619eb4835
Parents: 4e88ff5
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Jun 16 13:38:03 2015 -0700
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Jun 16 13:38:03 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b578807b/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 11c2f2a..9a16a7d 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -858,6 +858,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12001. Fixed LdapGroupsMapping to include configurable Posix UID and
     GID attributes during the search. (Patrick White via vinodkv)
 
+    HADOOP-12095. org.apache.hadoop.fs.shell.TestCount fails.
+    (Brahma Reddy Battula via aajisaka)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b578807b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
index 44fc1e6..116c071 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
@@ -281,7 +281,7 @@ public class TestCount {
     count.processOptions(options);
     String withStorageTypeHeader =
         // <----13---> <-------17------> <----13-----> <------17------->
-        "   DISK_QUOTA    REM_DISK_QUOTA     SSD_QUOTA     REM_SSD_QUOTA " +
+        "    SSD_QUOTA     REM_SSD_QUOTA    DISK_QUOTA    REM_DISK_QUOTA " +
         // <----13---> <-------17------>
         "ARCHIVE_QUOTA REM_ARCHIVE_QUOTA " +
         "PATHNAME";
@@ -335,8 +335,8 @@ public class TestCount {
     count.processOptions(options);
     String withStorageTypeHeader =
         // <----13---> <-------17------>
-        "   DISK_QUOTA    REM_DISK_QUOTA " +
         "    SSD_QUOTA     REM_SSD_QUOTA " +
+        "   DISK_QUOTA    REM_DISK_QUOTA " +
         "ARCHIVE_QUOTA REM_ARCHIVE_QUOTA " +
         "PATHNAME";
     verify(out).println(withStorageTypeHeader);


[18/50] [abbrv] hadoop git commit: YARN-3617. Fix WindowsResourceCalculatorPlugin.getCpuFrequency() returning always -1. Contributed by J.Andreina.

Posted by ar...@apache.org.
YARN-3617. Fix WindowsResourceCalculatorPlugin.getCpuFrequency() returning
always -1. Contributed by J.Andreina.


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

Branch: refs/heads/HDFS-7240
Commit: 318d2cde7cb5c05a5f87c4ee967446bb60d28ae4
Parents: 5dbc8c9
Author: Devaraj K <de...@apache.org>
Authored: Wed Jun 17 13:54:09 2015 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Wed Jun 17 13:54:09 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 3 +++
 .../apache/hadoop/yarn/util/WindowsResourceCalculatorPlugin.java  | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/318d2cde/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index b61eb27..c793a31 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -530,6 +530,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3714. AM proxy filter can not get RM webapp address from
     yarn.resourcemanager.hostname.rm-id. (Masatake Iwasaki via xgong)
 
+    YARN-3617. Fix WindowsResourceCalculatorPlugin.getCpuFrequency()
+    returning always -1. (J.Andreina via devaraj)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/318d2cde/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsResourceCalculatorPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsResourceCalculatorPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsResourceCalculatorPlugin.java
index 0e89118..cdbf525 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsResourceCalculatorPlugin.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsResourceCalculatorPlugin.java
@@ -157,7 +157,7 @@ public class WindowsResourceCalculatorPlugin extends ResourceCalculatorPlugin {
   @Override
   public long getCpuFrequency() {
     refreshIfNeeded();
-    return -1;
+    return cpuFrequencyKhz;
   }
 
   /** {@inheritDoc} */


[50/50] [abbrv] hadoop git commit: Fix build break after merging trunk

Posted by ar...@apache.org.
Fix build break after merging trunk


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

Branch: refs/heads/HDFS-7240
Commit: f827d3b25212f41f5b9395b5a04b3f1ed1ec2697
Parents: dd784e9
Author: Arpit Agarwal <ar...@apache.org>
Authored: Sat Jun 20 14:23:16 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Sat Jun 20 14:23:16 2015 -0700

----------------------------------------------------------------------
 .../hadoop/storagecontainer/StorageContainerNameService.java      | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f827d3b2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/storagecontainer/StorageContainerNameService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/storagecontainer/StorageContainerNameService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/storagecontainer/StorageContainerNameService.java
index 23f2ade..dd3bb67 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/storagecontainer/StorageContainerNameService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/storagecontainer/StorageContainerNameService.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.storagecontainer;
 
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
@@ -149,7 +150,7 @@ public class StorageContainerNameService implements Namesystem {
   }
 
   @Override
-  public void decrementSafeBlockCount(Block b) {
+  public void decrementSafeBlockCount(BlockInfo b) {
     // Do nothing
   }
 }


[15/50] [abbrv] hadoop git commit: HADOOP-12076. Incomplete Cache Mechanism in CredentialProvider API. Contributed by Larry McCay.

Posted by ar...@apache.org.
HADOOP-12076. Incomplete Cache Mechanism in CredentialProvider API. Contributed by Larry McCay.


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

Branch: refs/heads/HDFS-7240
Commit: fbf55dcaf45285e1795cb107e7846799e4042b0b
Parents: 9cd5ad9
Author: cnauroth <cn...@apache.org>
Authored: Tue Jun 16 14:44:03 2015 -0700
Committer: cnauroth <cn...@apache.org>
Committed: Tue Jun 16 14:44:03 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt      |  3 +++
 .../security/alias/AbstractJavaKeyStoreProvider.java | 15 +--------------
 2 files changed, 4 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbf55dca/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 9a16a7d..67e9c76 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -861,6 +861,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12095. org.apache.hadoop.fs.shell.TestCount fails.
     (Brahma Reddy Battula via aajisaka)
 
+    HADOOP-12076. Incomplete Cache Mechanism in CredentialProvider API.
+    (Larry McCay via cnauroth)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbf55dca/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/AbstractJavaKeyStoreProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/AbstractJavaKeyStoreProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/AbstractJavaKeyStoreProvider.java
index 76b8cd5..9656261 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/AbstractJavaKeyStoreProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/AbstractJavaKeyStoreProvider.java
@@ -41,9 +41,7 @@ import java.security.UnrecoverableKeyException;
 import java.security.cert.CertificateException;
 import java.util.ArrayList;
 import java.util.Enumeration;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -174,13 +172,6 @@ public abstract class AbstractJavaKeyStoreProvider extends CredentialProvider {
     return keyStore;
   }
 
-  public Map<String, CredentialEntry> getCache() {
-    return cache;
-  }
-
-  private final Map<String, CredentialEntry> cache =
-      new HashMap<String, CredentialEntry>();
-
   protected final String getPathAsString() {
     return getPath().toString();
   }
@@ -213,9 +204,6 @@ public abstract class AbstractJavaKeyStoreProvider extends CredentialProvider {
     try {
       SecretKeySpec key = null;
       try {
-        if (cache.containsKey(alias)) {
-          return cache.get(alias);
-        }
         if (!keyStore.containsAlias(alias)) {
           return null;
         }
@@ -269,7 +257,7 @@ public abstract class AbstractJavaKeyStoreProvider extends CredentialProvider {
       throws IOException {
     writeLock.lock();
     try {
-      if (keyStore.containsAlias(alias) || cache.containsKey(alias)) {
+      if (keyStore.containsAlias(alias)) {
         throw new IOException("Credential " + alias + " already exists in "
             + this);
       }
@@ -296,7 +284,6 @@ public abstract class AbstractJavaKeyStoreProvider extends CredentialProvider {
       } catch (KeyStoreException e) {
         throw new IOException("Problem removing " + name + " from " + this, e);
       }
-      cache.remove(name);
       changed = true;
     } finally {
       writeLock.unlock();


[08/50] [abbrv] hadoop git commit: HDFS-8591. Remove support for deprecated configuration key dfs.namenode.decommission.nodes.per.interval.

Posted by ar...@apache.org.
HDFS-8591. Remove support for deprecated configuration key dfs.namenode.decommission.nodes.per.interval.


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

Branch: refs/heads/HDFS-7240
Commit: a3990ca41415515b986a41dacefceee1f05622f8
Parents: b039e69
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Jun 16 10:03:34 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue Jun 16 10:03:34 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../blockmanagement/DecommissionManager.java    | 42 +++++---------------
 .../apache/hadoop/hdfs/TestDecommission.java    | 26 ------------
 3 files changed, 12 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3990ca4/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 0ae2882..e0ef52f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -31,6 +31,9 @@ Trunk (Unreleased)
  
     HDFS-8135. Remove the deprecated FSConstants class. (Li Lu via wheat9)
 
+    HDFS-8591. Remove support for deprecated configuration key
+    dfs.namenode.decommission.nodes.per.interval. (wang)
+
   NEW FEATURES
 
     HDFS-3125. Add JournalService to enable Journal Daemon. (suresh)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3990ca4/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 639d2b0..797d031 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
@@ -136,29 +136,20 @@ public class DecommissionManager {
     checkArgument(intervalSecs >= 0, "Cannot set a negative " +
         "value for " + DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY);
 
-    // By default, the new configuration key overrides the deprecated one.
-    // No # node limit is set.
     int blocksPerInterval = conf.getInt(
         DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_BLOCKS_PER_INTERVAL_KEY,
         DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_BLOCKS_PER_INTERVAL_DEFAULT);
-    int nodesPerInterval = Integer.MAX_VALUE;
 
-    // If the expected key isn't present and the deprecated one is, 
-    // use the deprecated one into the new one. This overrides the 
-    // default.
-    //
-    // Also print a deprecation warning.
     final String deprecatedKey =
         "dfs.namenode.decommission.nodes.per.interval";
     final String strNodes = conf.get(deprecatedKey);
     if (strNodes != null) {
-      nodesPerInterval = Integer.parseInt(strNodes);
-      blocksPerInterval = Integer.MAX_VALUE;
-      LOG.warn("Using deprecated configuration key {} value of {}.",
-          deprecatedKey, nodesPerInterval); 
+      LOG.warn("Deprecated configuration key {} will be ignored.",
+          deprecatedKey);
       LOG.warn("Please update your configuration to use {} instead.", 
           DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_BLOCKS_PER_INTERVAL_KEY);
     }
+
     checkArgument(blocksPerInterval > 0,
         "Must set a positive value for "
         + DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_BLOCKS_PER_INTERVAL_KEY);
@@ -170,15 +161,14 @@ public class DecommissionManager {
         "value for "
         + DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_MAX_CONCURRENT_TRACKED_NODES);
 
-    monitor = new Monitor(blocksPerInterval, 
-        nodesPerInterval, maxConcurrentTrackedNodes);
+    monitor = new Monitor(blocksPerInterval, maxConcurrentTrackedNodes);
     executor.scheduleAtFixedRate(monitor, intervalSecs, intervalSecs,
         TimeUnit.SECONDS);
 
     LOG.debug("Activating DecommissionManager with interval {} seconds, " +
-            "{} max blocks per interval, {} max nodes per interval, " +
+            "{} max blocks per interval, " +
             "{} max concurrently tracked nodes.", intervalSecs,
-        blocksPerInterval, nodesPerInterval, maxConcurrentTrackedNodes);
+        blocksPerInterval, maxConcurrentTrackedNodes);
   }
 
   /**
@@ -334,10 +324,6 @@ public class DecommissionManager {
      */
     private final int numBlocksPerCheck;
     /**
-     * The maximum number of nodes to check per tick.
-     */
-    private final int numNodesPerCheck;
-    /**
      * The maximum number of nodes to track in decomNodeBlocks. A value of 0
      * means no limit.
      */
@@ -348,7 +334,7 @@ public class DecommissionManager {
     private int numBlocksChecked = 0;
     /**
      * The number of nodes that have been checked on this tick. Used for 
-     * testing.
+     * statistics.
      */
     private int numNodesChecked = 0;
     /**
@@ -357,10 +343,8 @@ public class DecommissionManager {
     private DatanodeDescriptor iterkey = new DatanodeDescriptor(new 
         DatanodeID("", "", "", 0, 0, 0, 0));
 
-    Monitor(int numBlocksPerCheck, int numNodesPerCheck, int 
-        maxConcurrentTrackedNodes) {
+    Monitor(int numBlocksPerCheck, int maxConcurrentTrackedNodes) {
       this.numBlocksPerCheck = numBlocksPerCheck;
-      this.numNodesPerCheck = numNodesPerCheck;
       this.maxConcurrentTrackedNodes = maxConcurrentTrackedNodes;
     }
 
@@ -369,12 +353,6 @@ public class DecommissionManager {
       return numBlocksChecked >= numBlocksPerCheck;
     }
 
-    @Deprecated
-    private boolean exceededNumNodesPerCheck() {
-      LOG.trace("Processed {} nodes so far this tick", numNodesChecked);
-      return numNodesChecked >= numNodesPerCheck;
-    }
-
     @Override
     public void run() {
       if (!namesystem.isRunning()) {
@@ -416,9 +394,7 @@ public class DecommissionManager {
           it = new CyclicIteration<>(decomNodeBlocks, iterkey).iterator();
       final LinkedList<DatanodeDescriptor> toRemove = new LinkedList<>();
 
-      while (it.hasNext()
-          && !exceededNumBlocksPerCheck()
-          && !exceededNumNodesPerCheck()) {
+      while (it.hasNext() && !exceededNumBlocksPerCheck()) {
         numNodesChecked++;
         final Map.Entry<DatanodeDescriptor, AbstractList<BlockInfo>>
             entry = it.next();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3990ca4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
index 1f18014..8f965ad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
@@ -1045,32 +1045,6 @@ public class TestDecommission {
     doDecomCheck(datanodeManager, decomManager, 1);
   }
 
-  @Deprecated
-  @Test(timeout=120000)
-  public void testNodesPerInterval() throws Exception {
-    Configuration newConf = new Configuration(conf);
-    org.apache.log4j.Logger.getLogger(DecommissionManager.class)
-        .setLevel(Level.TRACE);
-    // Set the deprecated configuration key which limits the # of nodes per 
-    // interval
-    newConf.setInt("dfs.namenode.decommission.nodes.per.interval", 1);
-    // Disable the normal monitor runs
-    newConf.setInt(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY,
-        Integer.MAX_VALUE);
-    startCluster(1, 3, newConf);
-    final FileSystem fs = cluster.getFileSystem();
-    final DatanodeManager datanodeManager =
-        cluster.getNamesystem().getBlockManager().getDatanodeManager();
-    final DecommissionManager decomManager = datanodeManager.getDecomManager();
-
-    // Write a 3 block file, so each node has one block. Should scan 1 node 
-    // each time.
-    DFSTestUtil.createFile(fs, new Path("/file1"), 64, (short) 3, 0xBAD1DEA);
-    for (int i=0; i<3; i++) {
-      doDecomCheck(datanodeManager, decomManager, 1);
-    }
-  }
-
   private void doDecomCheck(DatanodeManager datanodeManager,
       DecommissionManager decomManager, int expectedNumCheckedNodes)
       throws IOException, ExecutionException, InterruptedException {


[11/50] [abbrv] hadoop git commit: HDFS-4660. Block corruption can happen during pipeline recovery. Contributed by Kihwal Lee.

Posted by ar...@apache.org.
HDFS-4660. Block corruption can happen during pipeline recovery. 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/c74517c4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c74517c4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c74517c4

Branch: refs/heads/HDFS-7240
Commit: c74517c46bf00af408ed866b6577623cdec02de1
Parents: b578807
Author: Kihwal Lee <ki...@apache.org>
Authored: Tue Jun 16 15:39:46 2015 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Tue Jun 16 15:40:16 2015 -0500

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../hdfs/server/datanode/BlockReceiver.java     | 126 ++++++++++++++-----
 2 files changed, 96 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c74517c4/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 42588cc..b921f2c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1026,6 +1026,8 @@ Release 2.7.1 - UNRELEASED
 
     HDFS-8597. Fix TestFSImage#testZeroBlockSize on Windows. (Xiaoyu Yao)
 
+    HDFS-4660. Block corruption can happen during pipeline recovery (kihwal)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c74517c4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
index c46892d..2468f43 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
@@ -588,29 +588,59 @@ class BlockReceiver implements Closeable {
       try {
         long onDiskLen = replicaInfo.getBytesOnDisk();
         if (onDiskLen<offsetInBlock) {
-          //finally write to the disk :
-          
-          if (onDiskLen % bytesPerChecksum != 0) { 
-            // prepare to overwrite last checksum
-            adjustCrcFilePosition();
+          // Normally the beginning of an incoming packet is aligned with the
+          // existing data on disk. If the beginning packet data offset is not
+          // checksum chunk aligned, the end of packet will not go beyond the
+          // next chunk boundary.
+          // When a failure-recovery is involved, the client state and the
+          // the datanode state may not exactly agree. I.e. the client may
+          // resend part of data that is already on disk. Correct number of
+          // bytes should be skipped when writing the data and checksum
+          // buffers out to disk.
+          long partialChunkSizeOnDisk = onDiskLen % bytesPerChecksum;
+          boolean alignedOnDisk = partialChunkSizeOnDisk == 0;
+          boolean alignedInPacket = firstByteInBlock % bytesPerChecksum == 0;
+
+          // Since data is always appended, not overwritten, partial CRC
+          // recalculation is necessary if the on-disk data is not chunk-
+          // aligned, regardless of whether the beginning of the data in
+          // the packet is chunk-aligned.
+          boolean doPartialCrc = !alignedOnDisk && !shouldNotWriteChecksum;
+
+          // If this is a partial chunk, then verify that this is the only
+          // chunk in the packet. If the starting offset is not chunk
+          // aligned, the packet should terminate at or before the next
+          // chunk boundary.
+          if (!alignedInPacket && len > bytesPerChecksum) {
+            throw new IOException("Unexpected packet data length for "
+                +  block + " from " + inAddr + ": a partial chunk must be "
+                + " sent in an individual packet (data length = " + len
+                +  " > bytesPerChecksum = " + bytesPerChecksum + ")");
           }
-          
-          // If this is a partial chunk, then read in pre-existing checksum
+
+          // If the last portion of the block file is not a full chunk,
+          // then read in pre-existing partial data chunk and recalculate
+          // the checksum so that the checksum calculation can continue
+          // from the right state.
           Checksum partialCrc = null;
-          if (!shouldNotWriteChecksum && firstByteInBlock % bytesPerChecksum != 0) {
+          if (doPartialCrc) {
             if (LOG.isDebugEnabled()) {
               LOG.debug("receivePacket for " + block 
-                  + ": bytesPerChecksum=" + bytesPerChecksum                  
-                  + " does not divide firstByteInBlock=" + firstByteInBlock);
+                  + ": previous write did not end at the chunk boundary."
+                  + " onDiskLen=" + onDiskLen);
             }
             long offsetInChecksum = BlockMetadataHeader.getHeaderSize() +
                 onDiskLen / bytesPerChecksum * checksumSize;
             partialCrc = computePartialChunkCrc(onDiskLen, offsetInChecksum);
           }
 
+          // The data buffer position where write will begin. If the packet
+          // data and on-disk data have no overlap, this will not be at the
+          // beginning of the buffer.
           int startByteToDisk = (int)(onDiskLen-firstByteInBlock) 
               + dataBuf.arrayOffset() + dataBuf.position();
 
+          // Actual number of data bytes to write.
           int numBytesToDisk = (int)(offsetInBlock-onDiskLen);
           
           // Write data to disk.
@@ -625,31 +655,63 @@ class BlockReceiver implements Closeable {
           final byte[] lastCrc;
           if (shouldNotWriteChecksum) {
             lastCrc = null;
-          } else if (partialCrc != null) {
-            // If this is a partial chunk, then verify that this is the only
-            // chunk in the packet. Calculate new crc for this chunk.
-            if (len > bytesPerChecksum) {
-              throw new IOException("Unexpected packet data length for "
-                  +  block + " from " + inAddr + ": a partial chunk must be "
-                  + " sent in an individual packet (data length = " + len
-                  +  " > bytesPerChecksum = " + bytesPerChecksum + ")");
+          } else {
+            int skip = 0;
+            byte[] crcBytes = null;
+
+            // First, overwrite the partial crc at the end, if necessary.
+            if (doPartialCrc) { // not chunk-aligned on disk
+              // Calculate new crc for this chunk.
+              int bytesToReadForRecalc =
+                  (int)(bytesPerChecksum - partialChunkSizeOnDisk);
+              if (numBytesToDisk < bytesToReadForRecalc) {
+                bytesToReadForRecalc = numBytesToDisk;
+              }
+
+              partialCrc.update(dataBuf.array(), startByteToDisk,
+                  bytesToReadForRecalc);
+              byte[] buf = FSOutputSummer.convertToByteStream(partialCrc,
+                  checksumSize);
+              crcBytes = copyLastChunkChecksum(buf, checksumSize, buf.length);
+              // prepare to overwrite last checksum
+              adjustCrcFilePosition();
+              checksumOut.write(buf);
+              if(LOG.isDebugEnabled()) {
+                LOG.debug("Writing out partial crc for data len " + len +
+                    ", skip=" + skip);
+              }
+              skip++; //  For the partial chunk that was just read.
             }
-            partialCrc.update(dataBuf.array(), startByteToDisk, numBytesToDisk);
-            byte[] buf = FSOutputSummer.convertToByteStream(partialCrc, checksumSize);
-            lastCrc = copyLastChunkChecksum(buf, checksumSize, buf.length);
-            checksumOut.write(buf);
-            if(LOG.isDebugEnabled()) {
-              LOG.debug("Writing out partial crc for data len " + len);
+
+            // Determine how many checksums need to be skipped up to the last
+            // boundary. The checksum after the boundary was already counted
+            // above. Only count the number of checksums skipped up to the
+            // boundary here.
+            long lastChunkBoundary = onDiskLen - (onDiskLen%bytesPerChecksum);
+            long skippedDataBytes = lastChunkBoundary - firstByteInBlock;
+
+            if (skippedDataBytes > 0) {
+              skip += (int)(skippedDataBytes / bytesPerChecksum) +
+                  ((skippedDataBytes % bytesPerChecksum == 0) ? 0 : 1);
             }
-            partialCrc = null;
-          } else {
-            // write checksum
+            skip *= checksumSize; // Convert to number of bytes
+
+            // write the rest of checksum
             final int offset = checksumBuf.arrayOffset() +
-                checksumBuf.position();
-            final int end = offset + checksumLen;
-            lastCrc = copyLastChunkChecksum(checksumBuf.array(), checksumSize,
-                end);
-            checksumOut.write(checksumBuf.array(), offset, checksumLen);
+                checksumBuf.position() + skip;
+            final int end = offset + checksumLen - skip;
+            // If offset > end, there is no more checksum to write.
+            // I.e. a partial chunk checksum rewrite happened and there is no
+            // more to write after that.
+            if (offset > end) {
+              assert crcBytes != null;
+              lastCrc = crcBytes;
+            } else {
+              final int remainingBytes = checksumLen - skip;
+              lastCrc = copyLastChunkChecksum(checksumBuf.array(),
+                  checksumSize, end);
+              checksumOut.write(checksumBuf.array(), offset, remainingBytes);
+            }
           }
 
           /// flush entire packet, sync if requested


[03/50] [abbrv] hadoop git commit: Move HDFS-8540 to 2.8 in CHANGES.txt.

Posted by ar...@apache.org.
Move HDFS-8540 to 2.8 in CHANGES.txt.


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

Branch: refs/heads/HDFS-7240
Commit: 1b6695a4c0d76fe18d6524cc1379bc1185708c6f
Parents: 321940c
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jun 15 17:01:55 2015 -0700
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Mon Jun 15 17:01:55 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b6695a4/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 584d94d..9822575 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -629,6 +629,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8499. Refactor BlockInfo class hierarchy with static helper class.
     (Zhe Zhang via wang)
 
+    HDFS-8540.  Mover should exit with NO_MOVE_BLOCK if no block can be moved.
+    (surendra singh lilhore via szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than
@@ -917,9 +920,6 @@ Release 2.7.1 - UNRELEASED
     HDFS-8521. Add VisibleForTesting annotation to
     BlockPoolSlice#selectReplicaToDelete. (cmccabe)
 
-    HDFS-8540.  Mover should exit with NO_MOVE_BLOCK if no block can be moved.
-    (surendra singh lilhore via szetszwo)
-
   OPTIMIZATIONS
 
   BUG FIXES


[30/50] [abbrv] hadoop git commit: MAPREDUCE-6373. The logger reports total input paths but it is referring to input files. Contributed by Bibin A Chundatt.

Posted by ar...@apache.org.
MAPREDUCE-6373. The logger reports total input paths but it is referring
to input files. Contributed by Bibin A Chundatt.


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

Branch: refs/heads/HDFS-7240
Commit: 1babe50a2cbaae3c8165229347e743d0dc94e979
Parents: 74351af
Author: Devaraj K <de...@apache.org>
Authored: Thu Jun 18 11:42:22 2015 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Thu Jun 18 11:42:22 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                              | 3 +++
 .../src/main/java/org/apache/hadoop/mapred/FileInputFormat.java   | 2 +-
 .../org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java    | 3 +--
 3 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1babe50a/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index a9f5053..1037091 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -477,6 +477,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6363. [NNBench] Lease mismatch error when running with multiple
     mappers. (Brahma Reddy Battula via aajisaka)
 
+    MAPREDUCE-6373. The logger reports total input paths but it is referring
+    to input files. (Bibin A Chundatt via devaraj)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1babe50a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java
index c6cbd50..2c58ebe 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java
@@ -253,7 +253,7 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
       LOG.debug("Time taken to get FileStatuses: "
           + sw.now(TimeUnit.MILLISECONDS));
     }
-    LOG.info("Total input paths to process : " + result.length);
+    LOG.info("Total input files to process : " + result.length);
     return result;
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1babe50a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
index f5cd5d1..0c5ede9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.mapred.LocatedFileStatusFetcher;
-import org.apache.hadoop.mapred.SplitLocationInfo;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Job;
@@ -287,7 +286,7 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
       LOG.debug("Time taken to get FileStatuses: "
           + sw.now(TimeUnit.MILLISECONDS));
     }
-    LOG.info("Total input paths to process : " + result.size()); 
+    LOG.info("Total input files to process : " + result.size());
     return result;
   }
 


[39/50] [abbrv] hadoop git commit: HDFS-8626. Reserved RBW space is not released if creation of RBW File fails. (Contributed by kanaka kumar avvaru)

Posted by ar...@apache.org.
HDFS-8626. Reserved RBW space is not released if creation of RBW File fails. (Contributed by kanaka kumar avvaru)


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

Branch: refs/heads/HDFS-7240
Commit: 55278c012fbd5dcabee0276f8f42d0ab920cec93
Parents: 49f5d20
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri Jun 19 18:23:34 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Fri Jun 19 18:23:34 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../datanode/fsdataset/impl/FsVolumeImpl.java   |  7 ++-
 .../fsdataset/impl/TestRbwSpaceReservation.java | 45 ++++++++++++++++++++
 3 files changed, 54 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/55278c01/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 749a3ce..a93a214 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1064,6 +1064,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8633. Fix setting of dfs.datanode.readahead.bytes in hdfs-default.xml
     to match DFSConfigKeys. (Ray Chiang via Yongjun Zhang)
 
+    HDFS-8626. Reserved RBW space is not released if creation of RBW File
+    fails. (kanaka kumar avvaru via Arpit Agarwal)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55278c01/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
index 49a56bb..e90f5d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
@@ -780,7 +780,12 @@ public class FsVolumeImpl implements FsVolumeSpi {
   File createRbwFile(String bpid, Block b) throws IOException {
     checkReference();
     reserveSpaceForRbw(b.getNumBytes());
-    return getBlockPoolSlice(bpid).createRbwFile(b);
+    try {
+      return getBlockPoolSlice(bpid).createRbwFile(b);
+    } catch (IOException exception) {
+      releaseReservedSpace(b.getNumBytes());
+      throw exception;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55278c01/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestRbwSpaceReservation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestRbwSpaceReservation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestRbwSpaceReservation.java
index 9b83b39..44bf81b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestRbwSpaceReservation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestRbwSpaceReservation.java
@@ -27,10 +27,13 @@ import org.apache.hadoop.conf.Configuration;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
 import static org.hamcrest.core.Is.is;
 import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.*;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
@@ -42,9 +45,12 @@ import org.junit.After;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
+import org.mockito.Mockito;
 
 import java.io.IOException;
 import java.io.OutputStream;
+import java.lang.reflect.Field;
+import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.TimeoutException;
 
@@ -279,6 +285,45 @@ public class TestRbwSpaceReservation {
     }
   }
 
+  @SuppressWarnings("unchecked")
+  @Test(timeout = 30000)
+  public void testRBWFileCreationError() throws Exception {
+
+    final short replication = 1;
+    startCluster(BLOCK_SIZE, replication, -1);
+
+    final FsVolumeImpl fsVolumeImpl = (FsVolumeImpl) cluster.getDataNodes()
+        .get(0).getFSDataset().getFsVolumeReferences().get(0);
+    final String methodName = GenericTestUtils.getMethodName();
+    final Path file = new Path("/" + methodName + ".01.dat");
+
+    // Mock BlockPoolSlice so that RBW file creation gives IOExcception
+    BlockPoolSlice blockPoolSlice = Mockito.mock(BlockPoolSlice.class);
+    Mockito.when(blockPoolSlice.createRbwFile((Block) Mockito.any()))
+        .thenThrow(new IOException("Synthetic IO Exception Throgh MOCK"));
+
+    Field field = FsVolumeImpl.class.getDeclaredField("bpSlices");
+    field.setAccessible(true);
+    Map<String, BlockPoolSlice> bpSlices = (Map<String, BlockPoolSlice>) field
+        .get(fsVolumeImpl);
+    bpSlices.put(fsVolumeImpl.getBlockPoolList()[0], blockPoolSlice);
+
+    try {
+      // Write 1 byte to the file
+      FSDataOutputStream os = fs.create(file, replication);
+      os.write(new byte[1]);
+      os.hsync();
+      os.close();
+      fail("Expecting IOException file creation failure");
+    } catch (IOException e) {
+      // Exception can be ignored (expected)
+    }
+
+    // Ensure RBW space reserved is released
+    assertTrue("Expected ZERO but got " + fsVolumeImpl.getReservedForRbw(),
+        fsVolumeImpl.getReservedForRbw() == 0);
+  }
+
   /**
    * Stress test to ensure we are not leaking reserved space.
    * @throws IOException


[25/50] [abbrv] hadoop git commit: HDFS-8446. Separate safemode related operations in GetBlockLocations(). Contributed by Haohui Mai.

Posted by ar...@apache.org.
HDFS-8446. Separate safemode related operations in GetBlockLocations(). Contributed by Haohui Mai.


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

Branch: refs/heads/HDFS-7240
Commit: 015535dc0ad00c2ba357afb3d1e283e56ddda0d6
Parents: a826d43
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Jun 17 16:21:37 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Wed Jun 17 16:38:43 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hdfs/server/namenode/FSDirAttrOp.java       |   3 +-
 .../server/namenode/FSDirStatAndListingOp.java  |  93 +++++++++--
 .../hdfs/server/namenode/FSDirectory.java       |  20 +++
 .../hdfs/server/namenode/FSNamesystem.java      | 153 +++----------------
 .../hdfs/server/namenode/NamenodeFsck.java      |   5 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |  16 +-
 7 files changed, 140 insertions(+), 153 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/015535dc/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 6ef405b..6dfcd18 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -647,6 +647,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8238. Move ClientProtocol to the hdfs-client.
     (Takanobu Asanuma via wheat9)
 
+    HDFS-8446. Separate safemode related operations in GetBlockLocations().
+    (wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/015535dc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
index 3b07320..b322b69 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
@@ -474,8 +474,7 @@ public class FSDirAttrOp {
 
       // if the last access time update was within the last precision interval, then
       // no need to store access time
-      if (atime <= inodeTime + fsd.getFSNamesystem().getAccessTimePrecision()
-          && !force) {
+      if (atime <= inodeTime + fsd.getAccessTimePrecision() && !force) {
         status =  false;
       } else {
         inode.setAccessTime(atime, latest);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/015535dc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index c636d93..201dabc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -35,6 +36,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
@@ -43,6 +45,8 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.Arrays;
 
+import static org.apache.hadoop.util.Time.now;
+
 class FSDirStatAndListingOp {
   static DirectoryListing getListingInt(FSDirectory fsd, final String srcArg,
       byte[] startAfter, boolean needLocation) throws IOException {
@@ -137,9 +141,69 @@ class FSDirStatAndListingOp {
     return getContentSummaryInt(fsd, iip);
   }
 
+  /**
+   * Get block locations within the specified range.
+   * @see ClientProtocol#getBlockLocations(String, long, long)
+   * @throws IOException
+   */
+  static GetBlockLocationsResult getBlockLocations(
+      FSDirectory fsd, FSPermissionChecker pc, String src, long offset,
+      long length, boolean needBlockToken) throws IOException {
+    Preconditions.checkArgument(offset >= 0,
+        "Negative offset is not supported. File: " + src);
+    Preconditions.checkArgument(length >= 0,
+        "Negative length is not supported. File: " + src);
+    CacheManager cm = fsd.getFSNamesystem().getCacheManager();
+    BlockManager bm = fsd.getBlockManager();
+    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    boolean isReservedName = FSDirectory.isReservedRawName(src);
+    fsd.readLock();
+    try {
+      src = fsd.resolvePath(pc, src, pathComponents);
+      final INodesInPath iip = fsd.getINodesInPath(src, true);
+      final INodeFile inode = INodeFile.valueOf(iip.getLastINode(), src);
+      if (fsd.isPermissionEnabled()) {
+        fsd.checkPathAccess(pc, iip, FsAction.READ);
+        fsd.checkUnreadableBySuperuser(pc, inode, iip.getPathSnapshotId());
+      }
+
+      final long fileSize = iip.isSnapshot()
+          ? inode.computeFileSize(iip.getPathSnapshotId())
+          : inode.computeFileSizeNotIncludingLastUcBlock();
+
+      boolean isUc = inode.isUnderConstruction();
+      if (iip.isSnapshot()) {
+        // if src indicates a snapshot file, we need to make sure the returned
+        // blocks do not exceed the size of the snapshot file.
+        length = Math.min(length, fileSize - offset);
+        isUc = false;
+      }
+
+      final FileEncryptionInfo feInfo = isReservedName ? null
+          : fsd.getFileEncryptionInfo(inode, iip.getPathSnapshotId(), iip);
+
+      final LocatedBlocks blocks = bm.createLocatedBlocks(
+          inode.getBlocks(iip.getPathSnapshotId()), fileSize, isUc, offset,
+          length, needBlockToken, iip.isSnapshot(), feInfo);
+
+      // Set caching information for the located blocks.
+      for (LocatedBlock lb : blocks.getLocatedBlocks()) {
+        cm.setCachedLocations(lb);
+      }
+
+      final long now = now();
+      boolean updateAccessTime = fsd.isAccessTimeSupported()
+          && !iip.isSnapshot()
+          && now > inode.getAccessTime() + fsd.getAccessTimePrecision();
+      return new GetBlockLocationsResult(updateAccessTime, blocks);
+    } finally {
+      fsd.readUnlock();
+    }
+  }
+
   private static byte getStoragePolicyID(byte inodePolicy, byte parentPolicy) {
-    return inodePolicy != HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED ? inodePolicy :
-        parentPolicy;
+    return inodePolicy != HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED
+        ? inodePolicy : parentPolicy;
   }
 
   /**
@@ -294,13 +358,11 @@ class FSDirStatAndListingOp {
       byte policyId = includeStoragePolicy && !i.isSymlink() ?
           i.getStoragePolicyID() :
           HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
-      INodeAttributes nodeAttrs = getINodeAttributes(
-          fsd, path, HdfsFileStatus.EMPTY_NAME, i, src.getPathSnapshotId());
-      return createFileStatus(
-          fsd, HdfsFileStatus.EMPTY_NAME,
-          i, nodeAttrs, policyId,
-          src.getPathSnapshotId(),
-          isRawPath, src);
+      INodeAttributes nodeAttrs = getINodeAttributes(fsd, path,
+                                                     HdfsFileStatus.EMPTY_NAME,
+                                                     i, src.getPathSnapshotId());
+      return createFileStatus(fsd, HdfsFileStatus.EMPTY_NAME, i, nodeAttrs,
+                              policyId, src.getPathSnapshotId(), isRawPath, src);
     } finally {
       fsd.readUnlock();
     }
@@ -520,4 +582,17 @@ class FSDirStatAndListingOp {
       fsd.readUnlock();
     }
   }
+
+  static class GetBlockLocationsResult {
+    final boolean updateAccessTime;
+    final LocatedBlocks blocks;
+    boolean updateAccessTime() {
+      return updateAccessTime;
+    }
+    private GetBlockLocationsResult(
+        boolean updateAccessTime, LocatedBlocks blocks) {
+      this.updateAccessTime = updateAccessTime;
+      this.blocks = blocks;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/015535dc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 5b8650e..c807fba 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -80,6 +80,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_DE
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.CRYPTO_XATTR_ENCRYPTION_ZONE;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.CRYPTO_XATTR_FILE_ENCRYPTION_INFO;
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.SECURITY_XATTR_UNREADABLE_BY_SUPERUSER;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID;
 
 /**
@@ -92,6 +93,7 @@ import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_S
 @InterfaceAudience.Private
 public class FSDirectory implements Closeable {
   static final Logger LOG = LoggerFactory.getLogger(FSDirectory.class);
+
   private static INodeDirectory createRoot(FSNamesystem namesystem) {
     final INodeDirectory r = new INodeDirectory(
         INodeId.ROOT_INODE_ID,
@@ -328,6 +330,9 @@ public class FSDirectory implements Closeable {
   boolean isAccessTimeSupported() {
     return accessTimePrecision > 0;
   }
+  long getAccessTimePrecision() {
+    return accessTimePrecision;
+  }
   boolean isQuotaByStorageTypeEnabled() {
     return quotaByStorageTypeEnabled;
   }
@@ -1550,6 +1555,21 @@ public class FSDirectory implements Closeable {
     }
   }
 
+  void checkUnreadableBySuperuser(
+      FSPermissionChecker pc, INode inode, int snapshotId)
+      throws IOException {
+    if (pc.isSuperUser()) {
+      for (XAttr xattr : FSDirXAttrOp.getXAttrs(this, inode, snapshotId)) {
+        if (XAttrHelper.getPrefixName(xattr).
+            equals(SECURITY_XATTR_UNREADABLE_BY_SUPERUSER)) {
+          throw new AccessControlException(
+              "Access is denied for " + pc.getUser() + " since the superuser "
+              + "is not allowed to perform this operation.");
+        }
+      }
+    }
+  }
+
   HdfsFileStatus getAuditFileInfo(INodesInPath iip)
       throws IOException {
     return (namesystem.isAuditEnabled() && namesystem.isExternalInvocation())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/015535dc/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 b97776a..d82da93 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
@@ -34,8 +34,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_DEF
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOGGERS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY;
@@ -87,7 +85,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROU
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
-import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.SECURITY_XATTR_UNREADABLE_BY_SUPERUSER;
+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;
 
@@ -169,7 +167,6 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.UnknownCryptoProtocolVersionException;
-import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
@@ -484,9 +481,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   private final long minBlockSize;         // minimum block size
   final long maxBlocksPerFile;     // maximum # of blocks per file
 
-  // precision of access times.
-  private final long accessTimePrecision;
-
   /** Lock to protect FSNamesystem. */
   private final FSNamesystemLock fsLock;
 
@@ -800,8 +794,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_DEFAULT);
       this.maxBlocksPerFile = conf.getLong(DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_KEY,
           DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_DEFAULT);
-      this.accessTimePrecision = conf.getLong(DFS_NAMENODE_ACCESSTIME_PRECISION_KEY,
-          DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT);
 
       this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
       
@@ -1631,14 +1623,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return serverDefaults;
   }
 
-  long getAccessTimePrecision() {
-    return accessTimePrecision;
-  }
-
-  private boolean isAccessTimeSupported() {
-    return accessTimePrecision > 0;
-  }
-
   /////////////////////////////////////////////////////////
   //
   // These methods are called by HadoopFS clients
@@ -1689,19 +1673,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     logAuditEvent(true, "setOwner", src, null, auditStat);
   }
 
-  static class GetBlockLocationsResult {
-    final boolean updateAccessTime;
-    final LocatedBlocks blocks;
-    boolean updateAccessTime() {
-      return updateAccessTime;
-    }
-    private GetBlockLocationsResult(
-        boolean updateAccessTime, LocatedBlocks blocks) {
-      this.updateAccessTime = updateAccessTime;
-      this.blocks = blocks;
-    }
-  }
-
   /**
    * Get block locations within the specified range.
    * @see ClientProtocol#getBlockLocations(String, long, long)
@@ -1714,7 +1685,23 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      res = getBlockLocations(pc, srcArg, offset, length, true, true);
+      res = FSDirStatAndListingOp.getBlockLocations(
+          dir, pc, srcArg, offset, length, true);
+      if (isInSafeMode()) {
+        for (LocatedBlock b : res.blocks.getLocatedBlocks()) {
+          // if safemode & no block locations yet then throw safemodeException
+          if ((b.getLocations() == null) || (b.getLocations().length == 0)) {
+            SafeModeException se = newSafemodeException(
+                "Zero blocklocations for " + srcArg);
+            if (haEnabled && haContext != null &&
+                haContext.getState().getServiceState() == HAServiceState.ACTIVE) {
+              throw new RetriableException(se);
+            } else {
+              throw se;
+            }
+          }
+        }
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, "open", srcArg);
       throw e;
@@ -1724,7 +1711,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     logAuditEvent(true, "open", srcArg);
 
-    if (res.updateAccessTime()) {
+    if (!isInSafeMode() && res.updateAccessTime()) {
       byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(
           srcArg);
       String src = srcArg;
@@ -1754,7 +1741,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         final INodesInPath iip = dir.getINodesInPath(src, true);
         INode inode = iip.getLastINode();
         boolean updateAccessTime = inode != null &&
-            now > inode.getAccessTime() + getAccessTimePrecision();
+            now > inode.getAccessTime() + dir.getAccessTimePrecision();
         if (!isInSafeMode() && updateAccessTime) {
           boolean changed = FSDirAttrOp.setTimes(dir,
               inode, -1, now, false, iip.getLatestSnapshotId());
@@ -1786,88 +1773,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   /**
-   * Get block locations within the specified range.
-   * @see ClientProtocol#getBlockLocations(String, long, long)
-   * @throws IOException
-   */
-  GetBlockLocationsResult getBlockLocations(
-      FSPermissionChecker pc, String src, long offset, long length,
-      boolean needBlockToken, boolean checkSafeMode) throws IOException {
-    if (offset < 0) {
-      throw new HadoopIllegalArgumentException(
-          "Negative offset is not supported. File: " + src);
-    }
-    if (length < 0) {
-      throw new HadoopIllegalArgumentException(
-          "Negative length is not supported. File: " + src);
-    }
-    final GetBlockLocationsResult ret = getBlockLocationsInt(
-        pc, src, offset, length, needBlockToken);
-
-    if (checkSafeMode && isInSafeMode()) {
-      for (LocatedBlock b : ret.blocks.getLocatedBlocks()) {
-        // if safemode & no block locations yet then throw safemodeException
-        if ((b.getLocations() == null) || (b.getLocations().length == 0)) {
-          SafeModeException se = newSafemodeException(
-              "Zero blocklocations for " + src);
-          if (haEnabled && haContext != null &&
-              haContext.getState().getServiceState() == HAServiceState.ACTIVE) {
-            throw new RetriableException(se);
-          } else {
-            throw se;
-          }
-        }
-      }
-    }
-    return ret;
-  }
-
-  private GetBlockLocationsResult getBlockLocationsInt(
-      FSPermissionChecker pc, final String srcArg, long offset, long length,
-      boolean needBlockToken)
-      throws IOException {
-    String src = srcArg;
-    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
-    src = dir.resolvePath(pc, srcArg, pathComponents);
-    final INodesInPath iip = dir.getINodesInPath(src, true);
-    final INodeFile inode = INodeFile.valueOf(iip.getLastINode(), src);
-    if (isPermissionEnabled) {
-      dir.checkPathAccess(pc, iip, FsAction.READ);
-      checkUnreadableBySuperuser(pc, inode, iip.getPathSnapshotId());
-    }
-
-    final long fileSize = iip.isSnapshot()
-        ? inode.computeFileSize(iip.getPathSnapshotId())
-        : inode.computeFileSizeNotIncludingLastUcBlock();
-    boolean isUc = inode.isUnderConstruction();
-    if (iip.isSnapshot()) {
-      // if src indicates a snapshot file, we need to make sure the returned
-      // blocks do not exceed the size of the snapshot file.
-      length = Math.min(length, fileSize - offset);
-      isUc = false;
-    }
-
-    final FileEncryptionInfo feInfo =
-        FSDirectory.isReservedRawName(srcArg) ? null
-            : dir.getFileEncryptionInfo(inode, iip.getPathSnapshotId(), iip);
-
-    final LocatedBlocks blocks = blockManager.createLocatedBlocks(
-        inode.getBlocks(iip.getPathSnapshotId()), fileSize, isUc, offset,
-        length, needBlockToken, iip.isSnapshot(), feInfo);
-
-    // Set caching information for the located blocks.
-    for (LocatedBlock lb : blocks.getLocatedBlocks()) {
-      cacheManager.setCachedLocations(lb);
-    }
-
-    final long now = now();
-    boolean updateAccessTime = isAccessTimeSupported() && !isInSafeMode()
-        && !iip.isSnapshot()
-        && now > inode.getAccessTime() + getAccessTimePrecision();
-    return new GetBlockLocationsResult(updateAccessTime, blocks);
-  }
-
-  /**
    * Moves all the blocks from {@code srcs} and appends them to {@code target}
    * To avoid rollbacks we will verify validity of ALL of the args
    * before we start actual move.
@@ -3912,8 +3817,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     readLock();
     try {
       checkOperation(NameNode.OperationCategory.READ);
-      dl = FSDirStatAndListingOp.getListingInt(dir, src, startAfter,
-          needLocation);
+      dl = getListingInt(dir, src, startAfter, needLocation);
     } catch (AccessControlException e) {
       logAuditEvent(false, "listStatus", src);
       throw e;
@@ -5309,21 +5213,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return new PermissionStatus(fsOwner.getShortUserName(), supergroup, permission);
   }
 
-  private void checkUnreadableBySuperuser(FSPermissionChecker pc,
-      INode inode, int snapshotId)
-      throws IOException {
-    if (pc.isSuperUser()) {
-      for (XAttr xattr : FSDirXAttrOp.getXAttrs(dir, inode, snapshotId)) {
-        if (XAttrHelper.getPrefixName(xattr).
-            equals(SECURITY_XATTR_UNREADABLE_BY_SUPERUSER)) {
-          throw new AccessControlException("Access is denied for " +
-              pc.getUser() + " since the superuser is not allowed to " +
-              "perform this operation.");
-        }
-      }
-    }
-  }
-
   @Override
   public void checkSuperuserPrivilege()
       throws AccessControlException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/015535dc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index f67d25a..7d4cd7e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -483,8 +483,9 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     final FSNamesystem fsn = namenode.getNamesystem();
     fsn.readLock();
     try {
-      blocks = fsn.getBlockLocations(
-          fsn.getPermissionChecker(), path, 0, fileLen, false, false)
+      blocks = FSDirStatAndListingOp.getBlockLocations(
+          fsn.getFSDirectory(), fsn.getPermissionChecker(),
+          path, 0, fileLen, false)
           .blocks;
     } catch (FileNotFoundException fnfe) {
       blocks = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/015535dc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index eabd0c8..8818f17 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@ -1159,20 +1159,21 @@ public class TestFsck {
     Configuration conf = new Configuration();
     NameNode namenode = mock(NameNode.class);
     NetworkTopology nettop = mock(NetworkTopology.class);
-    Map<String,String[]> pmap = new HashMap<String, String[]>();
+    Map<String,String[]> pmap = new HashMap<>();
     Writer result = new StringWriter();
     PrintWriter out = new PrintWriter(result, true);
     InetAddress remoteAddress = InetAddress.getLocalHost();
     FSNamesystem fsName = mock(FSNamesystem.class);
+    FSDirectory fsd = mock(FSDirectory.class);
     BlockManager blockManager = mock(BlockManager.class);
     DatanodeManager dnManager = mock(DatanodeManager.class);
+    INodesInPath iip = mock(INodesInPath.class);
 
     when(namenode.getNamesystem()).thenReturn(fsName);
-    when(fsName.getBlockLocations(any(FSPermissionChecker.class), anyString(),
-                                  anyLong(), anyLong(),
-                                  anyBoolean(), anyBoolean()))
-        .thenThrow(new FileNotFoundException());
     when(fsName.getBlockManager()).thenReturn(blockManager);
+    when(fsName.getFSDirectory()).thenReturn(fsd);
+    when(fsd.getFSNamesystem()).thenReturn(fsName);
+    when(fsd.getINodesInPath(anyString(), anyBoolean())).thenReturn(iip);
     when(blockManager.getDatanodeManager()).thenReturn(dnManager);
 
     NamenodeFsck fsck = new NamenodeFsck(conf, namenode, nettop, pmap, out,
@@ -1190,8 +1191,7 @@ public class TestFsck {
     String owner = "foo";
     String group = "bar";
     byte [] symlink = null;
-    byte [] path = new byte[128];
-    path = DFSUtil.string2Bytes(pathString);
+    byte [] path = DFSUtil.string2Bytes(pathString);
     long fileId = 312321L;
     int numChildren = 1;
     byte storagePolicy = 0;
@@ -1204,7 +1204,7 @@ public class TestFsck {
     try {
       fsck.check(pathString, file, res);
     } catch (Exception e) {
-      fail("Unexpected exception "+ e.getMessage());
+      fail("Unexpected exception " + e.getMessage());
     }
     assertTrue(res.toString().contains("HEALTHY"));
   }


[21/50] [abbrv] hadoop git commit: HDFS-8238. Move ClientProtocol to the hdfs-client. Contributed by Takanobu Asanuma.

Posted by ar...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8327744/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
deleted file mode 100644
index 55faf16..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ /dev/null
@@ -1,1459 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.protocol;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.EnumSet;
-import java.util.List;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.crypto.CryptoProtocolVersion;
-import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
-import org.apache.hadoop.fs.CacheFlag;
-import org.apache.hadoop.fs.ContentSummary;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.FsServerDefaults;
-import org.apache.hadoop.fs.InvalidPathException;
-import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.Options.Rename;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.fs.XAttr;
-import org.apache.hadoop.fs.XAttrSetFlag;
-import org.apache.hadoop.fs.permission.AclEntry;
-import org.apache.hadoop.fs.permission.AclStatus;
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.inotify.EventBatchList;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
-import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
-import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
-import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
-import org.apache.hadoop.io.EnumSetWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.retry.AtMostOnce;
-import org.apache.hadoop.io.retry.Idempotent;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.KerberosInfo;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenInfo;
-
-/**********************************************************************
- * ClientProtocol is used by user code via 
- * {@link org.apache.hadoop.hdfs.DistributedFileSystem} class to communicate 
- * with the NameNode.  User code can manipulate the directory namespace, 
- * as well as open/close file streams, etc.
- *
- **********************************************************************/
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-@KerberosInfo(
-    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY)
-@TokenInfo(DelegationTokenSelector.class)
-public interface ClientProtocol {
-
-  /**
-   * Until version 69, this class ClientProtocol served as both
-   * the client interface to the NN AND the RPC protocol used to 
-   * communicate with the NN.
-   * 
-   * This class is used by both the DFSClient and the 
-   * NN server side to insulate from the protocol serialization.
-   * 
-   * If you are adding/changing this interface then you need to 
-   * change both this class and ALSO related protocol buffer
-   * wire protocol definition in ClientNamenodeProtocol.proto.
-   * 
-   * For more details on protocol buffer wire protocol, please see 
-   * .../org/apache/hadoop/hdfs/protocolPB/overview.html
-   * 
-   * The log of historical changes can be retrieved from the svn).
-   * 69: Eliminate overloaded method names.
-   * 
-   * 69L is the last version id when this class was used for protocols
-   *  serialization. DO not update this version any further. 
-   */
-  public static final long versionID = 69L;
-  
-  ///////////////////////////////////////
-  // File contents
-  ///////////////////////////////////////
-  /**
-   * Get locations of the blocks of the specified file within the specified range.
-   * DataNode locations for each block are sorted by
-   * the proximity to the client.
-   * <p>
-   * Return {@link LocatedBlocks} which contains
-   * file length, blocks and their locations.
-   * DataNode locations for each block are sorted by
-   * the distance to the client's address.
-   * <p>
-   * The client will then have to contact 
-   * one of the indicated DataNodes to obtain the actual data.
-   * 
-   * @param src file name
-   * @param offset range start offset
-   * @param length range length
-   *
-   * @return file length and array of blocks with their locations
-   *
-   * @throws AccessControlException If access is denied
-   * @throws FileNotFoundException If file <code>src</code> does not exist
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public LocatedBlocks getBlockLocations(String src,
-                                         long offset,
-                                         long length) 
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * Get server default values for a number of configuration params.
-   * @return a set of server default configuration values
-   * @throws IOException
-   */
-  @Idempotent
-  public FsServerDefaults getServerDefaults() throws IOException;
-
-  /**
-   * Create a new file entry in the namespace.
-   * <p>
-   * This will create an empty file specified by the source path.
-   * The path should reflect a full path originated at the root.
-   * The name-node does not have a notion of "current" directory for a client.
-   * <p>
-   * Once created, the file is visible and available for read to other clients.
-   * Although, other clients cannot {@link #delete(String, boolean)}, re-create or 
-   * {@link #rename(String, String)} it until the file is completed
-   * or explicitly as a result of lease expiration.
-   * <p>
-   * Blocks have a maximum size.  Clients that intend to create
-   * multi-block files must also use 
-   * {@link #addBlock}
-   *
-   * @param src path of the file being created.
-   * @param masked masked permission.
-   * @param clientName name of the current client.
-   * @param flag indicates whether the file should be 
-   * overwritten if it already exists or create if it does not exist or append.
-   * @param createParent create missing parent directory if true
-   * @param replication block replication factor.
-   * @param blockSize maximum block size.
-   * @param supportedVersions CryptoProtocolVersions supported by the client
-   * 
-   * @return the status of the created file, it could be null if the server
-   *           doesn't support returning the file status
-   * @throws AccessControlException If access is denied
-   * @throws AlreadyBeingCreatedException if the path does not exist.
-   * @throws DSQuotaExceededException If file creation violates disk space 
-   *           quota restriction
-   * @throws FileAlreadyExistsException If file <code>src</code> already exists
-   * @throws FileNotFoundException If parent of <code>src</code> does not exist
-   *           and <code>createParent</code> is false
-   * @throws ParentNotDirectoryException If parent of <code>src</code> is not a
-   *           directory.
-   * @throws NSQuotaExceededException If file creation violates name space 
-   *           quota restriction
-   * @throws SafeModeException create not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred
-   *
-   * RuntimeExceptions:
-   * @throws InvalidPathException Path <code>src</code> is invalid
-   * <p>
-   * <em>Note that create with {@link CreateFlag#OVERWRITE} is idempotent.</em>
-   */
-  @AtMostOnce
-  public HdfsFileStatus create(String src, FsPermission masked,
-      String clientName, EnumSetWritable<CreateFlag> flag,
-      boolean createParent, short replication, long blockSize, 
-      CryptoProtocolVersion[] supportedVersions)
-      throws AccessControlException, AlreadyBeingCreatedException,
-      DSQuotaExceededException, FileAlreadyExistsException,
-      FileNotFoundException, NSQuotaExceededException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      SnapshotAccessControlException, IOException;
-
-  /**
-   * Append to the end of the file. 
-   * @param src path of the file being created.
-   * @param clientName name of the current client.
-   * @param flag indicates whether the data is appended to a new block.
-   * @return wrapper with information about the last partial block and file
-   *    status if any
-   * @throws AccessControlException if permission to append file is 
-   * denied by the system. As usually on the client side the exception will 
-   * be wrapped into {@link org.apache.hadoop.ipc.RemoteException}.
-   * Allows appending to an existing file if the server is
-   * configured with the parameter dfs.support.append set to true, otherwise
-   * throws an IOException.
-   * 
-   * @throws AccessControlException If permission to append to file is denied
-   * @throws FileNotFoundException If file <code>src</code> is not found
-   * @throws DSQuotaExceededException If append violates disk space quota 
-   *           restriction
-   * @throws SafeModeException append not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred.
-   *
-   * RuntimeExceptions:
-   * @throws UnsupportedOperationException if append is not supported
-   */
-  @AtMostOnce
-  public LastBlockWithStatus append(String src, String clientName,
-      EnumSetWritable<CreateFlag> flag) throws AccessControlException,
-      DSQuotaExceededException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
-
-  /**
-   * Set replication for an existing file.
-   * <p>
-   * The NameNode sets replication to the new value and returns.
-   * The actual block replication is not expected to be performed during  
-   * this method call. The blocks will be populated or removed in the 
-   * background as the result of the routine block maintenance procedures.
-   * 
-   * @param src file name
-   * @param replication new replication
-   * 
-   * @return true if successful;
-   *         false if file does not exist or is a directory
-   *
-   * @throws AccessControlException If access is denied
-   * @throws DSQuotaExceededException If replication violates disk space 
-   *           quota restriction
-   * @throws FileNotFoundException If file <code>src</code> is not found
-   * @throws SafeModeException not allowed in safemode
-   * @throws UnresolvedLinkException if <code>src</code> contains a symlink
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public boolean setReplication(String src, short replication)
-      throws AccessControlException, DSQuotaExceededException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      SnapshotAccessControlException, IOException;
-
-  /**
-   * Get all the available block storage policies.
-   * @return All the in-use block storage policies currently.
-   */
-  @Idempotent
-  public BlockStoragePolicy[] getStoragePolicies() throws IOException;
-
-  /**
-   * Set the storage policy for a file/directory
-   * @param src Path of an existing file/directory. 
-   * @param policyName The name of the storage policy
-   * @throws SnapshotAccessControlException If access is denied
-   * @throws UnresolvedLinkException if <code>src</code> contains a symlink
-   * @throws FileNotFoundException If file/dir <code>src</code> is not found
-   * @throws QuotaExceededException If changes violate the quota restriction
-   */
-  @Idempotent
-  public void setStoragePolicy(String src, String policyName)
-      throws SnapshotAccessControlException, UnresolvedLinkException,
-      FileNotFoundException, QuotaExceededException, IOException;
-
-  /**
-   * Set permissions for an existing file/directory.
-   * 
-   * @throws AccessControlException If access is denied
-   * @throws FileNotFoundException If file <code>src</code> is not found
-   * @throws SafeModeException not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public void setPermission(String src, FsPermission permission)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
-
-  /**
-   * Set Owner of a path (i.e. a file or a directory).
-   * The parameters username and groupname cannot both be null.
-   * @param src file path
-   * @param username If it is null, the original username remains unchanged.
-   * @param groupname If it is null, the original groupname remains unchanged.
-   *
-   * @throws AccessControlException If access is denied
-   * @throws FileNotFoundException If file <code>src</code> is not found
-   * @throws SafeModeException not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public void setOwner(String src, String username, String groupname)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
-
-  /**
-   * The client can give up on a block by calling abandonBlock().
-   * The client can then either obtain a new block, or complete or abandon the 
-   * file.
-   * Any partial writes to the block will be discarded.
-   * 
-   * @param b         Block to abandon
-   * @param fileId    The id of the file where the block resides.  Older clients
-   *                    will pass GRANDFATHER_INODE_ID here.
-   * @param src       The path of the file where the block resides.
-   * @param holder    Lease holder.
-   *
-   * @throws AccessControlException If access is denied
-   * @throws FileNotFoundException file <code>src</code> is not found
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public void abandonBlock(ExtendedBlock b, long fileId,
-      String src, String holder)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * A client that wants to write an additional block to the 
-   * indicated filename (which must currently be open for writing)
-   * should call addBlock().  
-   *
-   * addBlock() allocates a new block and datanodes the block data
-   * should be replicated to.
-   * 
-   * addBlock() also commits the previous block by reporting
-   * to the name-node the actual generation stamp and the length
-   * of the block that the client has transmitted to data-nodes.
-   *
-   * @param src the file being created
-   * @param clientName the name of the client that adds the block
-   * @param previous  previous block
-   * @param excludeNodes a list of nodes that should not be
-   * allocated for the current block
-   * @param fileId the id uniquely identifying a file
-   * @param favoredNodes the list of nodes where the client wants the blocks.
-   *          Nodes are identified by either host name or address.
-   *
-   * @return LocatedBlock allocated block information.
-   *
-   * @throws AccessControlException If access is denied
-   * @throws FileNotFoundException If file <code>src</code> is not found
-   * @throws NotReplicatedYetException previous blocks of the file are not
-   *           replicated yet. Blocks cannot be added until replication
-   *           completes.
-   * @throws SafeModeException create not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public LocatedBlock addBlock(String src, String clientName,
-      ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId, 
-      String[] favoredNodes)
-      throws AccessControlException, FileNotFoundException,
-      NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
-      IOException;
-
-  /** 
-   * Get a datanode for an existing pipeline.
-   * 
-   * @param src the file being written
-   * @param fileId the ID of the file being written
-   * @param blk the block being written
-   * @param existings the existing nodes in the pipeline
-   * @param excludes the excluded nodes
-   * @param numAdditionalNodes number of additional datanodes
-   * @param clientName the name of the client
-   * 
-   * @return the located block.
-   * 
-   * @throws AccessControlException If access is denied
-   * @throws FileNotFoundException If file <code>src</code> is not found
-   * @throws SafeModeException create not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public LocatedBlock getAdditionalDatanode(final String src,
-      final long fileId, final ExtendedBlock blk,
-      final DatanodeInfo[] existings,
-      final String[] existingStorageIDs,
-      final DatanodeInfo[] excludes,
-      final int numAdditionalNodes, final String clientName
-      ) throws AccessControlException, FileNotFoundException,
-          SafeModeException, UnresolvedLinkException, IOException;
-
-  /**
-   * The client is done writing data to the given filename, and would 
-   * like to complete it.  
-   *
-   * The function returns whether the file has been closed successfully.
-   * If the function returns false, the caller should try again.
-   * 
-   * close() also commits the last block of file by reporting
-   * to the name-node the actual generation stamp and the length
-   * of the block that the client has transmitted to data-nodes.
-   *
-   * A call to complete() will not return true until all the file's
-   * blocks have been replicated the minimum number of times.  Thus,
-   * DataNode failures may cause a client to call complete() several
-   * times before succeeding.
-   *
-   * @param src the file being created
-   * @param clientName the name of the client that adds the block
-   * @param last the last block info
-   * @param fileId the id uniquely identifying a file
-   *
-   * @return true if all file blocks are minimally replicated or false otherwise
-   *
-   * @throws AccessControlException If access is denied
-   * @throws FileNotFoundException If file <code>src</code> is not found
-   * @throws SafeModeException create not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink 
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public boolean complete(String src, String clientName,
-                          ExtendedBlock last, long fileId)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The client wants to report corrupted blocks (blocks with specified
-   * locations on datanodes).
-   * @param blocks Array of located blocks to report
-   */
-  @Idempotent
-  public void reportBadBlocks(LocatedBlock[] blocks) throws IOException;
-
-  ///////////////////////////////////////
-  // Namespace management
-  ///////////////////////////////////////
-  /**
-   * Rename an item in the file system namespace.
-   * @param src existing file or directory name.
-   * @param dst new name.
-   * @return true if successful, or false if the old name does not exist
-   * or if the new name already belongs to the namespace.
-   * 
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException an I/O error occurred 
-   */
-  @AtMostOnce
-  public boolean rename(String src, String dst) 
-      throws UnresolvedLinkException, SnapshotAccessControlException, IOException;
-
-  /**
-   * Moves blocks from srcs to trg and delete srcs
-   * 
-   * @param trg existing file
-   * @param srcs - list of existing files (same block size, same replication)
-   * @throws IOException if some arguments are invalid
-   * @throws UnresolvedLinkException if <code>trg</code> or <code>srcs</code>
-   *           contains a symlink
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   */
-  @AtMostOnce
-  public void concat(String trg, String[] srcs) 
-      throws IOException, UnresolvedLinkException, SnapshotAccessControlException;
-
-  /**
-   * Rename src to dst.
-   * <ul>
-   * <li>Fails if src is a file and dst is a directory.
-   * <li>Fails if src is a directory and dst is a file.
-   * <li>Fails if the parent of dst does not exist or is a file.
-   * </ul>
-   * <p>
-   * Without OVERWRITE option, rename fails if the dst already exists.
-   * With OVERWRITE option, rename overwrites the dst, if it is a file 
-   * or an empty directory. Rename fails if dst is a non-empty directory.
-   * <p>
-   * This implementation of rename is atomic.
-   * <p>
-   * @param src existing file or directory name.
-   * @param dst new name.
-   * @param options Rename options
-   * 
-   * @throws AccessControlException If access is denied
-   * @throws DSQuotaExceededException If rename violates disk space 
-   *           quota restriction
-   * @throws FileAlreadyExistsException If <code>dst</code> already exists and
-   *           <code>options</options> has {@link Rename#OVERWRITE} option
-   *           false.
-   * @throws FileNotFoundException If <code>src</code> does not exist
-   * @throws NSQuotaExceededException If rename violates namespace 
-   *           quota restriction
-   * @throws ParentNotDirectoryException If parent of <code>dst</code> 
-   *           is not a directory
-   * @throws SafeModeException rename not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> or
-   *           <code>dst</code> contains a symlink
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred
-   */
-  @AtMostOnce
-  public void rename2(String src, String dst, Options.Rename... options)
-      throws AccessControlException, DSQuotaExceededException,
-      FileAlreadyExistsException, FileNotFoundException,
-      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
-
-  /**
-   * Truncate file src to new size.
-   * <ul>
-   * <li>Fails if src is a directory.
-   * <li>Fails if src does not exist.
-   * <li>Fails if src is not closed.
-   * <li>Fails if new size is greater than current size.
-   * </ul>
-   * <p>
-   * This implementation of truncate is purely a namespace operation if truncate
-   * occurs at a block boundary. Requires DataNode block recovery otherwise.
-   * <p>
-   * @param src  existing file
-   * @param newLength  the target size
-   *
-   * @return true if client does not need to wait for block recovery,
-   * false if client needs to wait for block recovery.
-   *
-   * @throws AccessControlException If access is denied
-   * @throws FileNotFoundException If file <code>src</code> is not found
-   * @throws SafeModeException truncate not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public boolean truncate(String src, long newLength, String clientName)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
-
-  /**
-   * Delete the given file or directory from the file system.
-   * <p>
-   * same as delete but provides a way to avoid accidentally 
-   * deleting non empty directories programmatically. 
-   * @param src existing name
-   * @param recursive if true deletes a non empty directory recursively,
-   * else throws an exception.
-   * @return true only if the existing file or directory was actually removed 
-   * from the file system.
-   * 
-   * @throws AccessControlException If access is denied
-   * @throws FileNotFoundException If file <code>src</code> is not found
-   * @throws SafeModeException create not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred
-   */
-  @AtMostOnce
-  public boolean delete(String src, boolean recursive)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
-  
-  /**
-   * Create a directory (or hierarchy of directories) with the given
-   * name and permission.
-   *
-   * @param src The path of the directory being created
-   * @param masked The masked permission of the directory being created
-   * @param createParent create missing parent directory if true
-   *
-   * @return True if the operation success.
-   *
-   * @throws AccessControlException If access is denied
-   * @throws FileAlreadyExistsException If <code>src</code> already exists
-   * @throws FileNotFoundException If parent of <code>src</code> does not exist
-   *           and <code>createParent</code> is false
-   * @throws NSQuotaExceededException If file creation violates quota restriction
-   * @throws ParentNotDirectoryException If parent of <code>src</code> 
-   *           is not a directory
-   * @throws SafeModeException create not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred.
-   *
-   * RunTimeExceptions:
-   * @throws InvalidPathException If <code>src</code> is invalid
-   */
-  @Idempotent
-  public boolean mkdirs(String src, FsPermission masked, boolean createParent)
-      throws AccessControlException, FileAlreadyExistsException,
-      FileNotFoundException, NSQuotaExceededException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      SnapshotAccessControlException, IOException;
-
-  /**
-   * Get a partial listing of the indicated directory
-   *
-   * @param src the directory name
-   * @param startAfter the name to start listing after encoded in java UTF8
-   * @param needLocation if the FileStatus should contain block locations
-   *
-   * @return a partial listing starting after startAfter
-   *
-   * @throws AccessControlException permission denied
-   * @throws FileNotFoundException file <code>src</code> is not found
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public DirectoryListing getListing(String src,
-                                     byte[] startAfter,
-                                     boolean needLocation)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
-  
-  /**
-   * Get listing of all the snapshottable directories
-   * 
-   * @return Information about all the current snapshottable directory
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
-      throws IOException;
-
-  ///////////////////////////////////////
-  // System issues and management
-  ///////////////////////////////////////
-
-  /**
-   * Client programs can cause stateful changes in the NameNode
-   * that affect other clients.  A client may obtain a file and 
-   * neither abandon nor complete it.  A client might hold a series
-   * of locks that prevent other clients from proceeding.
-   * Clearly, it would be bad if a client held a bunch of locks
-   * that it never gave up.  This can happen easily if the client
-   * dies unexpectedly.
-   * <p>
-   * So, the NameNode will revoke the locks and live file-creates
-   * for clients that it thinks have died.  A client tells the
-   * NameNode that it is still alive by periodically calling
-   * renewLease().  If a certain amount of time passes since
-   * the last call to renewLease(), the NameNode assumes the
-   * client has died.
-   *
-   * @throws AccessControlException permission denied
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public void renewLease(String clientName) throws AccessControlException,
-      IOException;
-
-  /**
-   * Start lease recovery.
-   * Lightweight NameNode operation to trigger lease recovery
-   * 
-   * @param src path of the file to start lease recovery
-   * @param clientName name of the current client
-   * @return true if the file is already closed
-   * @throws IOException
-   */
-  @Idempotent
-  public boolean recoverLease(String src, String clientName) throws IOException;
-
-  public int GET_STATS_CAPACITY_IDX = 0;
-  public int GET_STATS_USED_IDX = 1;
-  public int GET_STATS_REMAINING_IDX = 2;
-  public int GET_STATS_UNDER_REPLICATED_IDX = 3;
-  public int GET_STATS_CORRUPT_BLOCKS_IDX = 4;
-  public int GET_STATS_MISSING_BLOCKS_IDX = 5;
-  public int GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX = 6;
-  
-  /**
-   * Get a set of statistics about the filesystem.
-   * Right now, only seven values are returned.
-   * <ul>
-   * <li> [0] contains the total storage capacity of the system, in bytes.</li>
-   * <li> [1] contains the total used space of the system, in bytes.</li>
-   * <li> [2] contains the available storage of the system, in bytes.</li>
-   * <li> [3] contains number of under replicated blocks in the system.</li>
-   * <li> [4] contains number of blocks with a corrupt replica. </li>
-   * <li> [5] contains number of blocks without any good replicas left. </li>
-   * <li> [6] contains number of blocks which have replication factor
-   *          1 and have lost the only replica. </li>
-   * </ul>
-   * Use public constants like {@link #GET_STATS_CAPACITY_IDX} in place of 
-   * actual numbers to index into the array.
-   */
-  @Idempotent
-  public long[] getStats() throws IOException;
-
-  /**
-   * Get a report on the system's current datanodes.
-   * One DatanodeInfo object is returned for each DataNode.
-   * Return live datanodes if type is LIVE; dead datanodes if type is DEAD;
-   * otherwise all datanodes if type is ALL.
-   */
-  @Idempotent
-  public DatanodeInfo[] getDatanodeReport(HdfsConstants.DatanodeReportType type)
-      throws IOException;
-
-  /**
-   * Get a report on the current datanode storages.
-   */
-  @Idempotent
-  public DatanodeStorageReport[] getDatanodeStorageReport(
-      HdfsConstants.DatanodeReportType type) throws IOException;
-
-  /**
-   * Get the block size for the given file.
-   * @param filename The name of the file
-   * @return The number of bytes in each block
-   * @throws IOException
-   * @throws UnresolvedLinkException if the path contains a symlink. 
-   */
-  @Idempotent
-  public long getPreferredBlockSize(String filename) 
-      throws IOException, UnresolvedLinkException;
-
-  /**
-   * Enter, leave or get safe mode.
-   * <p>
-   * Safe mode is a name node state when it
-   * <ol><li>does not accept changes to name space (read-only), and</li>
-   * <li>does not replicate or delete blocks.</li></ol>
-   * 
-   * <p>
-   * Safe mode is entered automatically at name node startup.
-   * Safe mode can also be entered manually using
-   * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean) setSafeMode(SafeModeAction.SAFEMODE_ENTER,false)}.
-   * <p>
-   * At startup the name node accepts data node reports collecting
-   * information about block locations.
-   * In order to leave safe mode it needs to collect a configurable
-   * percentage called threshold of blocks, which satisfy the minimal 
-   * replication condition.
-   * The minimal replication condition is that each block must have at least
-   * <tt>dfs.namenode.replication.min</tt> replicas.
-   * When the threshold is reached the name node extends safe mode
-   * for a configurable amount of time
-   * to let the remaining data nodes to check in before it
-   * will start replicating missing blocks.
-   * Then the name node leaves safe mode.
-   * <p>
-   * If safe mode is turned on manually using
-   * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean) setSafeMode(SafeModeAction.SAFEMODE_ENTER,false)}
-   * then the name node stays in safe mode until it is manually turned off
-   * using {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean) setSafeMode(SafeModeAction.SAFEMODE_LEAVE,false)}.
-   * Current state of the name node can be verified using
-   * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean) setSafeMode(SafeModeAction.SAFEMODE_GET,false)}
-   * <h4>Configuration parameters:</h4>
-   * <tt>dfs.safemode.threshold.pct</tt> is the threshold parameter.<br>
-   * <tt>dfs.safemode.extension</tt> is the safe mode extension parameter.<br>
-   * <tt>dfs.namenode.replication.min</tt> is the minimal replication parameter.
-   * 
-   * <h4>Special cases:</h4>
-   * The name node does not enter safe mode at startup if the threshold is 
-   * set to 0 or if the name space is empty.<br>
-   * If the threshold is set to 1 then all blocks need to have at least 
-   * minimal replication.<br>
-   * If the threshold value is greater than 1 then the name node will not be 
-   * able to turn off safe mode automatically.<br>
-   * Safe mode can always be turned off manually.
-   * 
-   * @param action  <ul> <li>0 leave safe mode;</li>
-   *                <li>1 enter safe mode;</li>
-   *                <li>2 get safe mode state.</li></ul>
-   * @param isChecked If true then action will be done only in ActiveNN.
-   * 
-   * @return <ul><li>0 if the safe mode is OFF or</li> 
-   *         <li>1 if the safe mode is ON.</li></ul>
-   *                   
-   * @throws IOException
-   */
-  @Idempotent
-  public boolean setSafeMode(HdfsConstants.SafeModeAction action, boolean isChecked) 
-      throws IOException;
-
-  /**
-   * Save namespace image.
-   * <p>
-   * Saves current namespace into storage directories and reset edits log.
-   * Requires superuser privilege and safe mode.
-   *
-   * @param timeWindow NameNode does a checkpoint if the latest checkpoint was
-   *                   done beyond the given time period (in seconds).
-   * @param txGap NameNode does a checkpoint if the gap between the latest
-   *              checkpoint and the latest transaction id is greater this gap.
-   * @return whether an extra checkpoint has been done
-   *
-   * @throws IOException if image creation failed.
-   */
-  @AtMostOnce
-  public boolean saveNamespace(long timeWindow, long txGap) throws IOException;
-
-  
-  /**
-   * Roll the edit log.
-   * Requires superuser privileges.
-   * 
-   * @throws AccessControlException if the superuser privilege is violated
-   * @throws IOException if log roll fails
-   * @return the txid of the new segment
-   */
-  @Idempotent
-  public long rollEdits() throws AccessControlException, IOException;
-
-  /**
-   * Enable/Disable restore failed storage.
-   * <p>
-   * sets flag to enable restore of failed storage replicas
-   * 
-   * @throws AccessControlException if the superuser privilege is violated.
-   */
-  @Idempotent
-  public boolean restoreFailedStorage(String arg) 
-      throws AccessControlException, IOException;
-
-  /**
-   * Tells the namenode to reread the hosts and exclude files. 
-   * @throws IOException
-   */
-  @Idempotent
-  public void refreshNodes() throws IOException;
-
-  /**
-   * Finalize previous upgrade.
-   * Remove file system state saved during the upgrade.
-   * The upgrade will become irreversible.
-   * 
-   * @throws IOException
-   */
-  @Idempotent
-  public void finalizeUpgrade() throws IOException;
-
-  /**
-   * Rolling upgrade operations.
-   * @param action either query, prepare or finalize.
-   * @return rolling upgrade information.
-   */
-  @Idempotent
-  public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
-      throws IOException;
-
-  /**
-   * @return CorruptFileBlocks, containing a list of corrupt files (with
-   *         duplicates if there is more than one corrupt block in a file)
-   *         and a cookie
-   * @throws IOException
-   *
-   * Each call returns a subset of the corrupt files in the system. To obtain
-   * all corrupt files, call this method repeatedly and each time pass in the
-   * cookie returned from the previous call.
-   */
-  @Idempotent
-  public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
-      throws IOException;
-  
-  /**
-   * Dumps namenode data structures into specified file. If the file
-   * already exists, then append.
-   *
-   * @throws IOException
-   */
-  @Idempotent
-  public void metaSave(String filename) throws IOException;
-
-  /**
-   * Tell all datanodes to use a new, non-persistent bandwidth value for
-   * dfs.balance.bandwidthPerSec.
-   *
-   * @param bandwidth Blanacer bandwidth in bytes per second for this datanode.
-   * @throws IOException
-   */
-  @Idempotent
-  public void setBalancerBandwidth(long bandwidth) throws IOException;
-  
-  /**
-   * Get the file info for a specific file or directory.
-   * @param src The string representation of the path to the file
-   *
-   * @return object containing information regarding the file
-   *         or null if file not found
-   * @throws AccessControlException permission denied
-   * @throws FileNotFoundException file <code>src</code> is not found
-   * @throws UnresolvedLinkException if the path contains a symlink. 
-   * @throws IOException If an I/O error occurred        
-   */
-  @Idempotent
-  public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException;
-  
-  /**
-   * Get the close status of a file
-   * @param src The string representation of the path to the file
-   *
-   * @return return true if file is closed
-   * @throws AccessControlException permission denied
-   * @throws FileNotFoundException file <code>src</code> is not found
-   * @throws UnresolvedLinkException if the path contains a symlink.
-   * @throws IOException If an I/O error occurred     
-   */
-  @Idempotent
-  public boolean isFileClosed(String src) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException;
-  
-  /**
-   * Get the file info for a specific file or directory. If the path 
-   * refers to a symlink then the FileStatus of the symlink is returned.
-   * @param src The string representation of the path to the file
-   *
-   * @return object containing information regarding the file
-   *         or null if file not found
-   *
-   * @throws AccessControlException permission denied
-   * @throws UnresolvedLinkException if <code>src</code> contains a symlink
-   * @throws IOException If an I/O error occurred        
-   */
-  @Idempotent
-  public HdfsFileStatus getFileLinkInfo(String src)
-      throws AccessControlException, UnresolvedLinkException, IOException;
-  
-  /**
-   * Get {@link ContentSummary} rooted at the specified directory.
-   * @param path The string representation of the path
-   *
-   * @throws AccessControlException permission denied
-   * @throws FileNotFoundException file <code>path</code> is not found
-   * @throws UnresolvedLinkException if <code>path</code> contains a symlink. 
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public ContentSummary getContentSummary(String path)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * Set the quota for a directory.
-   * @param path  The string representation of the path to the directory
-   * @param namespaceQuota Limit on the number of names in the tree rooted 
-   *                       at the directory
-   * @param storagespaceQuota Limit on storage space occupied all the files under
-   *                       this directory.
-   * @param type StorageType that the space quota is intended to be set on.
-   *             It may be null when called by traditional space/namespace quota.
-   *             When type is is not null, the storagespaceQuota parameter is for
-   *             type specified and namespaceQuota must be
-   *             {@link HdfsConstants#QUOTA_DONT_SET}.
-   *
-   * <br><br>
-   *                       
-   * The quota can have three types of values : (1) 0 or more will set 
-   * the quota to that value, (2) {@link HdfsConstants#QUOTA_DONT_SET}  implies
-   * the quota will not be changed, and (3) {@link HdfsConstants#QUOTA_RESET}
-   * implies the quota will be reset. Any other value is a runtime error.
-   * 
-   * @throws AccessControlException permission denied
-   * @throws FileNotFoundException file <code>path</code> is not found
-   * @throws QuotaExceededException if the directory size 
-   *           is greater than the given quota
-   * @throws UnresolvedLinkException if the <code>path</code> contains a symlink. 
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
-      StorageType type) throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
-
-  /**
-   * Write all metadata for this file into persistent storage.
-   * The file must be currently open for writing.
-   * @param src The string representation of the path
-   * @param inodeId The inode ID, or GRANDFATHER_INODE_ID if the client is
-   *                too old to support fsync with inode IDs.
-   * @param client The string representation of the client
-   * @param lastBlockLength The length of the last block (under construction) 
-   *                        to be reported to NameNode 
-   * @throws AccessControlException permission denied
-   * @throws FileNotFoundException file <code>src</code> is not found
-   * @throws UnresolvedLinkException if <code>src</code> contains a symlink. 
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public void fsync(String src, long inodeId, String client,
-                    long lastBlockLength)
-      throws AccessControlException, FileNotFoundException, 
-      UnresolvedLinkException, IOException;
-
-  /**
-   * Sets the modification and access time of the file to the specified time.
-   * @param src The string representation of the path
-   * @param mtime The number of milliseconds since Jan 1, 1970.
-   *              Setting mtime to -1 means that modification time should not be set
-   *              by this call.
-   * @param atime The number of milliseconds since Jan 1, 1970.
-   *              Setting atime to -1 means that access time should not be set
-   *              by this call.
-   *              
-   * @throws AccessControlException permission denied
-   * @throws FileNotFoundException file <code>src</code> is not found
-   * @throws UnresolvedLinkException if <code>src</code> contains a symlink. 
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public void setTimes(String src, long mtime, long atime)
-      throws AccessControlException, FileNotFoundException, 
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
-
-  /**
-   * Create symlink to a file or directory.
-   * @param target The path of the destination that the
-   *               link points to.
-   * @param link The path of the link being created.
-   * @param dirPerm permissions to use when creating parent directories
-   * @param createParent - if true then missing parent dirs are created
-   *                       if false then parent must exist
-   *
-   * @throws AccessControlException permission denied
-   * @throws FileAlreadyExistsException If file <code>link</code> already exists
-   * @throws FileNotFoundException If parent of <code>link</code> does not exist
-   *           and <code>createParent</code> is false
-   * @throws ParentNotDirectoryException If parent of <code>link</code> is not a
-   *           directory.
-   * @throws UnresolvedLinkException if <code>link</target> contains a symlink. 
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred
-   */
-  @AtMostOnce
-  public void createSymlink(String target, String link, FsPermission dirPerm,
-      boolean createParent) throws AccessControlException,
-      FileAlreadyExistsException, FileNotFoundException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      SnapshotAccessControlException, IOException;
-
-  /**
-   * Return the target of the given symlink. If there is an intermediate
-   * symlink in the path (ie a symlink leading up to the final path component)
-   * then the given path is returned with this symlink resolved.
-   *
-   * @param path The path with a link that needs resolution.
-   * @return The path after resolving the first symbolic link in the path.
-   * @throws AccessControlException permission denied
-   * @throws FileNotFoundException If <code>path</code> does not exist
-   * @throws IOException If the given path does not refer to a symlink
-   *           or an I/O error occurred
-   */
-  @Idempotent
-  public String getLinkTarget(String path) throws AccessControlException,
-      FileNotFoundException, IOException; 
-  
-  /**
-   * Get a new generation stamp together with an access token for 
-   * a block under construction
-   * 
-   * This method is called only when a client needs to recover a failed
-   * pipeline or set up a pipeline for appending to a block.
-   * 
-   * @param block a block
-   * @param clientName the name of the client
-   * @return a located block with a new generation stamp and an access token
-   * @throws IOException if any error occurs
-   */
-  @Idempotent
-  public LocatedBlock updateBlockForPipeline(ExtendedBlock block,
-      String clientName) throws IOException;
-
-  /**
-   * Update a pipeline for a block under construction
-   * 
-   * @param clientName the name of the client
-   * @param oldBlock the old block
-   * @param newBlock the new block containing new generation stamp and length
-   * @param newNodes datanodes in the pipeline
-   * @throws IOException if any error occurs
-   */
-  @AtMostOnce
-  public void updatePipeline(String clientName, ExtendedBlock oldBlock, 
-      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
-      throws IOException;
-
-  /**
-   * Get a valid Delegation Token.
-   * 
-   * @param renewer the designated renewer for the token
-   * @return Token<DelegationTokenIdentifier>
-   * @throws IOException
-   */
-  @Idempotent
-  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer) 
-      throws IOException;
-
-  /**
-   * Renew an existing delegation token.
-   * 
-   * @param token delegation token obtained earlier
-   * @return the new expiration time
-   * @throws IOException
-   */
-  @Idempotent
-  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws IOException;
-  
-  /**
-   * Cancel an existing delegation token.
-   * 
-   * @param token delegation token
-   * @throws IOException
-   */
-  @Idempotent
-  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws IOException;
-  
-  /**
-   * @return encryption key so a client can encrypt data sent via the
-   *         DataTransferProtocol to/from DataNodes.
-   * @throws IOException
-   */
-  @Idempotent
-  public DataEncryptionKey getDataEncryptionKey() throws IOException;
-  
-  /**
-   * Create a snapshot
-   * @param snapshotRoot the path that is being snapshotted
-   * @param snapshotName name of the snapshot created
-   * @return the snapshot path.
-   * @throws IOException
-   */
-  @AtMostOnce
-  public String createSnapshot(String snapshotRoot, String snapshotName)
-      throws IOException;
-
-  /**
-   * Delete a specific snapshot of a snapshottable directory
-   * @param snapshotRoot  The snapshottable directory
-   * @param snapshotName Name of the snapshot for the snapshottable directory
-   * @throws IOException
-   */
-  @AtMostOnce
-  public void deleteSnapshot(String snapshotRoot, String snapshotName)
-      throws IOException;
-  
-  /**
-   * Rename a snapshot
-   * @param snapshotRoot the directory path where the snapshot was taken 
-   * @param snapshotOldName old name of the snapshot
-   * @param snapshotNewName new name of the snapshot
-   * @throws IOException
-   */
-  @AtMostOnce
-  public void renameSnapshot(String snapshotRoot, String snapshotOldName,
-      String snapshotNewName) throws IOException;
-  
-  /**
-   * Allow snapshot on a directory.
-   * @param snapshotRoot the directory to be snapped
-   * @throws IOException on error
-   */
-  @Idempotent
-  public void allowSnapshot(String snapshotRoot)
-      throws IOException;
-    
-  /**
-   * Disallow snapshot on a directory.
-   * @param snapshotRoot the directory to disallow snapshot
-   * @throws IOException on error
-   */
-  @Idempotent
-  public void disallowSnapshot(String snapshotRoot)
-      throws IOException;
-  
-  /**
-   * Get the difference between two snapshots, or between a snapshot and the
-   * current tree of a directory.
-   * 
-   * @param snapshotRoot
-   *          full path of the directory where snapshots are taken
-   * @param fromSnapshot
-   *          snapshot name of the from point. Null indicates the current
-   *          tree
-   * @param toSnapshot
-   *          snapshot name of the to point. Null indicates the current
-   *          tree.
-   * @return The difference report represented as a {@link SnapshotDiffReport}.
-   * @throws IOException on error
-   */
-  @Idempotent
-  public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
-      String fromSnapshot, String toSnapshot) throws IOException;
-
-  /**
-   * Add a CacheDirective to the CacheManager.
-   * 
-   * @param directive A CacheDirectiveInfo to be added
-   * @param flags {@link CacheFlag}s to use for this operation.
-   * @return A CacheDirectiveInfo associated with the added directive
-   * @throws IOException if the directive could not be added
-   */
-  @AtMostOnce
-  public long addCacheDirective(CacheDirectiveInfo directive,
-      EnumSet<CacheFlag> flags) throws IOException;
-
-  /**
-   * Modify a CacheDirective in the CacheManager.
-   * 
-   * @param flags {@link CacheFlag}s to use for this operation.
-   * @throws IOException if the directive could not be modified
-   */
-  @AtMostOnce
-  public void modifyCacheDirective(CacheDirectiveInfo directive,
-      EnumSet<CacheFlag> flags) throws IOException;
-
-  /**
-   * Remove a CacheDirectiveInfo from the CacheManager.
-   * 
-   * @param id of a CacheDirectiveInfo
-   * @throws IOException if the cache directive could not be removed
-   */
-  @AtMostOnce
-  public void removeCacheDirective(long id) throws IOException;
-
-  /**
-   * List the set of cached paths of a cache pool. Incrementally fetches results
-   * from the server.
-   * 
-   * @param prevId The last listed entry ID, or -1 if this is the first call to
-   *               listCacheDirectives.
-   * @param filter Parameters to use to filter the list results, 
-   *               or null to display all directives visible to us.
-   * @return A batch of CacheDirectiveEntry objects.
-   */
-  @Idempotent
-  public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(
-      long prevId, CacheDirectiveInfo filter) throws IOException;
-
-  /**
-   * Add a new cache pool.
-   * 
-   * @param info Description of the new cache pool
-   * @throws IOException If the request could not be completed.
-   */
-  @AtMostOnce
-  public void addCachePool(CachePoolInfo info) throws IOException;
-
-  /**
-   * Modify an existing cache pool.
-   *
-   * @param req
-   *          The request to modify a cache pool.
-   * @throws IOException 
-   *          If the request could not be completed.
-   */
-  @AtMostOnce
-  public void modifyCachePool(CachePoolInfo req) throws IOException;
-  
-  /**
-   * Remove a cache pool.
-   * 
-   * @param pool name of the cache pool to remove.
-   * @throws IOException if the cache pool did not exist, or could not be
-   *           removed.
-   */
-  @AtMostOnce
-  public void removeCachePool(String pool) throws IOException;
-
-  /**
-   * List the set of cache pools. Incrementally fetches results from the server.
-   * 
-   * @param prevPool name of the last pool listed, or the empty string if this is
-   *          the first invocation of listCachePools
-   * @return A batch of CachePoolEntry objects.
-   */
-  @Idempotent
-  public BatchedEntries<CachePoolEntry> listCachePools(String prevPool)
-      throws IOException;
-
-  /**
-   * Modifies ACL entries of files and directories.  This method can add new ACL
-   * entries or modify the permissions on existing ACL entries.  All existing
-   * ACL entries that are not specified in this call are retained without
-   * changes.  (Modifications are merged into the current ACL.)
-   */
-  @Idempotent
-  public void modifyAclEntries(String src, List<AclEntry> aclSpec)
-      throws IOException;
-
-  /**
-   * Removes ACL entries from files and directories.  Other ACL entries are
-   * retained.
-   */
-  @Idempotent
-  public void removeAclEntries(String src, List<AclEntry> aclSpec)
-      throws IOException;
-
-  /**
-   * Removes all default ACL entries from files and directories.
-   */
-  @Idempotent
-  public void removeDefaultAcl(String src) throws IOException;
-
-  /**
-   * Removes all but the base ACL entries of files and directories.  The entries
-   * for user, group, and others are retained for compatibility with permission
-   * bits.
-   */
-  @Idempotent
-  public void removeAcl(String src) throws IOException;
-
-  /**
-   * Fully replaces ACL of files and directories, discarding all existing
-   * entries.
-   */
-  @Idempotent
-  public void setAcl(String src, List<AclEntry> aclSpec) throws IOException;
-
-  /**
-   * Gets the ACLs of files and directories.
-   */
-  @Idempotent
-  public AclStatus getAclStatus(String src) throws IOException;
-  
-  /**
-   * Create an encryption zone
-   */
-  @AtMostOnce
-  public void createEncryptionZone(String src, String keyName)
-    throws IOException;
-
-  /**
-   * Get the encryption zone for a path.
-   */
-  @Idempotent
-  public EncryptionZone getEZForPath(String src)
-    throws IOException;
-
-  /**
-   * Used to implement cursor-based batched listing of {@EncryptionZone}s.
-   *
-   * @param prevId ID of the last item in the previous batch. If there is no
-   *               previous batch, a negative value can be used.
-   * @return Batch of encryption zones.
-   */
-  @Idempotent
-  public BatchedEntries<EncryptionZone> listEncryptionZones(
-      long prevId) throws IOException;
-
-  /**
-   * Set xattr of a file or directory.
-   * The name must be prefixed with the namespace followed by ".". For example,
-   * "user.attr".
-   * <p/>
-   * Refer to the HDFS extended attributes user documentation for details.
-   *
-   * @param src file or directory
-   * @param xAttr <code>XAttr</code> to set
-   * @param flag set flag
-   * @throws IOException
-   */
-  @AtMostOnce
-  public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag) 
-      throws IOException;
-  
-  /**
-   * Get xattrs of a file or directory. Values in xAttrs parameter are ignored.
-   * If xAttrs is null or empty, this is the same as getting all xattrs of the
-   * file or directory.  Only those xattrs for which the logged-in user has
-   * permissions to view are returned.
-   * <p/>
-   * Refer to the HDFS extended attributes user documentation for details.
-   *
-   * @param src file or directory
-   * @param xAttrs xAttrs to get
-   * @return List<XAttr> <code>XAttr</code> list 
-   * @throws IOException
-   */
-  @Idempotent
-  public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs) 
-      throws IOException;
-
-  /**
-   * List the xattrs names for a file or directory.
-   * Only the xattr names for which the logged in user has the permissions to
-   * access will be returned.
-   * <p/>
-   * Refer to the HDFS extended attributes user documentation for details.
-   *
-   * @param src file or directory
-   * @return List<XAttr> <code>XAttr</code> list
-   * @throws IOException
-   */
-  @Idempotent
-  public List<XAttr> listXAttrs(String src)
-      throws IOException;
-  
-  /**
-   * Remove xattr of a file or directory.Value in xAttr parameter is ignored.
-   * The name must be prefixed with the namespace followed by ".". For example,
-   * "user.attr".
-   * <p/>
-   * Refer to the HDFS extended attributes user documentation for details.
-   *
-   * @param src file or directory
-   * @param xAttr <code>XAttr</code> to remove
-   * @throws IOException
-   */
-  @AtMostOnce
-  public void removeXAttr(String src, XAttr xAttr) throws IOException;
-
-  /**
-   * Checks if the user can access a path.  The mode specifies which access
-   * checks to perform.  If the requested permissions are granted, then the
-   * method returns normally.  If access is denied, then the method throws an
-   * {@link AccessControlException}.
-   * In general, applications should avoid using this method, due to the risk of
-   * time-of-check/time-of-use race conditions.  The permissions on a file may
-   * change immediately after the access call returns.
-   *
-   * @param path Path to check
-   * @param mode type of access to check
-   * @throws AccessControlException if access is denied
-   * @throws FileNotFoundException if the path does not exist
-   * @throws IOException see specific implementation
-   */
-  @Idempotent
-  public void checkAccess(String path, FsAction mode) throws IOException;
-
-  /**
-   * Get the highest txid the NameNode knows has been written to the edit
-   * log, or -1 if the NameNode's edit log is not yet open for write. Used as
-   * the starting point for the inotify event stream.
-   */
-  @Idempotent
-  public long getCurrentEditLogTxid() throws IOException;
-
-  /**
-   * Get an ordered list of batches of events corresponding to the edit log
-   * transactions for txids equal to or greater than txid.
-   */
-  @Idempotent
-  public EventBatchList getEditsFromTxid(long txid) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8327744/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
deleted file mode 100644
index 8080bcf..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.server.namenode;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * This exception is thrown when the name node is in safe mode.
- * Client cannot modified namespace until the safe mode is off. 
- * 
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class SafeModeException extends IOException {
-  private static final long serialVersionUID = 1L;
-  public SafeModeException(String msg) {
-    super(msg);
-  }
-}
\ No newline at end of file


[40/50] [abbrv] hadoop git commit: HDFS-8582. Support getting a list of reconfigurable config properties and do not generate spurious reconfig warnings (Lei (Eddy) Xu via Colin P. McCabe)

Posted by ar...@apache.org.
HDFS-8582. Support getting a list of reconfigurable config properties and do not generate spurious reconfig warnings (Lei (Eddy) Xu via Colin P.  McCabe)


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

Branch: refs/heads/HDFS-7240
Commit: d112d183242f447d4b742139f3a4ea531a997f45
Parents: 55278c0
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Fri Jun 19 19:14:05 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Fri Jun 19 19:14:05 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/conf/ReconfigurableBase.java  |  14 ++-
 .../apache/hadoop/conf/TestReconfiguration.java |  12 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   4 +
 .../hdfs/protocol/ClientDatanodeProtocol.java   |   5 +
 ...tDatanodeProtocolServerSideTranslatorPB.java |  19 +++
 .../ClientDatanodeProtocolTranslatorPB.java     |  19 ++-
 .../hadoop/hdfs/server/datanode/DataNode.java   |  22 +++-
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |  59 ++++++++--
 .../src/main/proto/ClientDatanodeProtocol.proto |  12 ++
 .../apache/hadoop/hdfs/tools/TestDFSAdmin.java  | 117 ++++++++++++++-----
 10 files changed, 233 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d112d183/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurableBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurableBase.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurableBase.java
index 3664549..e50b85a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurableBase.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurableBase.java
@@ -88,6 +88,11 @@ public abstract class ReconfigurableBase
     reconfigurationUtil = Preconditions.checkNotNull(ru);
   }
 
+  /**
+   * Create a new configuration.
+   */
+  protected abstract Configuration getNewConf();
+
   @VisibleForTesting
   public Collection<PropertyChange> getChangedProperties(
       Configuration newConf, Configuration oldConf) {
@@ -108,17 +113,16 @@ public abstract class ReconfigurableBase
     public void run() {
       LOG.info("Starting reconfiguration task.");
       Configuration oldConf = this.parent.getConf();
-      Configuration newConf = new Configuration();
+      Configuration newConf = this.parent.getNewConf();
       Collection<PropertyChange> changes =
           this.parent.getChangedProperties(newConf, oldConf);
       Map<PropertyChange, Optional<String>> results = Maps.newHashMap();
       for (PropertyChange change : changes) {
         String errorMessage = null;
         if (!this.parent.isPropertyReconfigurable(change.prop)) {
-          errorMessage = "Property " + change.prop +
-              " is not reconfigurable";
-          LOG.info(errorMessage);
-          results.put(change, Optional.of(errorMessage));
+          LOG.info(String.format(
+              "Property %s is not configurable: old value: %s, new value: %s",
+              change.prop, change.oldVal, change.newVal));
           continue;
         }
         LOG.info("Change property: " + change.prop + " from \""

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d112d183/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestReconfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestReconfiguration.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestReconfiguration.java
index 238ebda..5f0516ae 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestReconfiguration.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestReconfiguration.java
@@ -118,6 +118,11 @@ public class TestReconfiguration {
       super(conf);
     }
 
+    @Override
+    protected Configuration getNewConf() {
+      return new Configuration();
+    }
+
     @Override 
     public Collection<String> getReconfigurableProperties() {
       return Arrays.asList(PROP1, PROP2, PROP4);
@@ -336,6 +341,11 @@ public class TestReconfiguration {
       super(conf);
     }
 
+    @Override
+    protected Configuration getNewConf() {
+      return new Configuration();
+    }
+
     final CountDownLatch latch = new CountDownLatch(1);
 
     @Override
@@ -397,7 +407,7 @@ public class TestReconfiguration {
 
     waitAsyncReconfigureTaskFinish(dummy);
     ReconfigurationTaskStatus status = dummy.getReconfigurationTaskStatus();
-    assertEquals(3, status.getStatus().size());
+    assertEquals(2, status.getStatus().size());
     for (Map.Entry<PropertyChange, Optional<String>> result :
         status.getStatus().entrySet()) {
       PropertyChange change = result.getKey();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d112d183/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index a93a214..527f6ca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -659,6 +659,10 @@ Release 2.8.0 - UNRELEASED
     HDFS-8605. Merge Refactor of DFSOutputStream from HDFS-7285 branch.
     (vinayakumarb via wang)
 
+    HDFS-8582. Support getting a list of reconfigurable config properties and
+    do not generate spurious reconfig warnings (Lei (Eddy) Xu via Colin P.
+    McCabe)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d112d183/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
index 969f1a3..75f8907 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
@@ -159,6 +159,11 @@ public interface ClientDatanodeProtocol {
   ReconfigurationTaskStatus getReconfigurationStatus() throws IOException;
 
   /**
+   * Get a list of allowed properties for reconfiguration.
+   */
+  List<String> listReconfigurableProperties() throws IOException;
+
+  /**
    * Trigger a new block report.
    */
   void triggerBlockReport(BlockReportOptions options)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d112d183/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
index 5c2c4a7..2bc6a18 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
@@ -44,6 +44,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdf
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdfsBlockLocationsResponseProto.Builder;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ListReconfigurablePropertiesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ListReconfigurablePropertiesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ShutdownDatanodeRequestProto;
@@ -209,6 +211,23 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
+  public ListReconfigurablePropertiesResponseProto listReconfigurableProperties(
+        RpcController controller,
+        ListReconfigurablePropertiesRequestProto request)
+      throws ServiceException {
+    ListReconfigurablePropertiesResponseProto.Builder builder =
+        ListReconfigurablePropertiesResponseProto.newBuilder();
+    try {
+      for (String name : impl.listReconfigurableProperties()) {
+        builder.addName(name);
+      }
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return builder.build();
+  }
+
+  @Override
   public GetReconfigurationStatusResponseProto getReconfigurationStatus(
       RpcController unused, GetReconfigurationStatusRequestProto request)
       throws ServiceException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d112d183/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
index f1a1b24..9d6375b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
@@ -52,6 +52,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetDat
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdfsBlockLocationsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetHdfsBlockLocationsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ListReconfigurablePropertiesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ListReconfigurablePropertiesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReconfigurationStatusRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReconfigurationStatusResponseProto;
@@ -59,7 +61,6 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetRec
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ShutdownDatanodeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.StartReconfigurationRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.TriggerBlockReportRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.TriggerBlockReportResponseProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
@@ -101,6 +102,9 @@ public class ClientDatanodeProtocolTranslatorPB implements
       GetReconfigurationStatusRequestProto.newBuilder().build();
   private final static StartReconfigurationRequestProto VOID_START_RECONFIG =
       StartReconfigurationRequestProto.newBuilder().build();
+  private static final ListReconfigurablePropertiesRequestProto
+      VOID_LIST_RECONFIGURABLE_PROPERTIES =
+      ListReconfigurablePropertiesRequestProto.newBuilder().build();
 
   public ClientDatanodeProtocolTranslatorPB(DatanodeID datanodeid,
       Configuration conf, int socketTimeout, boolean connectToDnViaHostname,
@@ -338,6 +342,19 @@ public class ClientDatanodeProtocolTranslatorPB implements
   }
 
   @Override
+  public List<String> listReconfigurableProperties()
+      throws IOException {
+    ListReconfigurablePropertiesResponseProto response;
+    try {
+      response = rpcProxy.listReconfigurableProperties(NULL_CONTROLLER,
+          VOID_LIST_RECONFIGURABLE_PROPERTIES);
+      return response.getNameList();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
   public void triggerBlockReport(BlockReportOptions options)
       throws IOException {
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d112d183/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 74acf81..6c8cf2b 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
@@ -272,6 +272,11 @@ public class DataNode extends ReconfigurableBase
 
   static final int CURRENT_BLOCK_FORMAT_VERSION = 1;
 
+  /** A list of property that are reconfigurable at runtime. */
+  private static final List<String> RECONFIGURABLE_PROPERTIES =
+      Collections.unmodifiableList(
+          Arrays.asList(DFS_DATANODE_DATA_DIR_KEY));
+
   /**
    * Use {@link NetUtils#createSocketAddr(String)} instead.
    */
@@ -449,6 +454,11 @@ public class DataNode extends ReconfigurableBase
             });
   }
 
+  @Override  // ReconfigurableBase
+  protected Configuration getNewConf() {
+    return new HdfsConfiguration();
+  }
+
   @Override
   public void reconfigurePropertyImpl(String property, String newVal)
       throws ReconfigurationException {
@@ -469,11 +479,9 @@ public class DataNode extends ReconfigurableBase
   /**
    * Get a list of the keys of the re-configurable properties in configuration.
    */
-  @Override
+  @Override // Reconfigurable
   public Collection<String> getReconfigurableProperties() {
-    List<String> reconfigurable =
-        Collections.unmodifiableList(Arrays.asList(DFS_DATANODE_DATA_DIR_KEY));
-    return reconfigurable;
+    return RECONFIGURABLE_PROPERTIES;
   }
 
   /**
@@ -3052,6 +3060,12 @@ public class DataNode extends ReconfigurableBase
   }
 
   @Override // ClientDatanodeProtocol
+  public List<String> listReconfigurableProperties()
+      throws IOException {
+    return RECONFIGURABLE_PROPERTIES;
+  }
+
+  @Override // ClientDatanodeProtocol
   public void triggerBlockReport(BlockReportOptions options)
       throws IOException {
     checkSuperuserPrivilege();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d112d183/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 b978189..4640bb3 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
@@ -407,7 +407,7 @@ public class DFSAdmin extends FsShell {
     "\t[-refreshSuperUserGroupsConfiguration]\n" +
     "\t[-refreshCallQueue]\n" +
     "\t[-refresh <host:ipc_port> <key> [arg1..argn]\n" +
-    "\t[-reconfig <datanode|...> <host:ipc_port> <start|status>]\n" +
+    "\t[-reconfig <datanode|...> <host:ipc_port> <start|status|properties>]\n" +
     "\t[-printTopology]\n" +
     "\t[-refreshNamenodes datanode_host:ipc_port]\n"+
     "\t[-deleteBlockPool datanode_host:ipc_port blockpoolId [force]]\n"+
@@ -979,8 +979,9 @@ public class DFSAdmin extends FsShell {
 
     String refreshCallQueue = "-refreshCallQueue: Reload the call queue from config\n";
 
-    String reconfig = "-reconfig <datanode|...> <host:ipc_port> <start|status>:\n" +
-        "\tStarts reconfiguration or gets the status of an ongoing reconfiguration.\n" +
+    String reconfig = "-reconfig <datanode|...> <host:ipc_port> <start|status|properties>:\n" +
+        "\tStarts or gets the status of a reconfiguration operation, \n" +
+        "\tor gets a list of reconfigurable properties.\n" +
         "\tThe second parameter specifies the node type.\n" +
         "\tCurrently, only reloading DataNode's configuration is supported.\n";
 
@@ -1439,6 +1440,9 @@ public class DFSAdmin extends FsShell {
       return startReconfiguration(nodeType, address);
     } else if ("status".equals(op)) {
       return getReconfigurationStatus(nodeType, address, System.out, System.err);
+    } else if ("properties".equals(op)) {
+      return getReconfigurableProperties(
+          nodeType, address, System.out, System.err);
     }
     System.err.println("Unknown operation: " + op);
     return -1;
@@ -1476,18 +1480,24 @@ public class DFSAdmin extends FsShell {
 
         out.println(" and finished at " +
             new Date(status.getEndTime()).toString() + ".");
+        if (status.getStatus() == null) {
+          // Nothing to report.
+          return 0;
+        }
         for (Map.Entry<PropertyChange, Optional<String>> result :
             status.getStatus().entrySet()) {
           if (!result.getValue().isPresent()) {
-            out.print("SUCCESS: ");
+            out.printf(
+                "SUCCESS: Changed property %s%n\tFrom: \"%s\"%n\tTo: \"%s\"%n",
+                result.getKey().prop, result.getKey().oldVal,
+                result.getKey().newVal);
           } else {
-            out.print("FAILED: ");
-          }
-          out.printf("Change property %s%n\tFrom: \"%s\"%n\tTo: \"%s\"%n",
-              result.getKey().prop, result.getKey().oldVal,
-              result.getKey().newVal);
-          if (result.getValue().isPresent()) {
-            out.println("\tError: " + result.getValue().get() + ".");
+            final String errorMsg = result.getValue().get();
+            out.printf(
+                  "FAILED: Change property %s%n\tFrom: \"%s\"%n\tTo: \"%s\"%n",
+                  result.getKey().prop, result.getKey().oldVal,
+                  result.getKey().newVal);
+            out.println("\tError: " + errorMsg + ".");
           }
         }
       } catch (IOException e) {
@@ -1495,7 +1505,32 @@ public class DFSAdmin extends FsShell {
         return 1;
       }
     } else {
-      err.println("Node type " + nodeType + " does not support reconfiguration.");
+      err.println("Node type " + nodeType +
+          " does not support reconfiguration.");
+      return 1;
+    }
+    return 0;
+  }
+
+  int getReconfigurableProperties(String nodeType, String address,
+      PrintStream out, PrintStream err) throws IOException {
+    if ("datanode".equals(nodeType)) {
+      ClientDatanodeProtocol dnProxy = getDataNodeProxy(address);
+      try {
+        List<String> properties =
+            dnProxy.listReconfigurableProperties();
+        out.println(
+            "Configuration properties that are allowed to be reconfigured:");
+        for (String name : properties) {
+          out.println(name);
+        }
+      } catch (IOException e) {
+        err.println("DataNode reconfiguration: " + e + ".");
+        return 1;
+      }
+    } else {
+      err.println("Node type " + nodeType +
+          " does not support reconfiguration.");
       return 1;
     }
     return 0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d112d183/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto
index 48f6dd1..e0d1f5f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto
@@ -180,6 +180,14 @@ message GetReconfigurationStatusResponseProto {
   repeated GetReconfigurationStatusConfigChangeProto changes = 3;
 }
 
+message ListReconfigurablePropertiesRequestProto {
+}
+
+/** Query the reconfigurable properties on DataNode. */
+message ListReconfigurablePropertiesResponseProto {
+  repeated string name = 1;
+}
+
 /**
  * Protocol used from client to the Datanode.
  * See the request and response for details of rpc call.
@@ -230,6 +238,10 @@ service ClientDatanodeProtocolService {
   rpc startReconfiguration(StartReconfigurationRequestProto)
       returns(StartReconfigurationResponseProto);
 
+  rpc listReconfigurableProperties(
+      ListReconfigurablePropertiesRequestProto)
+      returns(ListReconfigurablePropertiesResponseProto);
+
   rpc triggerBlockReport(TriggerBlockReportRequestProto)
       returns(TriggerBlockReportResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d112d183/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 9758955..a2b5638 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
@@ -22,6 +22,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.ReconfigurationUtil;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -42,6 +43,7 @@ import static org.hamcrest.CoreMatchers.allOf;
 import static org.hamcrest.CoreMatchers.anyOf;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.not;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.hamcrest.CoreMatchers.containsString;
@@ -50,18 +52,17 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 public class TestDFSAdmin {
+  private Configuration conf = null;
   private MiniDFSCluster cluster;
   private DFSAdmin admin;
   private DataNode datanode;
 
   @Before
   public void setUp() throws Exception {
-    Configuration conf = new Configuration();
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
-    cluster.waitActive();
+    conf = new Configuration();
+    restartCluster();
 
     admin = new DFSAdmin();
-    datanode = cluster.getDataNodes().get(0);
   }
 
   @After
@@ -72,6 +73,15 @@ public class TestDFSAdmin {
     }
   }
 
+  private void restartCluster() throws IOException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster.waitActive();
+    datanode = cluster.getDataNodes().get(0);
+  }
+
   private List<String> getReconfigureStatus(String nodeType, String address)
       throws IOException {
     ByteArrayOutputStream bufOut = new ByteArrayOutputStream();
@@ -87,16 +97,26 @@ public class TestDFSAdmin {
     return outputs;
   }
 
-  @Test(timeout = 30000)
-  public void testGetReconfigureStatus()
+  /**
+   * Test reconfiguration and check the status outputs.
+   * @param expectedSuccuss set true if the reconfiguration task should success.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void testGetReconfigurationStatus(boolean expectedSuccuss)
       throws IOException, InterruptedException {
     ReconfigurationUtil ru = mock(ReconfigurationUtil.class);
     datanode.setReconfigurationUtil(ru);
 
     List<ReconfigurationUtil.PropertyChange> changes =
-        new ArrayList<ReconfigurationUtil.PropertyChange>();
+        new ArrayList<>();
     File newDir = new File(cluster.getDataDirectory(), "data_new");
-    newDir.mkdirs();
+    if (expectedSuccuss) {
+      newDir.mkdirs();
+    } else {
+      // Inject failure.
+      newDir.createNewFile();
+    }
     changes.add(new ReconfigurationUtil.PropertyChange(
         DFS_DATANODE_DATA_DIR_KEY, newDir.toString(),
         datanode.getConf().get(DFS_DATANODE_DATA_DIR_KEY)));
@@ -121,31 +141,74 @@ public class TestDFSAdmin {
       Thread.sleep(100);
     }
     assertTrue(count > 0);
-    assertThat(outputs.size(), is(8));  // 3 (SUCCESS) + 4 (FAILED)
+    if (expectedSuccuss) {
+      assertThat(outputs.size(), is(4));
+    } else {
+      assertThat(outputs.size(), is(6));
+    }
 
     List<StorageLocation> locations = DataNode.getStorageLocations(
         datanode.getConf());
-    assertThat(locations.size(), is(1));
-    assertThat(locations.get(0).getFile(), is(newDir));
-    // Verify the directory is appropriately formatted.
-    assertTrue(new File(newDir, Storage.STORAGE_DIR_CURRENT).isDirectory());
-
-    int successOffset = outputs.get(1).startsWith("SUCCESS:") ? 1 : 5;
-    int failedOffset = outputs.get(1).startsWith("FAILED:") ? 1: 4;
-    assertThat(outputs.get(successOffset),
-        containsString("Change property " + DFS_DATANODE_DATA_DIR_KEY));
-    assertThat(outputs.get(successOffset + 1),
+    if (expectedSuccuss) {
+      assertThat(locations.size(), is(1));
+      assertThat(locations.get(0).getFile(), is(newDir));
+      // Verify the directory is appropriately formatted.
+      assertTrue(new File(newDir, Storage.STORAGE_DIR_CURRENT).isDirectory());
+    } else {
+      assertTrue(locations.isEmpty());
+    }
+
+    int offset = 1;
+    if (expectedSuccuss) {
+      assertThat(outputs.get(offset),
+          containsString("SUCCESS: Changed property " +
+              DFS_DATANODE_DATA_DIR_KEY));
+    } else {
+      assertThat(outputs.get(offset),
+          containsString("FAILED: Change property " +
+              DFS_DATANODE_DATA_DIR_KEY));
+    }
+    assertThat(outputs.get(offset + 1),
         is(allOf(containsString("From:"), containsString("data1"),
             containsString("data2"))));
-    assertThat(outputs.get(successOffset + 2),
+    assertThat(outputs.get(offset + 2),
         is(not(anyOf(containsString("data1"), containsString("data2")))));
-    assertThat(outputs.get(successOffset + 2),
+    assertThat(outputs.get(offset + 2),
         is(allOf(containsString("To"), containsString("data_new"))));
-    assertThat(outputs.get(failedOffset),
-        containsString("Change property randomKey"));
-    assertThat(outputs.get(failedOffset + 1),
-        containsString("From: \"old456\""));
-    assertThat(outputs.get(failedOffset + 2),
-        containsString("To: \"new123\""));
+  }
+
+  @Test(timeout = 30000)
+  public void testGetReconfigurationStatus()
+      throws IOException, InterruptedException {
+    testGetReconfigurationStatus(true);
+    restartCluster();
+    testGetReconfigurationStatus(false);
+  }
+
+  private List<String> getReconfigurationAllowedProperties(
+      String nodeType, String address)
+      throws IOException {
+    ByteArrayOutputStream bufOut = new ByteArrayOutputStream();
+    PrintStream out = new PrintStream(bufOut);
+    ByteArrayOutputStream bufErr = new ByteArrayOutputStream();
+    PrintStream err = new PrintStream(bufErr);
+    admin.getReconfigurableProperties(nodeType, address, out, err);
+    Scanner scanner = new Scanner(bufOut.toString());
+    List<String> outputs = Lists.newArrayList();
+    while (scanner.hasNextLine()) {
+      outputs.add(scanner.nextLine());
+    }
+    return outputs;
+  }
+
+  @Test(timeout = 30000)
+  public void testGetReconfigAllowedProperties() throws IOException {
+    final int port = datanode.getIpcPort();
+    final String address = "localhost:" + port;
+    List<String> outputs =
+        getReconfigurationAllowedProperties("datanode", address);
+    assertEquals(2, outputs.size());
+    assertEquals(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
+        outputs.get(1));
   }
 }
\ No newline at end of file


[14/50] [abbrv] hadoop git commit: HDFS-8551. Fix hdfs datanode CLI usage message. Contributed by Brahma Reddy Battula.

Posted by ar...@apache.org.
HDFS-8551. Fix hdfs datanode CLI usage message. 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/9cd5ad9d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9cd5ad9d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9cd5ad9d

Branch: refs/heads/HDFS-7240
Commit: 9cd5ad9d84e46295249877ade50cd49c34b9bf12
Parents: e27d5a1
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Tue Jun 16 14:12:45 2015 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Tue Jun 16 14:12:45 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                      | 3 +++
 .../java/org/apache/hadoop/hdfs/server/datanode/DataNode.java    | 4 +++-
 2 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9cd5ad9d/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index c8c36e6..2d4c062 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -903,6 +903,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8548. Minicluster throws NPE on shutdown.
     (surendra singh lilhore via xyao)
 
+    HDFS-8551. Fix hdfs datanode CLI usage message.
+    (Brahma Reddy Battula via xyao)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9cd5ad9d/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 3bd131e..74acf81 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
@@ -262,9 +262,11 @@ public class DataNode extends ReconfigurableBase
     LogFactory.getLog(DataNode.class.getName() + ".clienttrace");
   
   private static final String USAGE =
-      "Usage: java DataNode [-regular | -rollback]\n" +
+      "Usage: hdfs datanode [-regular | -rollback | -rollingupgrade rollback" +
+      " ]\n" +
       "    -regular                 : Normal DataNode startup (default).\n" +
       "    -rollback                : Rollback a standard or rolling upgrade.\n" +
+      "    -rollingupgrade rollback : Rollback a rolling upgrade operation.\n" +
       "  Refer to HDFS documentation for the difference between standard\n" +
       "  and rolling upgrades.";
 


[02/50] [abbrv] hadoop git commit: HDFS-8540. Mover should exit with NO_MOVE_BLOCK if no block can be moved. Contributed by surendra singh lilhore

Posted by ar...@apache.org.
HDFS-8540.  Mover should exit with NO_MOVE_BLOCK if no block can be moved.  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/321940cf
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/321940cf
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/321940cf

Branch: refs/heads/HDFS-7240
Commit: 321940cf19375febe9660e96d905360cfcc15f5f
Parents: 2cb09e9
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jun 15 16:26:53 2015 -0700
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Mon Jun 15 16:26:53 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../apache/hadoop/hdfs/server/mover/Mover.java  | 95 ++++++++++++++------
 .../hadoop/hdfs/server/mover/TestMover.java     | 29 ++++++
 .../hdfs/server/mover/TestStorageMover.java     | 18 ++--
 4 files changed, 107 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/321940cf/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 21acf98..584d94d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -917,6 +917,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8521. Add VisibleForTesting annotation to
     BlockPoolSlice#selectReplicaToDelete. (cmccabe)
 
+    HDFS-8540.  Mover should exit with NO_MOVE_BLOCK if no block can be moved.
+    (surendra singh lilhore via szetszwo)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/321940cf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
index 8715ce4..344b9fc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
@@ -27,7 +27,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.BlockStoragePolicySpi;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -163,8 +162,7 @@ public class Mover {
   private ExitStatus run() {
     try {
       init();
-      boolean hasRemaining = new Processor().processNamespace();
-      return hasRemaining ? ExitStatus.IN_PROGRESS : ExitStatus.SUCCESS;
+      return new Processor().processNamespace().getExitStatus();
     } catch (IllegalArgumentException e) {
       System.out.println(e + ".  Exiting ...");
       return ExitStatus.ILLEGAL_ARGUMENTS;
@@ -262,11 +260,11 @@ public class Mover {
      * @return whether there is still remaining migration work for the next
      *         round
      */
-    private boolean processNamespace() throws IOException {
+    private Result processNamespace() throws IOException {
       getSnapshottableDirs();
-      boolean hasRemaining = false;
+      Result result = new Result();
       for (Path target : targetPaths) {
-        hasRemaining |= processPath(target.toUri().getPath());
+        processPath(target.toUri().getPath(), result);
       }
       // wait for pending move to finish and retry the failed migration
       boolean hasFailed = Dispatcher.waitForMoveCompletion(storages.targets
@@ -282,16 +280,15 @@ public class Mover {
         // Reset retry count if no failure.
         retryCount.set(0);
       }
-      hasRemaining |= hasFailed;
-      return hasRemaining;
+      result.updateHasRemaining(hasFailed);
+      return result;
     }
 
     /**
      * @return whether there is still remaing migration work for the next
      *         round
      */
-    private boolean processPath(String fullPath) {
-      boolean hasRemaining = false;
+    private void processPath(String fullPath, Result result) {
       for (byte[] lastReturnedName = HdfsFileStatus.EMPTY_NAME;;) {
         final DirectoryListing children;
         try {
@@ -299,73 +296,71 @@ public class Mover {
         } catch(IOException e) {
           LOG.warn("Failed to list directory " + fullPath
               + ". Ignore the directory and continue.", e);
-          return hasRemaining;
+          return;
         }
         if (children == null) {
-          return hasRemaining;
+          return;
         }
         for (HdfsFileStatus child : children.getPartialListing()) {
-          hasRemaining |= processRecursively(fullPath, child);
+          processRecursively(fullPath, child, result);
         }
         if (children.hasMore()) {
           lastReturnedName = children.getLastName();
         } else {
-          return hasRemaining;
+          return;
         }
       }
     }
 
     /** @return whether the migration requires next round */
-    private boolean processRecursively(String parent, HdfsFileStatus status) {
+    private void processRecursively(String parent, HdfsFileStatus status,
+        Result result) {
       String fullPath = status.getFullName(parent);
-      boolean hasRemaining = false;
       if (status.isDir()) {
         if (!fullPath.endsWith(Path.SEPARATOR)) {
           fullPath = fullPath + Path.SEPARATOR;
         }
 
-        hasRemaining = processPath(fullPath);
+        processPath(fullPath, result);
         // process snapshots if this is a snapshottable directory
         if (snapshottableDirs.contains(fullPath)) {
           final String dirSnapshot = fullPath + HdfsConstants.DOT_SNAPSHOT_DIR;
-          hasRemaining |= processPath(dirSnapshot);
+          processPath(dirSnapshot, result);
         }
       } else if (!status.isSymlink()) { // file
         try {
           if (!isSnapshotPathInCurrent(fullPath)) {
             // the full path is a snapshot path but it is also included in the
             // current directory tree, thus ignore it.
-            hasRemaining = processFile(fullPath, (HdfsLocatedFileStatus)status);
+            processFile(fullPath, (HdfsLocatedFileStatus) status, result);
           }
         } catch (IOException e) {
           LOG.warn("Failed to check the status of " + parent
               + ". Ignore it and continue.", e);
-          return false;
         }
       }
-      return hasRemaining;
     }
 
     /** @return true if it is necessary to run another round of migration */
-    private boolean processFile(String fullPath, HdfsLocatedFileStatus status) {
+    private void processFile(String fullPath, HdfsLocatedFileStatus status,
+        Result result) {
       final byte policyId = status.getStoragePolicy();
       // currently we ignore files with unspecified storage policy
       if (policyId == HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
-        return false;
+        return;
       }
       final BlockStoragePolicy policy = blockStoragePolicies[policyId];
       if (policy == null) {
         LOG.warn("Failed to get the storage policy of file " + fullPath);
-        return false;
+        return;
       }
       final List<StorageType> types = policy.chooseStorageTypes(
           status.getReplication());
 
       final LocatedBlocks locatedBlocks = status.getBlockLocations();
-      boolean hasRemaining = false;
       final boolean lastBlkComplete = locatedBlocks.isLastBlockComplete();
       List<LocatedBlock> lbs = locatedBlocks.getLocatedBlocks();
-      for(int i = 0; i < lbs.size(); i++) {
+      for (int i = 0; i < lbs.size(); i++) {
         if (i == lbs.size() - 1 && !lastBlkComplete) {
           // last block is incomplete, skip it
           continue;
@@ -375,12 +370,15 @@ public class Mover {
             lb.getStorageTypes());
         if (!diff.removeOverlap(true)) {
           if (scheduleMoves4Block(diff, lb)) {
-            hasRemaining |= (diff.existing.size() > 1 &&
-                diff.expected.size() > 1);
+            result.updateHasRemaining(diff.existing.size() > 1
+                && diff.expected.size() > 1);
+            // One block scheduled successfully, set noBlockMoved to false
+            result.setNoBlockMoved(false);
+          } else {
+            result.updateHasRemaining(true);
           }
         }
       }
-      return hasRemaining;
     }
 
     boolean scheduleMoves4Block(StorageTypeDiff diff, LocatedBlock lb) {
@@ -711,6 +709,45 @@ public class Mover {
     }
   }
 
+  private static class Result {
+
+    private boolean hasRemaining;
+    private boolean noBlockMoved;
+
+    Result() {
+      hasRemaining = false;
+      noBlockMoved = true;
+    }
+
+    boolean isHasRemaining() {
+      return hasRemaining;
+    }
+
+    boolean isNoBlockMoved() {
+      return noBlockMoved;
+    }
+
+    void updateHasRemaining(boolean hasRemaining) {
+      this.hasRemaining |= hasRemaining;
+    }
+
+    void setNoBlockMoved(boolean noBlockMoved) {
+      this.noBlockMoved = noBlockMoved;
+    }
+
+    /**
+     * @return SUCCESS if all moves are success and there is no remaining move.
+     *         Return NO_MOVE_BLOCK if there moves available but all the moves
+     *         cannot be scheduled. Otherwise, return IN_PROGRESS since there
+     *         must be some remaining moves.
+     */
+    ExitStatus getExitStatus() {
+      return !isHasRemaining() ? ExitStatus.SUCCESS
+          : isNoBlockMoved() ? ExitStatus.NO_MOVE_BLOCK
+              : ExitStatus.IN_PROGRESS;
+    }
+
+  }
   /**
    * Run a Mover in command line.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/321940cf/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
index f4bedab..49e2b23 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
@@ -328,6 +328,35 @@ public class TestMover {
     }
   }
 
+  @Test(timeout = 300000)
+  public void testMoveWhenStoragePolicyNotSatisfying() throws Exception {
+    // HDFS-8147
+    final Configuration conf = new HdfsConfiguration();
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(3)
+        .storageTypes(
+            new StorageType[][] { { StorageType.DISK }, { StorageType.DISK },
+                { StorageType.DISK } }).build();
+    try {
+      cluster.waitActive();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      final String file = "/testMoveWhenStoragePolicyNotSatisfying";
+      // write to DISK
+      final FSDataOutputStream out = dfs.create(new Path(file));
+      out.writeChars("testMoveWhenStoragePolicyNotSatisfying");
+      out.close();
+
+      // move to ARCHIVE
+      dfs.setStoragePolicy(new Path(file), "COLD");
+      int rc = ToolRunner.run(conf, new Mover.Cli(),
+          new String[] { "-p", file.toString() });
+      int exitcode = ExitStatus.NO_MOVE_BLOCK.getExitCode();
+      Assert.assertEquals("Exit code should be " + exitcode, exitcode, rc);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
   @Test
   public void testMoverFailedRetry() throws Exception {
     // HDFS-8147

http://git-wip-us.apache.org/repos/asf/hadoop/blob/321940cf/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
index d8b40d4..3095f30 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
@@ -219,7 +219,7 @@ public class TestStorageMover {
         verify(true);
 
         setStoragePolicy();
-        migrate();
+        migrate(ExitStatus.SUCCESS);
         verify(true);
       } finally {
         if (shutdown) {
@@ -250,8 +250,8 @@ public class TestStorageMover {
     /**
      * Run the migration tool.
      */
-    void migrate() throws Exception {
-      runMover();
+    void migrate(ExitStatus expectedExitCode) throws Exception {
+      runMover(expectedExitCode);
       Thread.sleep(5000); // let the NN finish deletion
     }
 
@@ -267,14 +267,14 @@ public class TestStorageMover {
       }
     }
 
-    private void runMover() throws Exception {
+    private void runMover(ExitStatus expectedExitCode) throws Exception {
       Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
       Map<URI, List<Path>> nnMap = Maps.newHashMap();
       for (URI nn : namenodes) {
         nnMap.put(nn, null);
       }
       int result = Mover.run(nnMap, conf);
-      Assert.assertEquals(ExitStatus.SUCCESS.getExitCode(), result);
+      Assert.assertEquals(expectedExitCode.getExitCode(), result);
     }
 
     private void verifyNamespace() throws Exception {
@@ -555,7 +555,7 @@ public class TestStorageMover {
     try {
       banner("start data migration");
       test.setStoragePolicy(); // set /foo to COLD
-      test.migrate();
+      test.migrate(ExitStatus.SUCCESS);
 
       // make sure the under construction block has not been migrated
       LocatedBlocks lbs = test.dfs.getClient().getLocatedBlocks(
@@ -605,7 +605,7 @@ public class TestStorageMover {
     try {
       test.runBasicTest(false);
       pathPolicyMap.moveAround(test.dfs);
-      test.migrate();
+      test.migrate(ExitStatus.SUCCESS);
 
       test.verify(true);
     } finally {
@@ -695,7 +695,7 @@ public class TestStorageMover {
       //test move a hot file to warm
       final Path file1 = new Path(pathPolicyMap.hot, "file1");
       test.dfs.rename(file1, pathPolicyMap.warm);
-      test.migrate();
+      test.migrate(ExitStatus.NO_MOVE_BLOCK);
       test.verifyFile(new Path(pathPolicyMap.warm, "file1"), WARM.getId());
     } finally {
       test.shutdownCluster();
@@ -753,7 +753,7 @@ public class TestStorageMover {
       { //test move a cold file to warm
         final Path file1 = new Path(pathPolicyMap.cold, "file1");
         test.dfs.rename(file1, pathPolicyMap.warm);
-        test.migrate();
+        test.migrate(ExitStatus.SUCCESS);
         test.verify(true);
       }
     } finally {


[43/50] [abbrv] hadoop git commit: Move HDFS-8337 to branch 2.8 in CHANGES.txt.

Posted by ar...@apache.org.
Move HDFS-8337 to branch 2.8 in CHANGES.txt.


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

Branch: refs/heads/HDFS-7240
Commit: 20c03c96066752f0a7158846de4d7bc4253e83f6
Parents: 971dc83
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Sat Jun 20 00:24:41 2015 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Sat Jun 20 00:25:33 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/20c03c96/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 78360a7..4a633fe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -937,6 +937,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8615. Correct HTTP method in WebHDFS document.
     (Brahma Reddy Battula via aajisaka)
 
+    HDFS-8337. Accessing httpfs via webhdfs doesn't work from a jar with
+    kerberos. (Yongjun Zhang)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -1071,9 +1074,6 @@ Release 2.7.1 - UNRELEASED
     HDFS-8626. Reserved RBW space is not released if creation of RBW File
     fails. (kanaka kumar avvaru via Arpit Agarwal)
 
-    HDFS-8337. Accessing httpfs via webhdfs doesn't work from a jar with
-    kerberos. (Yongjun Zhang)
-
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES


[12/50] [abbrv] hadoop git commit: HDFS-8548. Minicluster throws NPE on shutdown. Contributed by surendra singh lilhore.

Posted by ar...@apache.org.
HDFS-8548. Minicluster throws NPE on shutdown. 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/6a76250b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6a76250b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6a76250b

Branch: refs/heads/HDFS-7240
Commit: 6a76250b39f33466bdc8dabab33070c90aa1a389
Parents: c74517c
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Tue Jun 16 13:52:50 2015 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Tue Jun 16 13:57:02 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                    | 3 +++
 .../apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java   | 6 +++++-
 2 files changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6a76250b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index b921f2c..c8c36e6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -900,6 +900,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8592. SafeModeException never get unwrapped. (wheat9)
 
+    HDFS-8548. Minicluster throws NPE on shutdown.
+    (surendra singh lilhore via xyao)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6a76250b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
index 9eb1059..0dbf485 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
@@ -198,7 +198,11 @@ class BlocksMap {
   }
 
   int size() {
-    return blocks.size();
+    if (blocks != null) {
+      return blocks.size();
+    } else {
+      return 0;
+    }
   }
 
   Iterable<BlockInfo> getBlocks() {


[37/50] [abbrv] hadoop git commit: HDFS-8633. Fix setting of dfs.datanode.readahead.bytes in hdfs-default.xml to match DFSConfigKeys. Contributed by Ray Chiang.

Posted by ar...@apache.org.
HDFS-8633. Fix setting of dfs.datanode.readahead.bytes in hdfs-default.xml to match DFSConfigKeys. Contributed by Ray Chiang.


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

Branch: refs/heads/HDFS-7240
Commit: b42f1ec13d0a837f188cf424a8dd177720478cc9
Parents: 1010724
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Fri Jun 19 09:47:31 2015 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Fri Jun 19 09:47:31 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../hadoop-hdfs/src/main/resources/hdfs-default.xml               | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b42f1ec1/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index a61cf78..749a3ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1061,6 +1061,9 @@ Release 2.7.1 - UNRELEASED
 
     HDFS-4660. Block corruption can happen during pipeline recovery (kihwal)
 
+    HDFS-8633. Fix setting of dfs.datanode.readahead.bytes in hdfs-default.xml
+    to match DFSConfigKeys. (Ray Chiang via Yongjun Zhang)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b42f1ec1/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 fdb0bc8..aaa1c2f 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
@@ -1079,7 +1079,7 @@
 
 <property>
   <name>dfs.datanode.readahead.bytes</name>
-  <value>4193404</value>
+  <value>4194304</value>
   <description>
         While reading block files, if the Hadoop native libraries are available,
         the datanode can use the posix_fadvise system call to explicitly


[17/50] [abbrv] hadoop git commit: HDFS-7164. Feature documentation for HDFS-6581. (Contributed by Arpit Agarwal)

Posted by ar...@apache.org.
HDFS-7164. Feature documentation for HDFS-6581. (Contributed by Arpit Agarwal)


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

Branch: refs/heads/HDFS-7240
Commit: 5dbc8c9cb00da1ba55e1c94c4c1e19d34cf1bd5a
Parents: d4929f4
Author: Arpit Agarwal <ar...@apache.org>
Authored: Tue Jun 16 19:05:44 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Tue Jun 16 19:05:44 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../site/markdown/CentralizedCacheManagement.md |   2 +
 .../src/site/markdown/MemoryStorage.md          | 130 +++++++++++++++++++
 .../site/resources/images/LazyPersistWrites.png | Bin 0 -> 107161 bytes
 hadoop-project/src/site/site.xml                |   1 +
 5 files changed, 135 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dbc8c9c/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index a36e047..07cd4a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -638,6 +638,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-8606. Cleanup DFSOutputStream by removing unwanted changes
     from HDFS-8386. (Rakesh R via szetszwo)
 
+    HDFS-7164. Feature documentation for HDFS-6581. (Arpit Agarwal)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dbc8c9c/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/CentralizedCacheManagement.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/CentralizedCacheManagement.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/CentralizedCacheManagement.md
index b4f08c8..72c125d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/CentralizedCacheManagement.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/CentralizedCacheManagement.md
@@ -233,6 +233,8 @@ Be sure to configure the following:
 
     This determines the maximum amount of memory a DataNode will use for caching. On Unix-like systems, the "locked-in-memory size" ulimit (`ulimit -l`) of the DataNode user also needs to be increased to match this parameter (see below section on [OS Limits](#OS_Limits)). When setting this value, please remember that you will need space in memory for other things as well, such as the DataNode and application JVM heaps and the operating system page cache.
 
+    This setting is shared with the [Lazy Persist Writes feature](./MemoryStorage.html). The Data Node will ensure that the combined memory used by Lazy Persist Writes and Centralized Cache Management does not exceed the amount configured in `dfs.datanode.max.locked.memory`.
+
 #### Optional
 
 The following properties are not required, but may be specified for tuning:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dbc8c9c/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/MemoryStorage.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/MemoryStorage.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/MemoryStorage.md
new file mode 100644
index 0000000..1ed4b5a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/MemoryStorage.md
@@ -0,0 +1,130 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+Memory Storage Support in HDFS
+==============================
+
+* [Introduction](#Introduction)
+* [Administrator Configuration](#Administrator_Configuration)
+    * [Limit RAM used for replicas in Memory](#Limit_RAM_used_for_replicas_in_Memory)
+    * [Setup RAM Disks on Data Nodes](#Setup_RAM_Disks_on_Data_Nodes)
+        * [Choosing `tmpfs` \(vs `ramfs`\)](#Choosing_`tmpfs`_\(vs_`ramfs`\))
+        * [Mount RAM Disks](#Mount_RAM_Disks)
+        * [Tag `tmpfs` volume with the RAM\_DISK Storage Type](#Tag_`tmpfs`_volume_with_the_RAM\_DISK_Storage_Type)
+        * [Ensure Storage Policies are enabled](#Ensure_Storage_Policies_are_enabled)
+* [Application Usage](#Application_Usage)
+    * [Use the LAZY\_PERSIST Storage Policy](#Use_the_LAZY\_PERSIST_Storage_Policy)
+        * [Invoke `hdfs storagepolicies` command for directories](#Invoke_hdfs_storagepolicies_command_for_directories)
+        * [Call `setStoragePolicy` method for directories](#Call_`setStoragePolicy`_method_for_directories)
+        * [Pass `LAZY_PERSIST` `CreateFlag` for new files](#Pass_`LAZY_PERSIST`_`CreateFlag`_for_new_files)
+
+Introduction
+------------
+
+HDFS supports writing to off-heap memory managed by the Data Nodes. The Data Nodes will flush in-memory data to disk asynchronously thus removing expensive disk IO and checksum computations from the performance-sensitive IO path, hence we call such writes *Lazy Persist* writes. HDFS provides best-effort persistence guarantees for Lazy Persist Writes. Rare data loss is possible in the event of a node restart before replicas are persisted to disk. Applications can choose to use Lazy Persist Writes to trade off some durability guarantees in favor of reduced latency.
+
+This feature is available starting with Apache Hadoop 2.6.0 and was developed under Jira [HDFS-6581](https://issues.apache.org/jira/browse/HDFS-6581).
+
+![Lazy Persist Writes](images/LazyPersistWrites.png)
+
+The target use cases are applications that would benefit from writing relatively low amounts of data (from a few GB up to tens of GBs depending on available memory) with low latency. Memory storage is for applications that run within the cluster and collocated with HDFS Data Nodes. We have observed that the latency overhead from network replication negates the benefits of writing to memory.
+
+Applications that use Lazy Persist Writes will continue to work by falling back to DISK storage if memory is insufficient or unconfigured.
+
+Administrator Configuration
+---------------------------
+
+This section enumerates the administrative steps required before applications can start using the feature in a cluster.
+
+## Limit RAM used for replicas in Memory
+
+First decide the amount of memory to be dedicated for replicas stored in memory. Set `dfs.datanode.max.locked.memory` accordingly in `hdfs-site.xml`. This is the same setting used by the [Centralized Cache Management](./CentralizedCacheManagement.html) feature. The Data Node will ensure that the combined memory used by Lazy Persist Writes and Centralized Cache Management does not exceed the amount configured in `dfs.datanode.max.locked.memory`.
+
+E.g. To reserve 32 GB for in-memory replicas
+
+        <property>
+          <name>dfs.datanode.max.locked.memory</name>
+          <value>34359738368</value>
+        </property>
+
+This memory is not allocated by the Data Node on startup.
+
+On Unix-like systems, the "locked-in-memory size" ulimit (`ulimit -l`) of the Data Node user also needs to be increased to match this parameter (see the related section on [OS Limits](./CentralizedCacheManagement.html#OS_Limits)). When setting this value, please remember that you will need space in memory for other things as well, such as the Data Node and application JVM heaps and the operating system page cache. You will also need memory for YARN containers if there is a YARN Node Manager process running on the same node as the Data Node.
+
+## Setup RAM Disks on Data Nodes
+
+Initialize a RAM disk on each Data Node. The choice of RAM Disk allows better data persistence across Data Node process restarts. The following setup will work on most Linux distributions. Using RAM disks on other platforms is not currently supported.
+
+### Choosing `tmpfs` \(vs `ramfs`\)
+
+Linux supports using two kinds of RAM disks - `tmpfs` and `ramfs`. The size of `tmpfs` is limited by the Linux kernel while `ramfs` grows to fill all available system memory. There is a downside to `tmpfs` since its contents can be swapped to disk under memory pressure. However many performance-sensitive deployments run with swapping disabled so we do not expect this to be an issue in practice.
+
+HDFS currently supports using `tmpfs` partitions. Support for adding `ramfs` is in progress (See [HDFS-8584](https://issues.apache.org/jira/browse/HDFS-8584)).
+
+### Mount RAM Disks
+
+Mount the RAM Disk partition with the Unix `mount` command. E.g. to mount a 32 GB `tmpfs` partition under `/mnt/dn-tmpfs/`
+
+        sudo mount -t tmpfs -o size=32g tmpfs /mnt/dn-tmpfs/
+
+It is recommended you create an entry in the `/etc/fstab` so the RAM Disk is recreated automatically on node restarts. Another option is to use a sub-directory under `/dev/shm` which is a `tmpfs` mount available by default on most Linux distributions. Ensure that the size of the mount is greater than or equal to your `dfs.datanode.max.locked.memory` setting else override it in `/etc/fstab`. Using more than one `tmpfs` partition per Data Node for Lazy Persist Writes is not recommended.
+
+### Tag `tmpfs` volume with the RAM\_DISK Storage Type
+
+Tag the `tmpfs` directory with the RAM_DISK storage type via the `dfs.datanode.data.dir` configuration setting in `hdfs-site.xml`. E.g. On a Data Node with three hard disk volumes `/grid/0`, `/grid/1` and `/grid/2` and a `tmpfs` mount `/mnt/dn-tmpfs`, `dfs.datanode.data.dir` must be set as follows:
+
+        <property>
+          <name>dfs.datanode.data.dir</name>
+          <value>/grid/0,/grid/1,/grid/2,[RAM_DISK]/mnt/dn-tmpfs</value>
+        </property>
+
+This step is crucial. Without the RAM_DISK tag, HDFS will treat the `tmpfs` volume as non-volatile storage and data will not be saved to persistent storage. You will lose data on node restart.
+
+### Ensure Storage Policies are enabled
+
+Ensure that the global setting to turn on Storage Policies is enabled [as documented here](ArchivalStorage.html#Configuration). This setting is on by default.
+
+
+Application Usage
+-----------------
+
+## Use the LAZY\_PERSIST Storage Policy
+
+Applications indicate that HDFS can use Lazy Persist Writes for a file with the `LAZY_PERSIST` storage policy. Administrative privileges are *not* required to set the policy and it can be set in one of three ways.
+
+### Invoke `hdfs storagepolicies` command for directories
+
+Setting the policy on a directory causes it to take effect for all new files in the directory. The `hdfs storagepolicies` command can be used to set the policy as described in the [Storage Policies documentation](ArchivalStorage.html#Storage_Policy_Commands).
+
+        hdfs storagepolicies -setStoragePolicy -path <path> -policy LAZY_PERSIST
+
+### Call `setStoragePolicy` method for directories
+
+Starting with Apache Hadoop 2.8.0, an application can programmatically set the Storage Policy with `FileSystem.setStoragePolicy`. E.g.
+
+        fs.setStoragePolicy(path, "LAZY_PERSIST");
+
+### Pass `LAZY_PERSIST` `CreateFlag` for new files
+
+An application can pass `CreateFlag#LAZY_PERSIST` when creating a new file with `FileSystem#create` API. E.g.
+
+        FSDataOutputStream fos =
+            fs.create(
+                path,
+                FsPermission.getFileDefault(),
+                EnumSet.of(CreateFlag.CREATE, CreateFlag.LAZY_PERSIST),
+                bufferLength,
+                replicationFactor,
+                blockSize,
+                null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dbc8c9c/hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/LazyPersistWrites.png
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/LazyPersistWrites.png b/hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/LazyPersistWrites.png
new file mode 100644
index 0000000..b2bde93
Binary files /dev/null and b/hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/LazyPersistWrites.png differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dbc8c9c/hadoop-project/src/site/site.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml
index 865d63e..55be0d9 100644
--- a/hadoop-project/src/site/site.xml
+++ b/hadoop-project/src/site/site.xml
@@ -96,6 +96,7 @@
       <item name="Transparent Encryption" href="hadoop-project-dist/hadoop-hdfs/TransparentEncryption.html"/>
       <item name="Multihoming" href="hadoop-project-dist/hadoop-hdfs/HdfsMultihoming.html"/>
       <item name="Storage Policies" href="hadoop-project-dist/hadoop-hdfs/ArchivalStorage.html"/>
+      <item name="Memory Storage Support" href="hadoop-project-dist/hadoop-hdfs/MemoryStorage.html"/>
       <item name="Synthetic Load Generator" href="hadoop-project-dist/hadoop-hdfs/SLGUserGuide.html"/>
     </menu>
 


[06/50] [abbrv] hadoop git commit: HDFS-8592. SafeModeException never get unwrapped. Contributed by Haohui Mai.

Posted by ar...@apache.org.
HDFS-8592. SafeModeException never get unwrapped. Contributed by Haohui Mai.


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

Branch: refs/heads/HDFS-7240
Commit: 32e39d8a29fdea2647b4372d4422246c9521beb7
Parents: e8c5143
Author: Haohui Mai <wh...@apache.org>
Authored: Mon Jun 15 22:07:24 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Mon Jun 15 22:07:24 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt              |  2 ++
 .../apache/hadoop/hdfs/server/namenode/FSNamesystem.java | 11 ++++++++---
 .../hadoop/hdfs/server/namenode/SafeModeException.java   |  5 ++---
 .../test/java/org/apache/hadoop/hdfs/TestSafeMode.java   |  4 ++--
 4 files changed, 14 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/32e39d8a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 79e7820..0ae2882 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -895,6 +895,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-8607. TestFileCorruption doesn't work as expected. (Walter Su via
     Arpit Agarwal)
 
+    HDFS-8592. SafeModeException never get unwrapped. (wheat9)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32e39d8a/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 518adb4..40c71ea 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
@@ -1323,7 +1323,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void checkNameNodeSafeMode(String errorMsg)
       throws RetriableException, SafeModeException {
     if (isInSafeMode()) {
-      SafeModeException se = new SafeModeException(errorMsg, safeMode);
+      SafeModeException se = newSafemodeException(errorMsg);
       if (haEnabled && haContext != null
           && haContext.getState().getServiceState() == HAServiceState.ACTIVE
           && shouldRetrySafeMode(this.safeMode)) {
@@ -1334,6 +1334,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
   }
 
+  private SafeModeException newSafemodeException(String errorMsg) {
+    return new SafeModeException(errorMsg + ". Name node is in safe " +
+        "mode.\n" + safeMode.getTurnOffTip());
+  }
+
   boolean isPermissionEnabled() {
     return isPermissionEnabled;
   }
@@ -1803,8 +1808,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       for (LocatedBlock b : ret.blocks.getLocatedBlocks()) {
         // if safemode & no block locations yet then throw safemodeException
         if ((b.getLocations() == null) || (b.getLocations().length == 0)) {
-          SafeModeException se = new SafeModeException(
-              "Zero blocklocations for " + src, safeMode);
+          SafeModeException se = newSafemodeException(
+              "Zero blocklocations for " + src);
           if (haEnabled && haContext != null &&
               haContext.getState().getServiceState() == HAServiceState.ACTIVE) {
             throw new RetriableException(se);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32e39d8a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
index 458f052..8080bcf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
@@ -32,8 +32,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceStability.Evolving
 public class SafeModeException extends IOException {
   private static final long serialVersionUID = 1L;
-
-  public SafeModeException(String text, FSNamesystem.SafeModeInfo mode ) {
-    super(text + ". Name node is in safe mode.\n" + mode.getTurnOffTip());
+  public SafeModeException(String msg) {
+    super(msg);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32e39d8a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
index 80fe9ee..6cea7e8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
@@ -295,8 +295,8 @@ public class TestSafeMode {
       fail(msg);
     } catch (RemoteException re) {
       assertEquals(SafeModeException.class.getName(), re.getClassName());
-      GenericTestUtils.assertExceptionContains(
-          "Name node is in safe mode", re);
+      GenericTestUtils.assertExceptionContains("Name node is in safe mode", re);
+    } catch (SafeModeException ignored) {
     } catch (IOException ioe) {
       fail(msg + " " + StringUtils.stringifyException(ioe));
     }


[45/50] [abbrv] hadoop git commit: MAPREDUCE-6316. Task Attempt List entries should link to the task overview. (Siqi Li via gera)

Posted by ar...@apache.org.
MAPREDUCE-6316. Task Attempt List entries should link to the task overview. (Siqi Li via gera)


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

Branch: refs/heads/HDFS-7240
Commit: 63d40d528654707b3f56619a4eb20e21cdeadc7e
Parents: bcb3c40
Author: Gera Shegalov <ge...@apache.org>
Authored: Sat Jun 20 11:38:21 2015 -0700
Committer: Gera Shegalov <ge...@apache.org>
Committed: Sat Jun 20 11:38:21 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 +
 .../mapreduce/v2/app/webapp/AttemptsPage.java   |  8 ++
 .../mapreduce/v2/app/webapp/TaskPage.java       | 11 ++-
 .../mapreduce/v2/app/webapp/TestBlocks.java     | 92 +++++++++++++++++++-
 .../mapreduce/v2/hs/webapp/HsAttemptsPage.java  |  7 ++
 .../mapreduce/v2/hs/webapp/HsTaskPage.java      | 11 ++-
 .../mapreduce/v2/hs/webapp/TestBlocks.java      |  2 +-
 .../resources/webapps/static/yarn.dt.plugins.js | 16 ++--
 8 files changed, 133 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d40d52/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index dfac028..71934b7 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -352,6 +352,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6395. Improve the commit failure messages in MRAppMaster recovery.
     (Brahma Reddy Battula via gera)
 
+    MAPREDUCE-6316. Task Attempt List entries should link to the task overview.
+    (Siqi Li via gera)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d40d52/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AttemptsPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AttemptsPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AttemptsPage.java
index 5dda01e..3b45b16 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AttemptsPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AttemptsPage.java
@@ -29,9 +29,11 @@ import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRApps.TaskAttemptStateUI;
 import org.apache.hadoop.yarn.webapp.SubView;
@@ -51,6 +53,12 @@ public class AttemptsPage extends TaskPage {
     }
 
     @Override
+    protected String getAttemptId(TaskId taskId, TaskAttemptInfo ta) {
+      return "<a href='" + url("task", taskId.toString()) +
+          "'>" + ta.getId() + "</a>";
+    }
+
+    @Override
     protected Collection<TaskAttempt> getTaskAttempts() {
       List<TaskAttempt> fewTaskAttemps = new ArrayList<TaskAttempt>();
       String taskTypeStr = $(TASK_TYPE);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d40d52/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
index d9f17c8..19b0d7c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
 import org.apache.hadoop.mapreduce.v2.util.MRWebAppUtil;
@@ -128,8 +129,9 @@ public class TaskPage extends AppView {
 
         String nodeHttpAddr = ta.getNode();
         String diag = ta.getNote() == null ? "" : ta.getNote();
+        TaskId taskId = attempt.getID().getTaskId();
         attemptsTableData.append("[\"")
-        .append(ta.getId()).append("\",\"")
+        .append(getAttemptId(taskId, ta)).append("\",\"")
         .append(progress).append("\",\"")
         .append(ta.getState().toString()).append("\",\"")
         .append(StringEscapeUtils.escapeJavaScript(
@@ -182,6 +184,10 @@ public class TaskPage extends AppView {
 
     }
 
+    protected String getAttemptId(TaskId taskId, TaskAttemptInfo ta) {
+      return ta.getId();
+    }
+
     protected boolean isValidRequest() {
       return app.getTask() != null;
     }
@@ -215,6 +221,9 @@ public class TaskPage extends AppView {
     .append("\n{'aTargets': [ 5 ]")
     .append(", 'bSearchable': false }")
 
+    .append("\n, {'sType':'string', 'aTargets': [ 0 ]")
+    .append(", 'mRender': parseHadoopID }")
+
     .append("\n, {'sType':'numeric', 'aTargets': [ 6, 7")
     .append(" ], 'mRender': renderHadoopDate }")
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d40d52/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestBlocks.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestBlocks.java
index 13f91e0..3876fe8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestBlocks.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestBlocks.java
@@ -24,19 +24,24 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.junit.Test;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptReport;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskReport;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.api.records.impl.pb.JobIdPBImpl;
+import org.apache.hadoop.mapreduce.v2.api.records.impl.pb.TaskAttemptIdPBImpl;
 import org.apache.hadoop.mapreduce.v2.api.records.impl.pb.TaskIdPBImpl;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.Task;
+import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
+import org.apache.hadoop.mapreduce.v2.app.webapp.AttemptsPage.FewAttemptsBlock;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
 import org.apache.hadoop.yarn.webapp.view.BlockForTest;
@@ -137,8 +142,68 @@ public class TestBlocks {
     assertTrue(data.toString().contains("100011"));
     assertFalse(data.toString().contains("Dummy Status \n*"));
     assertTrue(data.toString().contains("Dummy Status \\n*"));
+  }
+
+  /**
+   * test AttemptsBlock's rendering.
+   */
+  @Test
+  public void testAttemptsBlock() {
+    AppContext ctx = mock(AppContext.class);
+    AppForTest app = new AppForTest(ctx);
+
+    JobId jobId = new JobIdPBImpl();
+    jobId.setId(0);
+    jobId.setAppId(ApplicationIdPBImpl.newInstance(0,1));
+
+    TaskId taskId = new TaskIdPBImpl();
+    taskId.setId(0);
+    taskId.setTaskType(TaskType.REDUCE);
+    taskId.setJobId(jobId);
+    Task task = mock(Task.class);
+    when(task.getID()).thenReturn(taskId);
+    TaskReport report = mock(TaskReport.class);
+
+    when(task.getReport()).thenReturn(report);
+    when(task.getType()).thenReturn(TaskType.REDUCE);
+
+    Map<TaskId, Task> tasks =
+        new HashMap<TaskId, Task>();
+    Map<TaskAttemptId, TaskAttempt> attempts =
+        new HashMap<TaskAttemptId, TaskAttempt>();
+    TaskAttempt attempt = mock(TaskAttempt.class);
+    TaskAttemptId taId = new TaskAttemptIdPBImpl();
+    taId.setId(0);
+    taId.setTaskId(task.getID());
+    when(attempt.getID()).thenReturn(taId);
+
+    final TaskAttemptState taState = TaskAttemptState.SUCCEEDED;
+    when(attempt.getState()).thenReturn(taState);
+    TaskAttemptReport taReport = mock(TaskAttemptReport.class);
+    when(taReport.getTaskAttemptState()).thenReturn(taState);
+    when(attempt.getReport()).thenReturn(taReport);
+    attempts.put(taId, attempt);
+    tasks.put(taskId, task);
+    when(task.getAttempts()).thenReturn(attempts);
+
+    app.setTask(task);
+    Job job = mock(Job.class);
+    when(job.getTasks(TaskType.REDUCE)).thenReturn(tasks);
+    app.setJob(job);
+
+    AttemptsBlockForTest block = new AttemptsBlockForTest(app,
+        new Configuration());
+    block.addParameter(AMParams.TASK_TYPE, "r");
+    block.addParameter(AMParams.ATTEMPT_STATE, "SUCCESSFUL");
 
+    PrintWriter pWriter = new PrintWriter(data);
+    Block html = new BlockForTest(new HtmlBlockForTest(), pWriter, 0, false);
 
+    block.render(html);
+    pWriter.flush();
+    assertTrue(data.toString().contains(
+        "<a href='" + block.url("task",task.getID().toString()) +"'>"
+        +"attempt_0_0001_r_000000_0</a>"));
   }
 
   private class ConfBlockForTest extends ConfBlock {
@@ -168,4 +233,29 @@ public class TestBlocks {
     }
   }
 
+  private class AttemptsBlockForTest extends FewAttemptsBlock {
+    private final Map<String, String> params = new HashMap<String, String>();
+
+    public void addParameter(String name, String value) {
+      params.put(name, value);
+    }
+
+    public String $(String key, String defaultValue) {
+      String value = params.get(key);
+      return value == null ? defaultValue : value;
+    }
+
+    public AttemptsBlockForTest(App ctx, Configuration conf) {
+      super(ctx, conf);
+    }
+
+    @Override
+    public String url(String... parts) {
+      String result = "url://";
+      for (String string : parts) {
+        result += string + ":";
+      }
+      return result;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d40d52/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsAttemptsPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsAttemptsPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsAttemptsPage.java
index 1a6bab9..9233fd3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsAttemptsPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsAttemptsPage.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
 import org.apache.hadoop.mapreduce.v2.app.webapp.App;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRApps.TaskAttemptStateUI;
 import org.apache.hadoop.yarn.webapp.SubView;
@@ -59,6 +60,12 @@ public class HsAttemptsPage extends HsTaskPage {
       return app.getJob() != null;
     }
 
+    @Override
+    protected String getAttemptId(TaskId taskId, TaskAttemptInfo ta) {
+      return "<a href='" + url("task", taskId.toString()) +
+          "'>" + ta.getId() + "</a>";
+    }
+
     /*
      * (non-Javadoc)
      * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsTaskPage.AttemptsBlock#getTaskAttempts()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d40d52/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java
index 5bd8684..ba75018 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java
@@ -143,11 +143,10 @@ public class HsTaskPage extends HsView {
         }
         long attemptElapsed =
             Times.elapsed(attemptStartTime, attemptFinishTime, false);
-        int sortId = attempt.getID().getId()
-                   + (attempt.getID().getTaskId().getId() * 10000);
+        TaskId taskId = attempt.getID().getTaskId();
 
         attemptsTableData.append("[\"")
-        .append(sortId + " ").append(taid).append("\",\"")
+        .append(getAttemptId(taskId, ta)).append("\",\"")
         .append(ta.getState()).append("\",\"")
         .append(StringEscapeUtils.escapeJavaScript(
               StringEscapeUtils.escapeHtml(ta.getStatus()))).append("\",\"")
@@ -234,6 +233,10 @@ public class HsTaskPage extends HsView {
       footRow._()._()._();
     }
 
+    protected String getAttemptId(TaskId taskId, TaskAttemptInfo ta) {
+      return ta.getId();
+    }
+
     /**
      * @return true if this is a valid request else false.
      */
@@ -296,7 +299,7 @@ public class HsTaskPage extends HsView {
       .append(", 'bSearchable': false }")
 
       .append("\n, {'sType':'numeric', 'aTargets': [ 0 ]")
-      .append(", 'mRender': parseHadoopAttemptID }")
+      .append(", 'mRender': parseHadoopID }")
 
       .append("\n, {'sType':'numeric', 'aTargets': [ 5, 6")
       //Column numbers are different for maps and reduces

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d40d52/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestBlocks.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestBlocks.java
index b82965a..d1a704f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestBlocks.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestBlocks.java
@@ -200,7 +200,7 @@ public class TestBlocks {
     block.render(html);
     pWriter.flush();
     // should be printed information about attempts
-    assertTrue(data.toString().contains("0 attempt_0_0001_r_000000_0"));
+    assertTrue(data.toString().contains("attempt_0_0001_r_000000_0"));
     assertTrue(data.toString().contains("SUCCEEDED"));
     assertFalse(data.toString().contains("Processed 128/128 records <p> \n"));
     assertTrue(data.toString().contains("Processed 128\\/128 records &lt;p&gt; \\n"));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d40d52/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/yarn.dt.plugins.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/yarn.dt.plugins.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/yarn.dt.plugins.js
index c9416fd..c003272 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/yarn.dt.plugins.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/yarn.dt.plugins.js
@@ -146,17 +146,13 @@ function parseHadoopID(data, type, full) {
   if (type === 'display') {
     return data;
   }
-  //Return the visible string rather than the entire HTML tag
-  return data.split('>')[1].split('<')[0];
-}
 
-//JSON array element is "20000 attempt_1360183373897_0001_m_000002_0"
-function parseHadoopAttemptID(data, type, full) {
-  if (type === 'display' || type === 'filter') {
-    return data.split(' ')[1];
-  }
-  //For sorting use the order as defined in the JSON element
-  return data.split(' ')[0];
+  var splits =  data.split('>');
+  // Return original string if there is no HTML tag
+  if (splits.length === 1) return data;
+
+  //Return the visible string rather than the entire HTML tag
+  return splits[1].split('<')[0];
 }
 
 function parseHadoopProgress(data, type, full) {


[33/50] [abbrv] hadoop git commit: HDFS-8605. Merge Refactor of DFSOutputStream from HDFS-7285 branch. (vinayakumarb)

Posted by ar...@apache.org.
HDFS-8605. Merge Refactor of DFSOutputStream from HDFS-7285 branch. (vinayakumarb)


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

Branch: refs/heads/HDFS-7240
Commit: 1c13519e1e7588c3e2974138d37bf3449ca8b3df
Parents: 2ad6687
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Jun 18 08:48:09 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Jun 18 08:48:09 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../org/apache/hadoop/hdfs/DFSOutputStream.java | 59 ++++++++++----------
 .../org/apache/hadoop/hdfs/DataStreamer.java    |  7 ++-
 3 files changed, 40 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c13519e/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 2545bcf..a61cf78 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -656,6 +656,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-6249. Output AclEntry in PBImageXmlWriter.
     (surendra singh lilhore via aajisaka)
 
+    HDFS-8605. Merge Refactor of DFSOutputStream from HDFS-7285 branch.
+    (vinayakumarb via wang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c13519e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 4622be6..c16aef2 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -64,6 +64,8 @@ import org.apache.hadoop.util.Time;
 import org.apache.htrace.Sampler;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -86,6 +88,7 @@ import com.google.common.base.Preconditions;
 @InterfaceAudience.Private
 public class DFSOutputStream extends FSOutputSummer
     implements Syncable, CanSetDropBehind {
+  static final Logger LOG = LoggerFactory.getLogger(DFSOutputStream.class);
   /**
    * Number of times to retry creating a file when there are transient 
    * errors (typically related to encryption zones and KeyProvider operations).
@@ -413,21 +416,30 @@ public class DFSOutputStream extends FSOutputSummer
     //
     if (currentPacket.getNumChunks() == currentPacket.getMaxChunks() ||
         getStreamer().getBytesCurBlock() == blockSize) {
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("DFSClient writeChunk packet full seqno=" +
-            currentPacket.getSeqno() +
-            ", src=" + src +
-            ", bytesCurBlock=" + getStreamer().getBytesCurBlock() +
-            ", blockSize=" + blockSize +
-            ", appendChunk=" + getStreamer().getAppendChunk());
-      }
-      getStreamer().waitAndQueuePacket(currentPacket);
-      currentPacket = null;
+      enqueueCurrentPacketFull();
+    }
+  }
 
-      adjustChunkBoundary();
+  void enqueueCurrentPacket() throws IOException {
+    getStreamer().waitAndQueuePacket(currentPacket);
+    currentPacket = null;
+  }
 
-      endBlock();
-    }
+  void enqueueCurrentPacketFull() throws IOException {
+    LOG.debug("enqueue full {}, src={}, bytesCurBlock={}, blockSize={},"
+        + " appendChunk={}, {}", currentPacket, src, getStreamer()
+        .getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(),
+        getStreamer());
+    enqueueCurrentPacket();
+    adjustChunkBoundary();
+    endBlock();
+  }
+
+  /** create an empty packet to mark the end of the block. */
+  void setCurrentPacketToEmpty() throws InterruptedIOException {
+    currentPacket = createPacket(0, 0, getStreamer().getBytesCurBlock(),
+        getStreamer().getAndIncCurrentSeqno(), true);
+    currentPacket.setSyncBlock(shouldSyncBlock);
   }
 
   /**
@@ -457,11 +469,8 @@ public class DFSOutputStream extends FSOutputSummer
    */
   protected void endBlock() throws IOException {
     if (getStreamer().getBytesCurBlock() == blockSize) {
-      currentPacket = createPacket(0, 0, getStreamer().getBytesCurBlock(),
-          getStreamer().getAndIncCurrentSeqno(), true);
-      currentPacket.setSyncBlock(shouldSyncBlock);
-      getStreamer().waitAndQueuePacket(currentPacket);
-      currentPacket = null;
+      setCurrentPacketToEmpty();
+      enqueueCurrentPacket();
       getStreamer().setBytesCurBlock(0);
       lastFlushOffset = 0;
     }
@@ -586,8 +595,7 @@ public class DFSOutputStream extends FSOutputSummer
         }
         if (currentPacket != null) {
           currentPacket.setSyncBlock(isSync);
-          getStreamer().waitAndQueuePacket(currentPacket);
-          currentPacket = null;
+          enqueueCurrentPacket();
         }
         if (endBlock && getStreamer().getBytesCurBlock() > 0) {
           // Need to end the current block, thus send an empty packet to
@@ -595,8 +603,7 @@ public class DFSOutputStream extends FSOutputSummer
           currentPacket = createPacket(0, 0, getStreamer().getBytesCurBlock(),
               getStreamer().getAndIncCurrentSeqno(), true);
           currentPacket.setSyncBlock(shouldSyncBlock || isSync);
-          getStreamer().waitAndQueuePacket(currentPacket);
-          currentPacket = null;
+          enqueueCurrentPacket();
           getStreamer().setBytesCurBlock(0);
           lastFlushOffset = 0;
         } else {
@@ -775,15 +782,11 @@ public class DFSOutputStream extends FSOutputSummer
       flushBuffer();       // flush from all upper layers
 
       if (currentPacket != null) {
-        getStreamer().waitAndQueuePacket(currentPacket);
-        currentPacket = null;
+        enqueueCurrentPacket();
       }
 
       if (getStreamer().getBytesCurBlock() != 0) {
-        // send an empty packet to mark the end of the block
-        currentPacket = createPacket(0, 0, getStreamer().getBytesCurBlock(),
-            getStreamer().getAndIncCurrentSeqno(), true);
-        currentPacket.setSyncBlock(shouldSyncBlock);
+        setCurrentPacketToEmpty();
       }
 
       flushInternal();             // flush all data to Datanodes

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c13519e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index cecd5a0..8dd85b7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -44,7 +44,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
@@ -1901,4 +1900,10 @@ class DataStreamer extends Daemon {
       s.close();
     }
   }
+
+  @Override
+  public String toString() {
+    return  (block == null? null: block.getLocalBlock())
+        + "@" + Arrays.toString(getNodes());
+  }
 }


[29/50] [abbrv] hadoop git commit: HADOOP-11965. determine-flaky-tests needs a summary mode. Contributed by Yufei Gu,

Posted by ar...@apache.org.
HADOOP-11965. determine-flaky-tests needs a summary mode. Contributed by Yufei Gu,


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

Branch: refs/heads/HDFS-7240
Commit: 74351af3b7521b194116258c96270ddaeccd8126
Parents: cc43288
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Wed Jun 17 15:48:29 2015 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Wed Jun 17 20:58:17 2015 -0700

----------------------------------------------------------------------
 dev-support/determine-flaky-tests-hadoop.py     | 57 +++++++++++++++++---
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 ++
 2 files changed, 52 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/74351af3/dev-support/determine-flaky-tests-hadoop.py
----------------------------------------------------------------------
diff --git a/dev-support/determine-flaky-tests-hadoop.py b/dev-support/determine-flaky-tests-hadoop.py
index ce152ba..8644299 100755
--- a/dev-support/determine-flaky-tests-hadoop.py
+++ b/dev-support/determine-flaky-tests-hadoop.py
@@ -62,12 +62,19 @@ import time
 DEFAULT_JENKINS_URL = "https://builds.apache.org"
 DEFAULT_JOB_NAME = "Hadoop-Common-trunk"
 DEFAULT_NUM_PREVIOUS_DAYS = 14
+DEFAULT_TOP_NUM_FAILED_TEST = -1
 
 SECONDS_PER_DAY = 86400
 
 # total number of runs to examine
 numRunsToExamine = 0
 
+#summary mode
+summary_mode = False
+
+#total number of errors
+error_count = 0
+
 """ Parse arguments """
 def parse_args():
   parser = OptionParser()
@@ -80,6 +87,10 @@ def parse_args():
   parser.add_option("-n", "--num-days", type="int",
                     dest="num_prev_days", help="Number of days to examine",
                     default=DEFAULT_NUM_PREVIOUS_DAYS)
+  parser.add_option("-t", "--top", type="int",
+                    dest="num_failed_tests",
+                    help="Summary Mode, only show top number of failed tests",
+                    default=DEFAULT_TOP_NUM_FAILED_TEST)
 
   (options, args) = parser.parse_args()
   if args:
@@ -100,6 +111,7 @@ def load_url_data(url):
  
 """ List all builds of the target project. """
 def list_builds(jenkins_url, job_name):
+  global summary_mode
   url = "%(jenkins)s/job/%(job_name)s/api/json?tree=builds[url,result,timestamp]" % dict(
       jenkins=jenkins_url,
       job_name=job_name)
@@ -108,19 +120,25 @@ def list_builds(jenkins_url, job_name):
     data = load_url_data(url)
 
   except:
-    logging.error("Could not fetch: %s" % url)
+    if not summary_mode:
+      logging.error("Could not fetch: %s" % url)
+    error_count += 1
     raise
   return data['builds']
 
 """ Find the names of any tests which failed in the given build output URL. """
 def find_failing_tests(testReportApiJson, jobConsoleOutput):
+  global summary_mode
+  global error_count
   ret = set()
   try:
     data = load_url_data(testReportApiJson)
 
   except:
-    logging.error("    Could not open testReport, check " +
+    if not summary_mode:
+      logging.error("    Could not open testReport, check " +
         jobConsoleOutput + " for why it was reported failed")
+    error_count += 1
     return ret
 
   for suite in data['suites']:
@@ -130,7 +148,7 @@ def find_failing_tests(testReportApiJson, jobConsoleOutput):
       if (status == 'REGRESSION' or status == 'FAILED' or (errDetails is not None)):
         ret.add(cs['className'] + "." + cs['name'])
 
-  if len(ret) == 0:
+  if len(ret) == 0 and (not summary_mode):
     logging.info("    No failed tests in testReport, check " +
         jobConsoleOutput + " for why it was reported failed.")
   return ret
@@ -138,6 +156,7 @@ def find_failing_tests(testReportApiJson, jobConsoleOutput):
 """ Iterate runs of specfied job within num_prev_days and collect results """
 def find_flaky_tests(jenkins_url, job_name, num_prev_days):
   global numRunsToExamine
+  global summary_mode
   all_failing = dict()
   # First list all builds
   builds = list_builds(jenkins_url, job_name)
@@ -153,7 +172,8 @@ def find_flaky_tests(jenkins_url, job_name, num_prev_days):
   tnum = len(builds)
   num = len(failing_build_urls)
   numRunsToExamine = tnum
-  logging.info("    THERE ARE " + str(num) + " builds (out of " + str(tnum)
+  if not summary_mode:
+    logging.info("    THERE ARE " + str(num) + " builds (out of " + str(tnum)
       + ") that have failed tests in the past " + str(num_prev_days) + " days"
       + ((".", ", as listed below:\n")[num > 0]))
 
@@ -165,17 +185,20 @@ def find_flaky_tests(jenkins_url, job_name, num_prev_days):
 
     ts = float(failed_build_with_time[1]) / 1000.
     st = datetime.datetime.fromtimestamp(ts).strftime('%Y-%m-%d %H:%M:%S')
-    logging.info("===>%s" % str(testReport) + " (" + st + ")")
+    if not summary_mode:
+      logging.info("===>%s" % str(testReport) + " (" + st + ")")
     failing = find_failing_tests(testReportApiJson, jobConsoleOutput)
     if failing:
       for ftest in failing:
-        logging.info("    Failed test: %s" % ftest)
+        if not summary_mode:
+          logging.info("    Failed test: %s" % ftest)
         all_failing[ftest] = all_failing.get(ftest,0)+1
 
   return all_failing
 
 def main():
   global numRunsToExamine
+  global summary_mode
   logging.basicConfig(format='%(levelname)s:%(message)s', level=logging.INFO)
 
   # set up logger to write to stdout
@@ -189,16 +212,34 @@ def main():
   logging.info("****Recently FAILED builds in url: " + opts.jenkins_url
       + "/job/" + opts.job_name + "")
 
+  if opts.num_failed_tests != -1:
+    summary_mode = True
+
   all_failing = find_flaky_tests(opts.jenkins_url, opts.job_name,
       opts.num_prev_days)
   if len(all_failing) == 0:
     raise SystemExit(0)
-  logging.info("\nAmong " + str(numRunsToExamine) + " runs examined, all failed "
-      + "tests <#failedRuns: testName>:")
+
+  if summary_mode and opts.num_failed_tests < len(all_failing):
+    logging.info("\nAmong " + str(numRunsToExamine) +
+                 " runs examined, top " + str(opts.num_failed_tests) +
+                 " failed tests <#failedRuns: testName>:")
+  else:
+      logging.info("\nAmong " + str(numRunsToExamine) +
+                   " runs examined, all failed tests <#failedRuns: testName>:")
 
   # print summary section: all failed tests sorted by how many times they failed
+  line_count = 0
   for tn in sorted(all_failing, key=all_failing.get, reverse=True):
     logging.info("    " + str(all_failing[tn])+ ": " + tn)
+    if summary_mode:
+      line_count += 1
+      if line_count == opts.num_failed_tests:
+        break
+
+  if summary_mode and error_count > 0:
+    logging.info("\n" + str(error_count) + " errors found, you may "
+                 + "re-run in non summary mode to see error details.");
 
 if __name__ == "__main__":
   main()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/74351af3/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 67e9c76..4b1b382 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -648,6 +648,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11971. Move test utilities for tracing from hadoop-hdfs to
     hadoop-common. (Masatake Iwasaki via aajisaka)
 
+    HADOOP-11965. determine-flaky-tests needs a summary mode.
+    (Yufei Gu via Yongjun Zhang)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp


[48/50] [abbrv] hadoop git commit: HDFS-8192. Eviction should key off used locked memory instead of ram disk free space. (Contributed by Arpit Agarwal)

Posted by ar...@apache.org.
HDFS-8192. Eviction should key off used locked memory instead of ram disk free space. (Contributed by Arpit Agarwal)


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

Branch: refs/heads/HDFS-7240
Commit: c7d022b66f0c5baafbb7000a435c1d6e39906efe
Parents: 658b5c8
Author: Arpit Agarwal <ar...@apache.org>
Authored: Sat Jun 20 13:27:52 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Sat Jun 20 13:27:52 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   4 -
 .../datanode/fsdataset/impl/FsDatasetCache.java |   7 +
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |  98 +++++-----
 .../hdfs/server/balancer/TestBalancer.java      |   1 -
 .../fsdataset/impl/LazyPersistTestCase.java     |  42 ++--
 .../impl/TestLazyPersistLockedMemory.java       |  25 ++-
 .../impl/TestLazyPersistReplicaPlacement.java   |  36 +++-
 .../datanode/fsdataset/impl/TestLazyWriter.java |  62 +++---
 .../fsdataset/impl/TestScrLazyPersistFiles.java | 193 +++++++------------
 10 files changed, 224 insertions(+), 247 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7d022b6/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index aad3c25..2e030b9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -658,6 +658,9 @@ Release 2.8.0 - UNRELEASED
     do not generate spurious reconfig warnings (Lei (Eddy) Xu via Colin P.
     McCabe)
 
+    HDFS-8192. Eviction should key off used locked memory instead of
+    ram disk free space. (Arpit Agarwal)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7d022b6/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 5ce2863..30540a9 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
@@ -94,10 +94,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_DATANODE_LAZY_WRITER_INTERVAL_DEFAULT_SEC = 60;
   public static final String  DFS_DATANODE_RAM_DISK_REPLICA_TRACKER_KEY = "dfs.datanode.ram.disk.replica.tracker";
   public static final Class<RamDiskReplicaLruTracker>  DFS_DATANODE_RAM_DISK_REPLICA_TRACKER_DEFAULT = RamDiskReplicaLruTracker.class;
-  public static final String  DFS_DATANODE_RAM_DISK_LOW_WATERMARK_PERCENT = "dfs.datanode.ram.disk.low.watermark.percent";
-  public static final float   DFS_DATANODE_RAM_DISK_LOW_WATERMARK_PERCENT_DEFAULT = 10.0f;
-  public static final String  DFS_DATANODE_RAM_DISK_LOW_WATERMARK_BYTES = "dfs.datanode.ram.disk.low.watermark.bytes";
-  public static final long    DFS_DATANODE_RAM_DISK_LOW_WATERMARK_BYTES_DEFAULT = DFS_BLOCK_SIZE_DEFAULT;
   public static final String  DFS_DATANODE_NETWORK_COUNTS_CACHE_MAX_SIZE_KEY = "dfs.datanode.network.counts.cache.max.size";
   public static final int     DFS_DATANODE_NETWORK_COUNTS_CACHE_MAX_SIZE_DEFAULT = Integer.MAX_VALUE;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7d022b6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetCache.java
index 6f524b2..f70d4af 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetCache.java
@@ -405,6 +405,13 @@ public class FsDatasetCache {
   }
 
   /**
+   * Round up to the OS page size.
+   */
+  long roundUpPageSize(long count) {
+    return usedBytesCount.rounder.roundUp(count);
+  }
+
+  /**
    * Background worker that mmaps, mlocks, and checksums a block
    */
   private class CachingTask implements Runnable {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7d022b6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 8ebd214..a1ff918 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -1302,14 +1302,13 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     if (allowLazyPersist &&
         lazyWriter != null &&
         b.getNumBytes() % cacheManager.getOsPageSize() == 0 &&
-        (cacheManager.reserve(b.getNumBytes())) > 0) {
+        reserveLockedMemory(b.getNumBytes())) {
       try {
         // First try to place the block on a transient volume.
         ref = volumes.getNextTransientVolume(b.getNumBytes());
         datanode.getMetrics().incrRamDiskBlocksWrite();
       } catch(DiskOutOfSpaceException de) {
         // Ignore the exception since we just fall back to persistent storage.
-        datanode.getMetrics().incrRamDiskBlocksWriteFallback();
       } finally {
         if (ref == null) {
           cacheManager.release(b.getNumBytes());
@@ -1323,6 +1322,11 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
     FsVolumeImpl v = (FsVolumeImpl) ref.getVolume();
     // create an rbw file to hold block in the designated volume
+
+    if (allowLazyPersist && !v.isTransientStorage()) {
+      datanode.getMetrics().incrRamDiskBlocksWriteFallback();
+    }
+
     File f;
     try {
       f = v.createRbwFile(b.getBlockPoolId(), b.getLocalBlock());
@@ -2833,20 +2837,11 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   class LazyWriter implements Runnable {
     private volatile boolean shouldRun = true;
     final int checkpointerInterval;
-    final float lowWatermarkFreeSpacePercentage;
-    final long lowWatermarkFreeSpaceBytes;
-
 
     public LazyWriter(Configuration conf) {
       this.checkpointerInterval = conf.getInt(
           DFSConfigKeys.DFS_DATANODE_LAZY_WRITER_INTERVAL_SEC,
           DFSConfigKeys.DFS_DATANODE_LAZY_WRITER_INTERVAL_DEFAULT_SEC);
-      this.lowWatermarkFreeSpacePercentage = conf.getFloat(
-          DFSConfigKeys.DFS_DATANODE_RAM_DISK_LOW_WATERMARK_PERCENT,
-          DFSConfigKeys.DFS_DATANODE_RAM_DISK_LOW_WATERMARK_PERCENT_DEFAULT);
-      this.lowWatermarkFreeSpaceBytes = conf.getLong(
-          DFSConfigKeys.DFS_DATANODE_RAM_DISK_LOW_WATERMARK_BYTES,
-          DFSConfigKeys.DFS_DATANODE_RAM_DISK_LOW_WATERMARK_BYTES_DEFAULT);
     }
 
     /**
@@ -2908,41 +2903,17 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       return succeeded;
     }
 
-    private boolean transientFreeSpaceBelowThreshold() throws IOException {
-      long free = 0;
-      long capacity = 0;
-      float percentFree = 0.0f;
-
-      // Don't worry about fragmentation for now. We don't expect more than one
-      // transient volume per DN.
-      try (FsVolumeReferences volumes = getFsVolumeReferences()) {
-        for (FsVolumeSpi fvs : volumes) {
-          FsVolumeImpl v = (FsVolumeImpl) fvs;
-          if (v.isTransientStorage()) {
-            capacity += v.getCapacity();
-            free += v.getAvailable();
-          }
-        }
-      }
-
-      if (capacity == 0) {
-        return false;
-      }
-
-      percentFree = (float) ((double)free * 100 / capacity);
-      return (percentFree < lowWatermarkFreeSpacePercentage) ||
-          (free < lowWatermarkFreeSpaceBytes);
-    }
-
     /**
      * Attempt to evict one or more transient block replicas until we
-     * have at least spaceNeeded bytes free.
+     * have at least bytesNeeded bytes free.
      */
-    private void evictBlocks() throws IOException {
+    public void evictBlocks(long bytesNeeded) throws IOException {
       int iterations = 0;
 
+      final long cacheCapacity = cacheManager.getCacheCapacity();
+
       while (iterations++ < MAX_BLOCK_EVICTIONS_PER_ITERATION &&
-             transientFreeSpaceBelowThreshold()) {
+             (cacheCapacity - cacheManager.getCacheUsed()) < bytesNeeded) {
         RamDiskReplica replicaState = ramDiskReplicaTracker.getNextCandidateForEviction();
 
         if (replicaState == null) {
@@ -2959,7 +2930,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
         final String bpid = replicaState.getBlockPoolId();
 
         synchronized (FsDatasetImpl.this) {
-          replicaInfo = getReplicaInfo(replicaState.getBlockPoolId(), replicaState.getBlockId());
+          replicaInfo = getReplicaInfo(replicaState.getBlockPoolId(),
+                                       replicaState.getBlockId());
           Preconditions.checkState(replicaInfo.getVolume().isTransientStorage());
           blockFile = replicaInfo.getBlockFile();
           metaFile = replicaInfo.getMetaFile();
@@ -2968,7 +2940,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           ramDiskReplicaTracker.discardReplica(replicaState.getBlockPoolId(),
               replicaState.getBlockId(), false);
 
-          // Move the replica from lazyPersist/ to finalized/ on target volume
+          // Move the replica from lazyPersist/ to finalized/ on
+          // the target volume
           BlockPoolSlice bpSlice =
               replicaState.getLazyPersistVolume().getBlockPoolSlice(bpid);
           File newBlockFile = bpSlice.activateSavedReplica(
@@ -2992,10 +2965,12 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           if (replicaState.getNumReads() == 0) {
             datanode.getMetrics().incrRamDiskBlocksEvictedWithoutRead();
           }
-        }
 
-        removeOldReplica(replicaInfo, newReplicaInfo, blockFile, metaFile,
-            blockFileUsed, metaFileUsed, bpid);
+          // Delete the block+meta files from RAM disk and release locked
+          // memory.
+          removeOldReplica(replicaInfo, newReplicaInfo, blockFile, metaFile,
+              blockFileUsed, metaFileUsed, bpid);
+        }
       }
     }
 
@@ -3006,7 +2981,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       while (fsRunning && shouldRun) {
         try {
           numSuccessiveFailures = saveNextReplica() ? 0 : (numSuccessiveFailures + 1);
-          evictBlocks();
 
           // Sleep if we have no more work to do or if it looks like we are not
           // making any forward progress. This is to ensure that if all persist
@@ -3094,5 +3068,37 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       cacheManager.releaseRoundDown(count);
     }
   }
+
+  /**
+   * Attempt to evict blocks from cache Manager to free the requested
+   * bytes.
+   *
+   * @param bytesNeeded
+   */
+  @VisibleForTesting
+  public void evictLazyPersistBlocks(long bytesNeeded) {
+    try {
+      ((LazyWriter) lazyWriter.getRunnable()).evictBlocks(bytesNeeded);
+    } catch(IOException ioe) {
+      LOG.info("Ignoring exception ", ioe);
+    }
+  }
+
+  /**
+   * Attempt to reserve the given amount of memory with the cache Manager.
+   * @param bytesNeeded
+   * @return
+   */
+  boolean reserveLockedMemory(long bytesNeeded) {
+    if (cacheManager.reserve(bytesNeeded) > 0) {
+      return true;
+    }
+
+    // Round up bytes needed to osPageSize and attempt to evict
+    // one more more blocks to free up the reservation.
+    bytesNeeded = cacheManager.roundUpPageSize(bytesNeeded);
+    evictLazyPersistBlocks(bytesNeeded);
+    return cacheManager.reserve(bytesNeeded) > 0;
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7d022b6/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 1f7bade..e1ce1b3 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
@@ -130,7 +130,6 @@ public class TestBalancer {
     conf.setLong(DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 500);
     conf.setInt(DFS_DATANODE_LAZY_WRITER_INTERVAL_SEC, 1);
-    conf.setInt(DFS_DATANODE_RAM_DISK_LOW_WATERMARK_BYTES, DEFAULT_RAM_DISK_BLOCK_SIZE);
     LazyPersistTestCase.initCacheManipulator();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7d022b6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
index 5ce5cc6..ce29fc8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
+import com.google.common.base.Supplier;
+import org.apache.commons.lang.UnhandledException;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 
 import static org.apache.hadoop.fs.CreateFlag.CREATE;
@@ -37,6 +39,7 @@ import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.Set;
 import java.util.UUID;
+import java.util.concurrent.TimeoutException;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.io.IOUtils;
@@ -55,6 +58,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
@@ -79,7 +83,6 @@ public abstract class LazyPersistTestCase {
 
   protected static final int BLOCK_SIZE = 5 * 1024 * 1024;
   protected static final int BUFFER_LENGTH = 4096;
-  protected static final int EVICTION_LOW_WATERMARK = 1;
   private static final long HEARTBEAT_INTERVAL_SEC = 1;
   private static final int HEARTBEAT_RECHECK_INTERVAL_MSEC = 500;
   private static final String JMX_RAM_DISK_METRICS_PATTERN = "^RamDisk";
@@ -236,7 +239,6 @@ public abstract class LazyPersistTestCase {
       StorageType[] storageTypes,
       int ramDiskReplicaCapacity,
       long ramDiskStorageLimit,
-      long evictionLowWatermarkReplicas,
       long maxLockedMemory,
       boolean useSCR,
       boolean useLegacyBlockReaderLocal,
@@ -256,8 +258,6 @@ public abstract class LazyPersistTestCase {
                 HEARTBEAT_RECHECK_INTERVAL_MSEC);
     conf.setInt(DFS_DATANODE_LAZY_WRITER_INTERVAL_SEC,
                 LAZY_WRITER_INTERVAL_SEC);
-    conf.setLong(DFS_DATANODE_RAM_DISK_LOW_WATERMARK_BYTES,
-                evictionLowWatermarkReplicas * BLOCK_SIZE);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY, 1);
     conf.setLong(DFS_DATANODE_MAX_LOCKED_MEMORY_KEY, maxLockedMemory);
 
@@ -389,12 +389,6 @@ public abstract class LazyPersistTestCase {
       return this;
     }
 
-    public ClusterWithRamDiskBuilder setEvictionLowWatermarkReplicas(
-        long evictionLowWatermarkReplicas) {
-      this.evictionLowWatermarkReplicas = evictionLowWatermarkReplicas;
-      return this;
-    }
-
     public ClusterWithRamDiskBuilder disableScrubber() {
       this.disableScrubber = true;
       return this;
@@ -403,8 +397,8 @@ public abstract class LazyPersistTestCase {
     public void build() throws IOException {
       LazyPersistTestCase.this.startUpCluster(
           numDatanodes, hasTransientStorage, storageTypes, ramDiskReplicaCapacity,
-          ramDiskStorageLimit, evictionLowWatermarkReplicas,
-          maxLockedMemory, useScr, useLegacyBlockReaderLocal, disableScrubber);
+          ramDiskStorageLimit, maxLockedMemory, useScr, useLegacyBlockReaderLocal,
+          disableScrubber);
     }
 
     private int numDatanodes = REPL_FACTOR;
@@ -415,7 +409,6 @@ public abstract class LazyPersistTestCase {
     private boolean hasTransientStorage = true;
     private boolean useScr = false;
     private boolean useLegacyBlockReaderLocal = false;
-    private long evictionLowWatermarkReplicas = EVICTION_LOW_WATERMARK;
     private boolean disableScrubber=false;
   }
 
@@ -513,4 +506,27 @@ public abstract class LazyPersistTestCase {
       e.printStackTrace();
     }
   }
+
+  protected void waitForMetric(final String metricName, final int expectedValue)
+      throws TimeoutException, InterruptedException {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        try {
+          final int currentValue = Integer.parseInt(jmx.getValue(metricName));
+          LOG.info("Waiting for " + metricName +
+                       " to reach value " + expectedValue +
+                       ", current value = " + currentValue);
+          return currentValue == expectedValue;
+        } catch (Exception e) {
+          throw new UnhandledException("Test failed due to unexpected exception", e);
+        }
+      }
+    }, 1000, Integer.MAX_VALUE);
+  }
+
+  protected void triggerEviction(DataNode dn) {
+    FsDatasetImpl fsDataset = (FsDatasetImpl) dn.getFSDataset();
+    fsDataset.evictLazyPersistBlocks(Long.MAX_VALUE); // Run one eviction cycle.
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7d022b6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistLockedMemory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistLockedMemory.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistLockedMemory.java
index 9ea4665..eef8f0b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistLockedMemory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistLockedMemory.java
@@ -28,9 +28,7 @@ import org.apache.hadoop.hdfs.DFSOutputStream;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
-import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.test.MetricsAsserts;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -103,25 +101,26 @@ public class TestLazyPersistLockedMemory extends LazyPersistTestCase {
    * Verify that locked RAM is released when blocks are evicted from RAM disk.
    */
   @Test
-  public void testReleaseOnEviction()
-      throws IOException, TimeoutException, InterruptedException {
+  public void testReleaseOnEviction() throws Exception {
     getClusterBuilder().setNumDatanodes(1)
                        .setMaxLockedMemory(BLOCK_SIZE)
                        .setRamDiskReplicaCapacity(BLOCK_SIZE * 2 - 1)
                        .build();
     final String METHOD_NAME = GenericTestUtils.getMethodName();
-    final FsDatasetSpi<?> fsd = cluster.getDataNodes().get(0).getFSDataset();
+    final FsDatasetImpl fsd =
+        (FsDatasetImpl) cluster.getDataNodes().get(0).getFSDataset();
 
-    Path path = new Path("/" + METHOD_NAME + ".dat");
-    makeTestFile(path, BLOCK_SIZE, true);
+    Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
+    makeTestFile(path1, BLOCK_SIZE, true);
+    assertThat(fsd.getCacheUsed(), is((long) BLOCK_SIZE));
 
-    // The block should get evicted soon since it pushes RAM disk free
-    // space below the threshold.
-    waitForLockedBytesUsed(fsd, 0);
+    // Wait until the replica is written to persistent storage.
+    waitForMetric("RamDiskBlocksLazyPersisted", 1);
 
-    MetricsRecordBuilder rb =
-        MetricsAsserts.getMetrics(cluster.getDataNodes().get(0).getMetrics().name());
-    MetricsAsserts.assertCounter("RamDiskBlocksEvicted", 1L, rb);
+    // Trigger eviction and verify locked bytes were released.
+    fsd.evictLazyPersistBlocks(Long.MAX_VALUE);
+    verifyRamDiskJMXMetric("RamDiskBlocksEvicted", 1);
+    waitForLockedBytesUsed(fsd, 0);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7d022b6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistReplicaPlacement.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistReplicaPlacement.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistReplicaPlacement.java
index 018eaba..c89475a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistReplicaPlacement.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistReplicaPlacement.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -28,6 +29,8 @@ import java.io.IOException;
 
 import static org.apache.hadoop.fs.StorageType.DEFAULT;
 import static org.apache.hadoop.fs.StorageType.RAM_DISK;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.fail;
 
 public class TestLazyPersistReplicaPlacement extends LazyPersistTestCase {
@@ -70,32 +73,50 @@ public class TestLazyPersistReplicaPlacement extends LazyPersistTestCase {
     ensureFileReplicasOnStorageType(path, DEFAULT);
   }
 
+  @Test
+  public void testSynchronousEviction() throws Exception {
+    getClusterBuilder().setMaxLockedMemory(BLOCK_SIZE).build();
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+
+    final Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
+    makeTestFile(path1, BLOCK_SIZE, true);
+    ensureFileReplicasOnStorageType(path1, RAM_DISK);
+
+    // Wait until the replica is written to persistent storage.
+    waitForMetric("RamDiskBlocksLazyPersisted", 1);
+
+    // Ensure that writing a new file to RAM DISK evicts the block
+    // for the previous one.
+    Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
+    makeTestFile(path2, BLOCK_SIZE, true);
+    verifyRamDiskJMXMetric("RamDiskBlocksEvictedWithoutRead", 1);
+  }
+
   /**
    * File can not fit in RamDisk even with eviction
    * @throws IOException
    */
   @Test
   public void testFallbackToDiskFull() throws Exception {
-    getClusterBuilder().setRamDiskReplicaCapacity(0).build();
+    getClusterBuilder().setMaxLockedMemory(BLOCK_SIZE / 2).build();
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path path = new Path("/" + METHOD_NAME + ".dat");
 
     makeTestFile(path, BLOCK_SIZE, true);
     ensureFileReplicasOnStorageType(path, DEFAULT);
-
     verifyRamDiskJMXMetric("RamDiskBlocksWriteFallback", 1);
   }
 
   /**
    * File partially fit in RamDisk after eviction.
    * RamDisk can fit 2 blocks. Write a file with 5 blocks.
-   * Expect 2 or less blocks are on RamDisk and 3 or more on disk.
+   * Expect 2 blocks are on RamDisk and rest on disk.
    * @throws IOException
    */
   @Test
   public void testFallbackToDiskPartial()
       throws IOException, InterruptedException {
-    getClusterBuilder().setRamDiskReplicaCapacity(2).build();
+    getClusterBuilder().setMaxLockedMemory(2 * BLOCK_SIZE).build();
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path path = new Path("/" + METHOD_NAME + ".dat");
 
@@ -122,8 +143,8 @@ public class TestLazyPersistReplicaPlacement extends LazyPersistTestCase {
 
     // Since eviction is asynchronous, depending on the timing of eviction
     // wrt writes, we may get 2 or less blocks on RAM disk.
-    assert(numBlocksOnRamDisk <= 2);
-    assert(numBlocksOnDisk >= 3);
+    assertThat(numBlocksOnRamDisk, is(2));
+    assertThat(numBlocksOnDisk, is(3));
   }
 
   /**
@@ -134,7 +155,8 @@ public class TestLazyPersistReplicaPlacement extends LazyPersistTestCase {
    */
   @Test
   public void testRamDiskNotChosenByDefault() throws IOException {
-    getClusterBuilder().build();
+    getClusterBuilder().setStorageTypes(new StorageType[] {RAM_DISK, RAM_DISK})
+                       .build();
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path path = new Path("/" + METHOD_NAME + ".dat");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7d022b6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyWriter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyWriter.java
index ee8aaf0..6b16066 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyWriter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyWriter.java
@@ -28,6 +28,7 @@ import org.junit.Test;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.concurrent.TimeoutException;
 
 import static org.apache.hadoop.fs.StorageType.DEFAULT;
 import static org.apache.hadoop.fs.StorageType.RAM_DISK;
@@ -38,18 +39,16 @@ import static org.junit.Assert.assertTrue;
 public class TestLazyWriter extends LazyPersistTestCase {
   @Test
   public void testLazyPersistBlocksAreSaved()
-      throws IOException, InterruptedException {
+      throws IOException, InterruptedException, TimeoutException {
     getClusterBuilder().build();
+    final int NUM_BLOCKS = 10;
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path path = new Path("/" + METHOD_NAME + ".dat");
 
     // Create a test file
-    makeTestFile(path, BLOCK_SIZE * 10, true);
+    makeTestFile(path, BLOCK_SIZE * NUM_BLOCKS, true);
     LocatedBlocks locatedBlocks = ensureFileReplicasOnStorageType(path, RAM_DISK);
-
-    // Sleep for a short time to allow the lazy writer thread to do its job
-    Thread.sleep(6 * LAZY_WRITER_INTERVAL_SEC * 1000);
-
+    waitForMetric("RamDiskBlocksLazyPersisted", NUM_BLOCKS);
     LOG.info("Verifying copy was saved to lazyPersist/");
 
     // Make sure that there is a saved copy of the replica on persistent
@@ -57,35 +56,22 @@ public class TestLazyWriter extends LazyPersistTestCase {
     ensureLazyPersistBlocksAreSaved(locatedBlocks);
   }
 
-  /**
-   * RamDisk eviction after lazy persist to disk.
-   * @throws Exception
-   */
   @Test
-  public void testRamDiskEviction() throws Exception {
-    getClusterBuilder().setRamDiskReplicaCapacity(1 + EVICTION_LOW_WATERMARK).build();
+  public void testSynchronousEviction() throws Exception {
+    getClusterBuilder().setMaxLockedMemory(BLOCK_SIZE).build();
     final String METHOD_NAME = GenericTestUtils.getMethodName();
-    Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
-    Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
 
-    final int SEED = 0xFADED;
-    makeRandomTestFile(path1, BLOCK_SIZE, true, SEED);
+    final Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
+    makeTestFile(path1, BLOCK_SIZE, true);
     ensureFileReplicasOnStorageType(path1, RAM_DISK);
 
-    // Sleep for a short time to allow the lazy writer thread to do its job.
-    Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
-    ensureFileReplicasOnStorageType(path1, RAM_DISK);
+    // Wait until the replica is written to persistent storage.
+    waitForMetric("RamDiskBlocksLazyPersisted", 1);
 
-    // Create another file with a replica on RAM_DISK.
+    // Ensure that writing a new file to RAM DISK evicts the block
+    // for the previous one.
+    Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
     makeTestFile(path2, BLOCK_SIZE, true);
-    Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
-    triggerBlockReport();
-
-    // Ensure the first file was evicted to disk, the second is still on
-    // RAM_DISK.
-    ensureFileReplicasOnStorageType(path2, RAM_DISK);
-    ensureFileReplicasOnStorageType(path1, DEFAULT);
-
     verifyRamDiskJMXMetric("RamDiskBlocksEvicted", 1);
     verifyRamDiskJMXMetric("RamDiskBlocksEvictedWithoutRead", 1);
   }
@@ -98,8 +84,8 @@ public class TestLazyWriter extends LazyPersistTestCase {
    */
   @Test
   public void testRamDiskEvictionBeforePersist()
-      throws IOException, InterruptedException {
-    getClusterBuilder().setRamDiskReplicaCapacity(1).build();
+      throws Exception {
+    getClusterBuilder().setMaxLockedMemory(BLOCK_SIZE).build();
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
     Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
@@ -116,6 +102,7 @@ public class TestLazyWriter extends LazyPersistTestCase {
 
     // Eviction should not happen for block of the first file that is not
     // persisted yet.
+    verifyRamDiskJMXMetric("RamDiskBlocksEvicted", 0);
     ensureFileReplicasOnStorageType(path1, RAM_DISK);
     ensureFileReplicasOnStorageType(path2, DEFAULT);
 
@@ -133,7 +120,7 @@ public class TestLazyWriter extends LazyPersistTestCase {
   public void testRamDiskEvictionIsLru()
       throws Exception {
     final int NUM_PATHS = 5;
-    getClusterBuilder().setRamDiskReplicaCapacity(NUM_PATHS + EVICTION_LOW_WATERMARK).build();
+    getClusterBuilder().setMaxLockedMemory(NUM_PATHS * BLOCK_SIZE).build();
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path paths[] = new Path[NUM_PATHS * 2];
 
@@ -145,8 +132,7 @@ public class TestLazyWriter extends LazyPersistTestCase {
       makeTestFile(paths[i], BLOCK_SIZE, true);
     }
 
-    // Sleep for a short time to allow the lazy writer thread to do its job.
-    Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
+    waitForMetric("RamDiskBlocksLazyPersisted", NUM_PATHS);
 
     for (int i = 0; i < NUM_PATHS; ++i) {
       ensureFileReplicasOnStorageType(paths[i], RAM_DISK);
@@ -227,16 +213,13 @@ public class TestLazyWriter extends LazyPersistTestCase {
 
     makeTestFile(path, BLOCK_SIZE, true);
     LocatedBlocks locatedBlocks = ensureFileReplicasOnStorageType(path, RAM_DISK);
-
-    // Sleep for a short time to allow the lazy writer thread to do its job
-    Thread.sleep(6 * LAZY_WRITER_INTERVAL_SEC * 1000);
+    waitForMetric("RamDiskBlocksLazyPersisted", 1);
 
     // Delete after persist
     client.delete(path.toString(), false);
     Assert.assertFalse(fs.exists(path));
 
     assertThat(verifyDeletedBlocks(locatedBlocks), is(true));
-
     verifyRamDiskJMXMetric("RamDiskBlocksLazyPersisted", 1);
     verifyRamDiskJMXMetric("RamDiskBytesLazyPersisted", BLOCK_SIZE);
   }
@@ -248,7 +231,7 @@ public class TestLazyWriter extends LazyPersistTestCase {
    */
   @Test
   public void testDfsUsageCreateDelete()
-      throws IOException, InterruptedException {
+      throws IOException, InterruptedException, TimeoutException {
     getClusterBuilder().setRamDiskReplicaCapacity(4).build();
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path path = new Path("/" + METHOD_NAME + ".dat");
@@ -261,8 +244,7 @@ public class TestLazyWriter extends LazyPersistTestCase {
 
     assertThat(usedAfterCreate, is((long) BLOCK_SIZE));
 
-    // Sleep for a short time to allow the lazy writer thread to do its job
-    Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
+    waitForMetric("RamDiskBlocksLazyPersisted", 1);
 
     long usedAfterPersist = fs.getUsed();
     assertThat(usedAfterPersist, is((long) BLOCK_SIZE));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7d022b6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestScrLazyPersistFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestScrLazyPersistFiles.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestScrLazyPersistFiles.java
index 7c7ba64..2512588 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestScrLazyPersistFiles.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestScrLazyPersistFiles.java
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+import com.google.common.base.Preconditions;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -26,6 +27,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
+import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.NativeCodeLoader;
@@ -39,13 +41,20 @@ import org.junit.rules.ExpectedException;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.concurrent.TimeoutException;
 
 import static org.apache.hadoop.fs.StorageType.DEFAULT;
 import static org.apache.hadoop.fs.StorageType.RAM_DISK;
 import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.core.Is.is;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
+/**
+ * Test Lazy persist behavior with short-circuit reads. These tests
+ * will be run on Linux only with Native IO enabled. The tests fake
+ * RAM_DISK storage using local disk.
+ */
 public class TestScrLazyPersistFiles extends LazyPersistTestCase {
 
   @BeforeClass
@@ -58,6 +67,10 @@ public class TestScrLazyPersistFiles extends LazyPersistTestCase {
     Assume.assumeThat(NativeCodeLoader.isNativeCodeLoaded() && !Path.WINDOWS,
         equalTo(true));
     Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
+
+    final long osPageSize = NativeIO.POSIX.getCacheManipulator().getOperatingSystemPageSize();
+    Preconditions.checkState(BLOCK_SIZE >= osPageSize);
+    Preconditions.checkState(BLOCK_SIZE % osPageSize == 0);
   }
 
   @Rule
@@ -69,35 +82,27 @@ public class TestScrLazyPersistFiles extends LazyPersistTestCase {
    */
   @Test
   public void testRamDiskShortCircuitRead()
-    throws IOException, InterruptedException {
-    getClusterBuilder().setNumDatanodes(REPL_FACTOR)
-                       .setStorageTypes(new StorageType[]{RAM_DISK, DEFAULT})
-                       .setRamDiskStorageLimit(2 * BLOCK_SIZE - 1)
-                       .setUseScr(true)
-                       .build();
+      throws IOException, InterruptedException, TimeoutException {
+    getClusterBuilder().setUseScr(true).build();
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     final int SEED = 0xFADED;
     Path path = new Path("/" + METHOD_NAME + ".dat");
 
+    // Create a file and wait till it is persisted.
     makeRandomTestFile(path, BLOCK_SIZE, true, SEED);
     ensureFileReplicasOnStorageType(path, RAM_DISK);
+    waitForMetric("RamDiskBlocksLazyPersisted", 1);
 
-    // Sleep for a short time to allow the lazy writer thread to do its job
-    Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
-
-    //assertThat(verifyReadRandomFile(path, BLOCK_SIZE, SEED), is(true));
-    FSDataInputStream fis = fs.open(path);
+    HdfsDataInputStream fis = (HdfsDataInputStream) fs.open(path);
 
     // Verify SCR read counters
     try {
-      fis = fs.open(path);
       byte[] buf = new byte[BUFFER_LENGTH];
       fis.read(0, buf, 0, BUFFER_LENGTH);
-      HdfsDataInputStream dfsis = (HdfsDataInputStream) fis;
       Assert.assertEquals(BUFFER_LENGTH,
-        dfsis.getReadStatistics().getTotalBytesRead());
+        fis.getReadStatistics().getTotalBytesRead());
       Assert.assertEquals(BUFFER_LENGTH,
-        dfsis.getReadStatistics().getTotalShortCircuitBytesRead());
+        fis.getReadStatistics().getTotalShortCircuitBytesRead());
     } finally {
       fis.close();
       fis = null;
@@ -111,106 +116,77 @@ public class TestScrLazyPersistFiles extends LazyPersistTestCase {
    * @throws InterruptedException
    */
   @Test
-  public void testRamDiskEvictionWithShortCircuitReadHandle()
-    throws IOException, InterruptedException {
-    // 5 replica + delta, SCR.
-    getClusterBuilder().setNumDatanodes(REPL_FACTOR)
-                       .setStorageTypes(new StorageType[]{RAM_DISK, DEFAULT})
-                       .setRamDiskStorageLimit(6 * BLOCK_SIZE - 1)
-                       .setEvictionLowWatermarkReplicas(3)
-                       .setUseScr(true)
-                       .build();
-
+  public void tesScrDuringEviction()
+      throws Exception {
+    getClusterBuilder().setUseScr(true).build();
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
-    Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
-    final int SEED = 0xFADED;
 
-    makeRandomTestFile(path1, BLOCK_SIZE, true, SEED);
+    // Create a file and wait till it is persisted.
+    makeTestFile(path1, BLOCK_SIZE, true);
     ensureFileReplicasOnStorageType(path1, RAM_DISK);
+    waitForMetric("RamDiskBlocksLazyPersisted", 1);
 
-    // Sleep for a short time to allow the lazy writer thread to do its job.
-    // However the block replica should not be evicted from RAM_DISK yet.
-    Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
-
-    // No eviction should happen as the free ratio is below the threshold
-    FSDataInputStream fis = fs.open(path1);
+    HdfsDataInputStream fis = (HdfsDataInputStream) fs.open(path1);
     try {
       // Keep and open read handle to path1 while creating path2
       byte[] buf = new byte[BUFFER_LENGTH];
       fis.read(0, buf, 0, BUFFER_LENGTH);
-
-      // Create the 2nd file that will trigger RAM_DISK eviction.
-      makeTestFile(path2, BLOCK_SIZE * 2, true);
-      ensureFileReplicasOnStorageType(path2, RAM_DISK);
+      triggerEviction(cluster.getDataNodes().get(0));
 
       // Ensure path1 is still readable from the open SCR handle.
-      fis.read(fis.getPos(), buf, 0, BUFFER_LENGTH);
-      HdfsDataInputStream dfsis = (HdfsDataInputStream) fis;
-      Assert.assertEquals(2 * BUFFER_LENGTH,
-        dfsis.getReadStatistics().getTotalBytesRead());
-      Assert.assertEquals(2 * BUFFER_LENGTH,
-        dfsis.getReadStatistics().getTotalShortCircuitBytesRead());
+      fis.read(0, buf, 0, BUFFER_LENGTH);
+      assertThat(fis.getReadStatistics().getTotalBytesRead(),
+          is((long) 2 * BUFFER_LENGTH));
+      assertThat(fis.getReadStatistics().getTotalShortCircuitBytesRead(),
+          is((long) 2 * BUFFER_LENGTH));
     } finally {
       IOUtils.closeQuietly(fis);
     }
-
-    // After the open handle is closed, path1 should be evicted to DISK.
-    triggerBlockReport();
-    ensureFileReplicasOnStorageType(path1, DEFAULT);
   }
 
   @Test
-  public void testShortCircuitReadAfterEviction()
-      throws IOException, InterruptedException {
-    Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
-    getClusterBuilder().setRamDiskReplicaCapacity(1 + EVICTION_LOW_WATERMARK)
-                       .setUseScr(true)
+  public void testScrAfterEviction()
+      throws IOException, InterruptedException, TimeoutException {
+    getClusterBuilder().setUseScr(true)
                        .setUseLegacyBlockReaderLocal(false)
                        .build();
     doShortCircuitReadAfterEvictionTest();
   }
 
   @Test
-  public void testLegacyShortCircuitReadAfterEviction()
-      throws IOException, InterruptedException {
-    getClusterBuilder().setRamDiskReplicaCapacity(1 + EVICTION_LOW_WATERMARK)
-                       .setUseScr(true)
+  public void testLegacyScrAfterEviction()
+      throws IOException, InterruptedException, TimeoutException {
+    getClusterBuilder().setUseScr(true)
                        .setUseLegacyBlockReaderLocal(true)
                        .build();
     doShortCircuitReadAfterEvictionTest();
+
+    // In the implementation of legacy short-circuit reads, any failure is
+    // trapped silently, reverts back to a remote read, and also disables all
+    // subsequent legacy short-circuit reads in the ClientContext.
+    // Assert that it didn't get disabled.
+    ClientContext clientContext = client.getClientContext();
+    Assert.assertFalse(clientContext.getDisableLegacyBlockReaderLocal());
   }
 
   private void doShortCircuitReadAfterEvictionTest() throws IOException,
-      InterruptedException {
+      InterruptedException, TimeoutException {
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
-    Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
 
     final int SEED = 0xFADED;
     makeRandomTestFile(path1, BLOCK_SIZE, true, SEED);
+    ensureFileReplicasOnStorageType(path1, RAM_DISK);
+    waitForMetric("RamDiskBlocksLazyPersisted", 1);
 
     // Verify short-circuit read from RAM_DISK.
-    ensureFileReplicasOnStorageType(path1, RAM_DISK);
     File metaFile = cluster.getBlockMetadataFile(0,
         DFSTestUtil.getFirstBlock(fs, path1));
     assertTrue(metaFile.length() <= BlockMetadataHeader.getHeaderSize());
     assertTrue(verifyReadRandomFile(path1, BLOCK_SIZE, SEED));
 
-    // Sleep for a short time to allow the lazy writer thread to do its job.
-    Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
-
-    // Verify short-circuit read from RAM_DISK once again.
-    ensureFileReplicasOnStorageType(path1, RAM_DISK);
-    metaFile = cluster.getBlockMetadataFile(0,
-        DFSTestUtil.getFirstBlock(fs, path1));
-    assertTrue(metaFile.length() <= BlockMetadataHeader.getHeaderSize());
-    assertTrue(verifyReadRandomFile(path1, BLOCK_SIZE, SEED));
-
-    // Create another file with a replica on RAM_DISK, which evicts the first.
-    makeRandomTestFile(path2, BLOCK_SIZE, true, SEED);
-    Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
-    triggerBlockReport();
+    triggerEviction(cluster.getDataNodes().get(0));
 
     // Verify short-circuit read still works from DEFAULT storage.  This time,
     // we'll have a checksum written during lazy persistence.
@@ -219,54 +195,35 @@ public class TestScrLazyPersistFiles extends LazyPersistTestCase {
         DFSTestUtil.getFirstBlock(fs, path1));
     assertTrue(metaFile.length() > BlockMetadataHeader.getHeaderSize());
     assertTrue(verifyReadRandomFile(path1, BLOCK_SIZE, SEED));
-
-    // In the implementation of legacy short-circuit reads, any failure is
-    // trapped silently, reverts back to a remote read, and also disables all
-    // subsequent legacy short-circuit reads in the ClientContext.  If the test
-    // uses legacy, then assert that it didn't get disabled.
-    ClientContext clientContext = client.getClientContext();
-    if (clientContext.getUseLegacyBlockReaderLocal()) {
-      Assert.assertFalse(clientContext.getDisableLegacyBlockReaderLocal());
-    }
   }
 
   @Test
-  public void testShortCircuitReadBlockFileCorruption() throws IOException,
-      InterruptedException {
-    Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
-    getClusterBuilder().setRamDiskReplicaCapacity(1 + EVICTION_LOW_WATERMARK)
-                       .setUseScr(true)
+  public void testScrBlockFileCorruption() throws IOException,
+      InterruptedException, TimeoutException {
+    getClusterBuilder().setUseScr(true)
                        .setUseLegacyBlockReaderLocal(false)
                        .build();
     doShortCircuitReadBlockFileCorruptionTest();
   }
 
   @Test
-  public void testLegacyShortCircuitReadBlockFileCorruption() throws IOException,
-      InterruptedException {
-    getClusterBuilder().setRamDiskReplicaCapacity(1 + EVICTION_LOW_WATERMARK)
-                       .setUseScr(true)
+  public void testLegacyScrBlockFileCorruption() throws IOException,
+      InterruptedException, TimeoutException {
+    getClusterBuilder().setUseScr(true)
                        .setUseLegacyBlockReaderLocal(true)
                        .build();
     doShortCircuitReadBlockFileCorruptionTest();
   }
 
   public void doShortCircuitReadBlockFileCorruptionTest() throws IOException,
-      InterruptedException {
+      InterruptedException, TimeoutException {
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
-    Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
 
-    final int SEED = 0xFADED;
-    makeRandomTestFile(path1, BLOCK_SIZE, true, SEED);
+    makeTestFile(path1, BLOCK_SIZE, true);
     ensureFileReplicasOnStorageType(path1, RAM_DISK);
-
-    // Create another file with a replica on RAM_DISK, which evicts the first.
-    makeRandomTestFile(path2, BLOCK_SIZE, true, SEED);
-
-    // Sleep for a short time to allow the lazy writer thread to do its job.
-    Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
-    triggerBlockReport();
+    waitForMetric("RamDiskBlocksLazyPersisted", 1);
+    triggerEviction(cluster.getDataNodes().get(0));
 
     // Corrupt the lazy-persisted block file, and verify that checksum
     // verification catches it.
@@ -277,42 +234,32 @@ public class TestScrLazyPersistFiles extends LazyPersistTestCase {
   }
 
   @Test
-  public void testShortCircuitReadMetaFileCorruption() throws IOException,
-      InterruptedException {
-    Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
-    getClusterBuilder().setRamDiskReplicaCapacity(1 + EVICTION_LOW_WATERMARK)
-                       .setUseScr(true)
+  public void testScrMetaFileCorruption() throws IOException,
+      InterruptedException, TimeoutException {
+    getClusterBuilder().setUseScr(true)
                        .setUseLegacyBlockReaderLocal(false)
                        .build();
     doShortCircuitReadMetaFileCorruptionTest();
   }
 
   @Test
-  public void testLegacyShortCircuitReadMetaFileCorruption() throws IOException,
-      InterruptedException {
-    getClusterBuilder().setRamDiskReplicaCapacity(1 + EVICTION_LOW_WATERMARK)
-                       .setUseScr(true)
+  public void testLegacyScrMetaFileCorruption() throws IOException,
+      InterruptedException, TimeoutException {
+    getClusterBuilder().setUseScr(true)
                        .setUseLegacyBlockReaderLocal(true)
                        .build();
     doShortCircuitReadMetaFileCorruptionTest();
   }
 
   public void doShortCircuitReadMetaFileCorruptionTest() throws IOException,
-      InterruptedException {
+      InterruptedException, TimeoutException {
     final String METHOD_NAME = GenericTestUtils.getMethodName();
     Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
-    Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
 
-    final int SEED = 0xFADED;
-    makeRandomTestFile(path1, BLOCK_SIZE, true, SEED);
+    makeTestFile(path1, BLOCK_SIZE, true);
     ensureFileReplicasOnStorageType(path1, RAM_DISK);
-
-    // Create another file with a replica on RAM_DISK, which evicts the first.
-    makeRandomTestFile(path2, BLOCK_SIZE, true, SEED);
-
-    // Sleep for a short time to allow the lazy writer thread to do its job.
-    Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
-    triggerBlockReport();
+    waitForMetric("RamDiskBlocksLazyPersisted", 1);
+    triggerEviction(cluster.getDataNodes().get(0));
 
     // Corrupt the lazy-persisted checksum file, and verify that checksum
     // verification catches it.


[32/50] [abbrv] hadoop git commit: YARN-3824. Fix two minor nits in member variable properties of YarnConfiguration. Contributed by Ray Chiang.

Posted by ar...@apache.org.
YARN-3824. Fix two minor nits in member variable properties of
YarnConfiguration. Contributed by Ray Chiang.


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

Branch: refs/heads/HDFS-7240
Commit: 2ad668748e2535e25e3b04656c29034d58074027
Parents: 295d678
Author: Devaraj K <de...@apache.org>
Authored: Thu Jun 18 16:44:08 2015 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Thu Jun 18 16:44:08 2015 +0530

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ad66874/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 243edb3..f00170e 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -536,6 +536,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3617. Fix WindowsResourceCalculatorPlugin.getCpuFrequency()
     returning always -1. (J.Andreina via devaraj)
 
+    YARN-3824. Fix two minor nits in member variable properties
+    of YarnConfiguration. (Ray Chiang via devaraj)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ad66874/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 3ea1558..5d75a21 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
@@ -278,7 +278,7 @@ public class YarnConfiguration extends Configuration {
       + "intermediate-data-encryption.enable";
 
   @Private
-  public static final Boolean DEFAULT_YARN_INTERMEDIATE_DATA_ENCRYPTION = false;
+  public static final boolean DEFAULT_YARN_INTERMEDIATE_DATA_ENCRYPTION = false;
 
   /** The address of the RM admin interface.*/
   public static final String RM_ADMIN_ADDRESS = 
@@ -729,7 +729,7 @@ public class YarnConfiguration extends Configuration {
 
   public static final String RM_PROXY_USER_PRIVILEGES_ENABLED = RM_PREFIX
       + "proxy-user-privileges.enabled";
-  public static boolean DEFAULT_RM_PROXY_USER_PRIVILEGES_ENABLED = false;
+  public static final boolean DEFAULT_RM_PROXY_USER_PRIVILEGES_ENABLED = false;
 
   /**
    * How many diagnostics/failure messages can be saved in RM for


[36/50] [abbrv] hadoop git commit: MAPREDUCE-6395. Improve the commit failure messages in MRAppMaster recovery. (Brahma Reddy Battula via gera)

Posted by ar...@apache.org.
MAPREDUCE-6395. Improve the commit failure messages in MRAppMaster recovery. (Brahma Reddy Battula via gera)


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

Branch: refs/heads/HDFS-7240
Commit: 10107243be66bae2212a2cd8575f9f5ade13fe9e
Parents: 5b5bb8d
Author: Gera Shegalov <ge...@apache.org>
Authored: Fri Jun 19 02:22:59 2015 -0700
Committer: Gera Shegalov <ge...@apache.org>
Committed: Fri Jun 19 02:22:59 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                  |  3 +++
 .../apache/hadoop/mapreduce/v2/app/MRAppMaster.java   | 14 ++++++++++----
 2 files changed, 13 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/10107243/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 1037091..7e57804 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -349,6 +349,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6392. Document mapred class path options.
     (Brahma Reddy Battula via cnauroth)
 
+    MAPREDUCE-6395. Improve the commit failure messages in MRAppMaster recovery.
+    (Brahma Reddy Battula via gera)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10107243/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
index 752b30c..0d3519b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
@@ -334,14 +334,20 @@ public class MRAppMaster extends CompositeService {
             " because a commit was started.");
         copyHistory = true;
         if (commitSuccess) {
-          shutDownMessage = "We crashed after successfully committing. Recovering.";
+          shutDownMessage =
+              "Job commit succeeded in a prior MRAppMaster attempt " +
+              "before it crashed. Recovering.";
           forcedState = JobStateInternal.SUCCEEDED;
         } else if (commitFailure) {
-          shutDownMessage = "We crashed after a commit failure.";
+          shutDownMessage =
+              "Job commit failed in a prior MRAppMaster attempt " +
+              "before it crashed. Not retrying.";
           forcedState = JobStateInternal.FAILED;
         } else {
           //The commit is still pending, commit error
-          shutDownMessage = "We crashed durring a commit";
+          shutDownMessage =
+              "Job commit from a prior MRAppMaster attempt is " +
+              "potentially in progress. Preventing multiple commit executions";
           forcedState = JobStateInternal.ERROR;
         }
       }
@@ -582,7 +588,7 @@ public class MRAppMaster extends CompositeService {
       //if isLastAMRetry comes as true, should never set it to false
       if ( !isLastAMRetry){
         if (((JobImpl)job).getInternalState() != JobStateInternal.REBOOT) {
-          LOG.info("We are finishing cleanly so this is the last retry");
+          LOG.info("Job finished cleanly, recording last MRAppMaster retry");
           isLastAMRetry = true;
         }
       }


[07/50] [abbrv] hadoop git commit: YARN-3789. Improve logs for LeafQueue#activateApplications(). Contributed by Bibin A Chundatt.

Posted by ar...@apache.org.
YARN-3789. Improve logs for LeafQueue#activateApplications(). Contributed
by Bibin A Chundatt.


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

Branch: refs/heads/HDFS-7240
Commit: b039e69bb03accef485361af301fa59f03d08d6a
Parents: 32e39d8
Author: Devaraj K <de...@apache.org>
Authored: Tue Jun 16 14:03:22 2015 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Tue Jun 16 14:03:22 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                    |  3 +++
 .../scheduler/capacity/LeafQueue.java              | 17 +++++++++--------
 2 files changed, 12 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b039e69b/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index e442244..d0d1e06 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -306,6 +306,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3785. Support for Resource as an argument during submitApp call in MockRM
     test class. (Sunil G via xgong)
 
+    YARN-3789. Improve logs for LeafQueue#activateApplications().
+    (Bibin A Chundatt via devaraj)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b039e69b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
index 15d3289..8e39133 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
@@ -56,7 +56,6 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.security.AccessType;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -604,7 +603,7 @@ public class LeafQueue extends AbstractCSQueue {
     for (Iterator<FiCaSchedulerApp> i=pendingApplications.iterator(); 
          i.hasNext(); ) {
       FiCaSchedulerApp application = i.next();
-      
+      ApplicationId applicationId = application.getApplicationId();
       // Check am resource limit
       Resource amIfStarted = 
         Resources.add(application.getAMResource(), queueUsage.getAMUsed());
@@ -624,7 +623,9 @@ public class LeafQueue extends AbstractCSQueue {
             " single application in queue, it is likely set too low." +
             " skipping enforcement to allow at least one application to start"); 
         } else {
-          LOG.info("not starting application as amIfStarted exceeds amLimit");
+          LOG.info("Not activating application " + applicationId
+              + " as  amIfStarted: " + amIfStarted + " exceeds amLimit: "
+              + amLimit);
           continue;
         }
       }
@@ -645,8 +646,9 @@ public class LeafQueue extends AbstractCSQueue {
             " single application in queue for user, it is likely set too low." +
             " skipping enforcement to allow at least one application to start"); 
         } else {
-          LOG.info("not starting application as amIfStarted exceeds " +
-            "userAmLimit");
+          LOG.info("Not activating application " + applicationId
+              + " for user: " + user + " as userAmIfStarted: "
+              + userAmIfStarted + " exceeds userAmLimit: " + userAMLimit);
           continue;
         }
       }
@@ -657,9 +659,8 @@ public class LeafQueue extends AbstractCSQueue {
       metrics.incAMUsed(application.getUser(), application.getAMResource());
       metrics.setAMResouceLimitForUser(application.getUser(), userAMLimit);
       i.remove();
-      LOG.info("Application " + application.getApplicationId() +
-          " from user: " + application.getUser() + 
-          " activated in queue: " + getQueueName());
+      LOG.info("Application " + applicationId + " from user: "
+          + application.getUser() + " activated in queue: " + getQueueName());
     }
   }
   


[38/50] [abbrv] hadoop git commit: HADOOP-12103. Small refactoring of DelegationTokenAuthenticationFilter to allow code sharing. Contributed by Yongjun Zhang.

Posted by ar...@apache.org.
HADOOP-12103. Small refactoring of DelegationTokenAuthenticationFilter to allow code sharing. 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/49f5d20e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/49f5d20e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/49f5d20e

Branch: refs/heads/HDFS-7240
Commit: 49f5d20efe7af7cd7c45d93edad33997a695a746
Parents: b42f1ec
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Fri Jun 19 14:06:17 2015 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Fri Jun 19 14:06:17 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt     |  3 +++
 .../web/DelegationTokenAuthenticationFilter.java    | 16 +++++++++++++++-
 2 files changed, 18 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/49f5d20e/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 2f5eda3..e1d9ca9 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -878,6 +878,9 @@ Release 2.7.1 - UNRELEASED
 
   IMPROVEMENTS
 
+    HADOOP-12103. Small refactoring of DelegationTokenAuthenticationFilter to
+    allow code sharing. (Yongjun Zhang)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49f5d20e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java
index b6e1a76..af66ee5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java
@@ -115,7 +115,22 @@ public class DelegationTokenAuthenticationFilter
   protected Properties getConfiguration(String configPrefix,
       FilterConfig filterConfig) throws ServletException {
     Properties props = super.getConfiguration(configPrefix, filterConfig);
+    setAuthHandlerClass(props);
+    return props;
+  }
+
+  /**
+   * Set AUTH_TYPE property to the name of the corresponding authentication
+   * handler class based on the input properties.
+   * @param props input properties.
+   */
+  protected void setAuthHandlerClass(Properties props)
+      throws ServletException {
     String authType = props.getProperty(AUTH_TYPE);
+    if (authType == null) {
+      throw new ServletException("Config property "
+          + AUTH_TYPE + " doesn't exist");
+    }
     if (authType.equals(PseudoAuthenticationHandler.TYPE)) {
       props.setProperty(AUTH_TYPE,
           PseudoDelegationTokenAuthenticationHandler.class.getName());
@@ -123,7 +138,6 @@ public class DelegationTokenAuthenticationFilter
       props.setProperty(AUTH_TYPE,
           KerberosDelegationTokenAuthenticationHandler.class.getName());
     }
-    return props;
   }
 
   /**


[09/50] [abbrv] hadoop git commit: HDFS-8597. Fix TestFSImage#testZeroBlockSize on Windows. Contributed by Xiaoyu Yao.

Posted by ar...@apache.org.
HDFS-8597. Fix TestFSImage#testZeroBlockSize on Windows. 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/4e88ff5b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4e88ff5b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4e88ff5b

Branch: refs/heads/HDFS-7240
Commit: 4e88ff5b27cc33d311ab7a7248c3cf6303997ddd
Parents: a3990ca
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Tue Jun 16 12:38:07 2015 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Tue Jun 16 12:38:07 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                      | 2 ++
 .../org/apache/hadoop/hdfs/server/datanode/StorageLocation.java  | 4 ++--
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e88ff5b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e0ef52f..42588cc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1024,6 +1024,8 @@ Release 2.7.1 - UNRELEASED
     HDFS-8576.  Lease recovery should return true if the lease can be released
     and the file can be closed.  (J.Andreina via szetszwo)
 
+    HDFS-8597. Fix TestFSImage#testZeroBlockSize on Windows. (Xiaoyu Yao)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4e88ff5b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
index 126086f..5c8dd85 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
@@ -26,8 +26,8 @@ import java.net.URI;
 import java.util.regex.Matcher;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.util.StringUtils;
 
 /**
@@ -94,7 +94,7 @@ public class StorageLocation {
       }
     }
 
-    return new StorageLocation(storageType, Util.stringAsURI(location));
+    return new StorageLocation(storageType, new Path(location).toUri());
   }
 
   @Override


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

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


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

Branch: refs/heads/HDFS-7240
Commit: dd784e975f0068698c7a615abf42ecd443137745
Parents: 197b8fb c7d022b
Author: Arpit Agarwal <ar...@apache.org>
Authored: Sat Jun 20 14:14:35 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Sat Jun 20 14:14:35 2015 -0700

----------------------------------------------------------------------
 dev-support/determine-flaky-tests-hadoop.py     |   57 +-
 hadoop-common-project/hadoop-common/CHANGES.txt |   22 +
 .../apache/hadoop/conf/ReconfigurableBase.java  |   14 +-
 .../java/org/apache/hadoop/fs/StorageType.java  |    7 +-
 .../hadoop/fs/permission/FsPermission.java      |    7 +-
 .../java/org/apache/hadoop/io/SequenceFile.java |   85 +-
 .../hadoop/security/LdapGroupsMapping.java      |   29 +-
 .../alias/AbstractJavaKeyStoreProvider.java     |   15 +-
 .../DelegationTokenAuthenticationFilter.java    |   16 +-
 .../src/main/resources/core-default.xml         |   18 +
 .../apache/hadoop/conf/TestReconfiguration.java |   12 +-
 .../org/apache/hadoop/fs/shell/TestCount.java   |    4 +-
 .../hadoop/io/TestSequenceFileAppend.java       |  311 ++++
 .../TestLdapGroupsMappingWithPosixGroup.java    |   25 +-
 .../hdfs/client/HdfsClientConfigKeys.java       |    1 +
 .../hadoop/hdfs/protocol/ClientProtocol.java    | 1459 ++++++++++++++++++
 .../hdfs/server/namenode/SafeModeException.java |   38 +
 .../http/server/HttpFSAuthenticationFilter.java |    5 +
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   72 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |    7 +-
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |   68 +-
 .../org/apache/hadoop/hdfs/DataStreamer.java    |    7 +-
 .../hdfs/protocol/ClientDatanodeProtocol.java   |    5 +
 .../hadoop/hdfs/protocol/ClientProtocol.java    | 1459 ------------------
 ...tDatanodeProtocolServerSideTranslatorPB.java |   19 +
 .../ClientDatanodeProtocolTranslatorPB.java     |   19 +-
 .../server/blockmanagement/BlockManager.java    |   54 +-
 .../BlockReportLeaseManager.java                |    1 -
 .../hdfs/server/blockmanagement/BlocksMap.java  |    6 +-
 .../blockmanagement/DecommissionManager.java    |   42 +-
 .../PendingReplicationBlocks.java               |   51 +-
 .../blockmanagement/UnderReplicatedBlocks.java  |   57 +-
 .../hdfs/server/datanode/BPServiceActor.java    |    1 -
 .../hdfs/server/datanode/BlockReceiver.java     |  126 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |   26 +-
 .../hdfs/server/datanode/StorageLocation.java   |    4 +-
 .../datanode/fsdataset/impl/FsDatasetCache.java |    7 +
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |   98 +-
 .../datanode/fsdataset/impl/FsVolumeImpl.java   |    7 +-
 .../apache/hadoop/hdfs/server/mover/Mover.java  |   95 +-
 .../hdfs/server/namenode/FSDirAttrOp.java       |   11 +-
 .../server/namenode/FSDirStatAndListingOp.java  |   93 +-
 .../hdfs/server/namenode/FSDirectory.java       |   20 +
 .../hdfs/server/namenode/FSNamesystem.java      |  203 +--
 .../hadoop/hdfs/server/namenode/INode.java      |   12 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |    4 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |   63 +-
 .../hadoop/hdfs/server/namenode/SafeMode.java   |    4 +-
 .../hdfs/server/namenode/SafeModeException.java |   39 -
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |   59 +-
 .../offlineImageViewer/PBImageXmlWriter.java    |   20 +-
 .../src/main/proto/ClientDatanodeProtocol.proto |   12 +
 .../src/main/resources/hdfs-default.xml         |    2 +-
 .../site/markdown/CentralizedCacheManagement.md |    2 +
 .../src/site/markdown/MemoryStorage.md          |  130 ++
 .../hadoop-hdfs/src/site/markdown/WebHDFS.md    |   12 +-
 .../site/resources/images/LazyPersistWrites.png |  Bin 0 -> 107161 bytes
 .../hadoop/hdfs/TestBlockStoragePolicy.java     |   75 +-
 .../apache/hadoop/hdfs/TestDecommission.java    |   26 -
 .../apache/hadoop/hdfs/TestLeaseRecovery.java   |   46 +
 .../org/apache/hadoop/hdfs/TestSafeMode.java    |    4 +-
 .../hdfs/server/balancer/TestBalancer.java      |    1 -
 .../blockmanagement/BlockManagerTestUtil.java   |    5 +-
 .../blockmanagement/TestBlockManager.java       |    8 +-
 .../server/blockmanagement/TestNodeCount.java   |    3 +-
 .../TestOverReplicatedBlocks.java               |    5 +-
 .../blockmanagement/TestPendingReplication.java |   27 +-
 .../TestRBWBlockInvalidation.java               |    4 +-
 .../blockmanagement/TestReplicationPolicy.java  |   70 +-
 .../TestUnderReplicatedBlockQueues.java         |   16 +-
 .../datanode/TestReadOnlySharedStorage.java     |   11 +-
 .../fsdataset/impl/LazyPersistTestCase.java     |   42 +-
 .../impl/TestLazyPersistLockedMemory.java       |   25 +-
 .../impl/TestLazyPersistReplicaPlacement.java   |   36 +-
 .../datanode/fsdataset/impl/TestLazyWriter.java |   62 +-
 .../fsdataset/impl/TestRbwSpaceReservation.java |   45 +
 .../fsdataset/impl/TestScrLazyPersistFiles.java |  193 +--
 .../hadoop/hdfs/server/mover/TestMover.java     |   29 +
 .../hdfs/server/mover/TestStorageMover.java     |   18 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |   16 +-
 .../namenode/TestProcessCorruptBlocks.java      |    5 +-
 .../apache/hadoop/hdfs/tools/TestDFSAdmin.java  |  117 +-
 .../TestOfflineImageViewerForAcl.java           |   24 +
 hadoop-mapreduce-project/CHANGES.txt            |   21 +
 .../hadoop/mapreduce/v2/app/MRAppMaster.java    |   14 +-
 .../mapreduce/v2/app/webapp/AttemptsPage.java   |    8 +
 .../mapreduce/v2/app/webapp/TaskPage.java       |   93 +-
 .../mapreduce/v2/app/webapp/TestBlocks.java     |   92 +-
 .../apache/hadoop/mapred/FileInputFormat.java   |    2 +-
 .../mapreduce/lib/input/FileInputFormat.java    |    3 +-
 .../mapreduce/v2/hs/webapp/HsAttemptsPage.java  |    7 +
 .../mapreduce/v2/hs/webapp/HsController.java    |   28 +-
 .../mapreduce/v2/hs/webapp/HsCountersPage.java  |    8 +-
 .../mapreduce/v2/hs/webapp/HsLogsPage.java      |   11 +-
 .../v2/hs/webapp/HsSingleCounterPage.java       |    8 +-
 .../mapreduce/v2/hs/webapp/HsTaskPage.java      |   11 +-
 .../hadoop/mapreduce/v2/hs/webapp/HsView.java   |   13 +
 .../mapreduce/v2/hs/webapp/TestBlocks.java      |    2 +-
 .../java/org/apache/hadoop/hdfs/NNBench.java    |   15 +-
 .../mapred/pipes/TestPipeApplication.java       |    5 +-
 hadoop-project/src/site/site.xml                |    1 +
 hadoop-yarn-project/CHANGES.txt                 |   24 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |    4 +-
 .../org/apache/hadoop/yarn/util/RMHAUtils.java  |   33 +-
 .../util/WindowsResourceCalculatorPlugin.java   |    2 +-
 .../resources/webapps/static/yarn.dt.plugins.js |   16 +-
 .../server/resourcemanager/AdminService.java    |   19 +-
 .../resourcemanager/rmnode/RMNodeImpl.java      |    8 +-
 .../scheduler/capacity/LeafQueue.java           |   17 +-
 .../resourcemanager/TestRMAdminService.java     |   49 +-
 .../resourcetracker/TestNMReconnect.java        |   67 +-
 .../server/webproxy/WebAppProxyServlet.java     |    5 +-
 .../server/webproxy/TestWebAppProxyServlet.java |   57 +
 .../amfilter/TestAmFilterInitializer.java       |   47 +
 .../src/site/markdown/ResourceManagerHA.md      |   16 +-
 115 files changed, 4134 insertions(+), 2527 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd784e97/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd784e97/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------


[23/50] [abbrv] hadoop git commit: Update CHANGES.txt for HDFS-8238.

Posted by ar...@apache.org.
Update CHANGES.txt for HDFS-8238.


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

Branch: refs/heads/HDFS-7240
Commit: 2de586f60ded874b2c962d0ca8ef2ca7cad25518
Parents: b832774
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Jun 17 16:19:45 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Wed Jun 17 16:19:45 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2de586f6/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index a01446a..6ef405b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -644,6 +644,9 @@ Release 2.8.0 - UNRELEASED
     of Block in UnderReplicatedBlocks and PendingReplicationBlocks).
     (Zhe Zhang via wang)
 
+    HDFS-8238. Move ClientProtocol to the hdfs-client.
+    (Takanobu Asanuma via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than


[26/50] [abbrv] hadoop git commit: HDFS-8589. Fix unused imports in BPServiceActor and BlockReportLeaseManager (cmccabe)

Posted by ar...@apache.org.
HDFS-8589. Fix unused imports in BPServiceActor and BlockReportLeaseManager (cmccabe)


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

Branch: refs/heads/HDFS-7240
Commit: 45ced38f10fcb9f831218b890786aaeb7987fed4
Parents: 015535d
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Wed Jun 17 17:01:42 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Wed Jun 17 17:01:42 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../hdfs/server/blockmanagement/BlockReportLeaseManager.java      | 1 -
 .../org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java    | 1 -
 3 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/45ced38f/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 6dfcd18..ef3530f3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -650,6 +650,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8446. Separate safemode related operations in GetBlockLocations().
     (wheat9)
 
+    HDFS-8589. Fix unused imports in BPServiceActor and BlockReportLeaseManager
+    (cmccabe)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45ced38f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockReportLeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockReportLeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockReportLeaseManager.java
index cd037f5..7db05c7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockReportLeaseManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockReportLeaseManager.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
-import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45ced38f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
index ea1abbd..f84dd99 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
@@ -29,7 +29,6 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.base.Joiner;


[13/50] [abbrv] hadoop git commit: YARN-3714. AM proxy filter can not get RM webapp address from yarn.resourcemanager.hostname.rm-id. Contributed by Masatake Iwasaki

Posted by ar...@apache.org.
YARN-3714. AM proxy filter can not get RM webapp address from
yarn.resourcemanager.hostname.rm-id. Contributed by Masatake Iwasaki


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

Branch: refs/heads/HDFS-7240
Commit: e27d5a13b0623e3eb43ac773eccd082b9d6fa9d0
Parents: 6a76250
Author: Xuan <xg...@apache.org>
Authored: Tue Jun 16 14:05:09 2015 -0700
Committer: Xuan <xg...@apache.org>
Committed: Tue Jun 16 14:06:16 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../org/apache/hadoop/yarn/util/RMHAUtils.java  | 33 ++++++++------
 .../amfilter/TestAmFilterInitializer.java       | 47 ++++++++++++++++++++
 3 files changed, 70 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e27d5a13/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index d0d1e06..b61eb27 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -527,6 +527,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3794. TestRMEmbeddedElector fails because of ambiguous LOG reference.
     (Chengbing Liu via devaraj)
 
+    YARN-3714. AM proxy filter can not get RM webapp address from
+    yarn.resourcemanager.hostname.rm-id. (Masatake Iwasaki via xgong)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e27d5a13/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/RMHAUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/RMHAUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/RMHAUtils.java
index a7e1ce9..2e996e9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/RMHAUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/RMHAUtils.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HAServiceTarget;
 import org.apache.hadoop.yarn.client.RMHAServiceTarget;
+import org.apache.hadoop.yarn.conf.HAUtil;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
 @Private
@@ -71,25 +72,31 @@ public class RMHAUtils {
 
   public static List<String> getRMHAWebappAddresses(
       final YarnConfiguration conf) {
+    String prefix;
+    String defaultPort;
+    if (YarnConfiguration.useHttps(conf)) {
+      prefix = YarnConfiguration.RM_WEBAPP_HTTPS_ADDRESS;
+      defaultPort = ":" + YarnConfiguration.DEFAULT_RM_WEBAPP_HTTPS_PORT;
+    } else {
+      prefix =YarnConfiguration.RM_WEBAPP_ADDRESS;
+      defaultPort = ":" + YarnConfiguration.DEFAULT_RM_WEBAPP_PORT;
+    }
     Collection<String> rmIds =
         conf.getStringCollection(YarnConfiguration.RM_HA_IDS);
     List<String> addrs = new ArrayList<String>();
-    if (YarnConfiguration.useHttps(conf)) {
-      for (String id : rmIds) {
-        String addr = conf.get(
-            YarnConfiguration.RM_WEBAPP_HTTPS_ADDRESS + "." + id);
-        if (addr != null) {
-          addrs.add(addr);
+    for (String id : rmIds) {
+      String addr = conf.get(HAUtil.addSuffix(prefix, id));
+      if (addr == null) {
+        String hostname =
+            conf.get(HAUtil.addSuffix(YarnConfiguration.RM_HOSTNAME, id));
+        if (hostname != null) {
+          addr = hostname + defaultPort;
         }
       }
-    } else {
-      for (String id : rmIds) {
-        String addr = conf.get(YarnConfiguration.RM_WEBAPP_ADDRESS + "." + id);
-        if (addr != null) {
-          addrs.add(addr);
-        }
+      if (addr != null) {
+        addrs.add(addr);
       }
     }
     return addrs;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e27d5a13/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/amfilter/TestAmFilterInitializer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/amfilter/TestAmFilterInitializer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/amfilter/TestAmFilterInitializer.java
index bec62ce..63c2cf3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/amfilter/TestAmFilterInitializer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/amfilter/TestAmFilterInitializer.java
@@ -135,6 +135,15 @@ public class TestAmFilterInitializer extends TestCase {
     assertEquals(WebAppUtils.getResolvedRMWebAppURLWithoutScheme(conf),
         proxyHosts.get(0));
 
+    // Check conf in which only RM hostname is set
+    conf = new Configuration(false);
+    conf.set(YarnConfiguration.RM_WEBAPP_ADDRESS,
+        "${yarn.resourcemanager.hostname}:8088"); // default in yarn-default.xml
+    conf.set(YarnConfiguration.RM_HOSTNAME, "host1");
+    proxyHosts = WebAppUtils.getProxyHostsAndPortsForAmFilter(conf);
+    assertEquals(1, proxyHosts.size());
+    assertEquals("host1:8088", proxyHosts.get(0));
+
     // Check PROXY_ADDRESS has priority
     conf = new Configuration(false);
     conf.set(YarnConfiguration.PROXY_ADDRESS, "host1:1000");
@@ -188,6 +197,44 @@ public class TestAmFilterInitializer extends TestCase {
     Collections.sort(proxyHosts);
     assertEquals("host5:5000", proxyHosts.get(0));
     assertEquals("host6:6000", proxyHosts.get(1));
+
+    // Check config without explicit RM_WEBAPP_ADDRESS settings (RM HA)
+    conf = new Configuration(false);
+    conf.setBoolean(YarnConfiguration.RM_HA_ENABLED, true);
+    conf.set(YarnConfiguration.RM_HA_IDS, "rm1,rm2,rm3");
+    conf.set(YarnConfiguration.RM_HOSTNAME + ".rm1", "host2");
+    conf.set(YarnConfiguration.RM_HOSTNAME + ".rm2", "host3");
+    conf.set(YarnConfiguration.RM_HOSTNAME + ".rm3", "host4");
+    conf.set(YarnConfiguration.RM_HOSTNAME + ".rm4", "dummy");
+    proxyHosts = WebAppUtils.getProxyHostsAndPortsForAmFilter(conf);
+    assertEquals(3, proxyHosts.size());
+    Collections.sort(proxyHosts);
+    assertEquals("host2:" + YarnConfiguration.DEFAULT_RM_WEBAPP_PORT,
+        proxyHosts.get(0));
+    assertEquals("host3:" + YarnConfiguration.DEFAULT_RM_WEBAPP_PORT,
+        proxyHosts.get(1));
+    assertEquals("host4:" + YarnConfiguration.DEFAULT_RM_WEBAPP_PORT,
+        proxyHosts.get(2));
+
+    // Check config without explicit RM_WEBAPP_HTTPS_ADDRESS settings (RM HA)
+    conf = new Configuration(false);
+    conf.set(YarnConfiguration.YARN_HTTP_POLICY_KEY,
+        HttpConfig.Policy.HTTPS_ONLY.toString());
+    conf.setBoolean(YarnConfiguration.RM_HA_ENABLED, true);
+    conf.set(YarnConfiguration.RM_HA_IDS, "rm1,rm2,rm3");
+    conf.set(YarnConfiguration.RM_HOSTNAME + ".rm1", "host2");
+    conf.set(YarnConfiguration.RM_HOSTNAME + ".rm2", "host3");
+    conf.set(YarnConfiguration.RM_HOSTNAME + ".rm3", "host4");
+    conf.set(YarnConfiguration.RM_HOSTNAME + ".rm4", "dummy");
+    proxyHosts = WebAppUtils.getProxyHostsAndPortsForAmFilter(conf);
+    assertEquals(3, proxyHosts.size());
+    Collections.sort(proxyHosts);
+    assertEquals("host2:" + YarnConfiguration.DEFAULT_RM_WEBAPP_HTTPS_PORT,
+        proxyHosts.get(0));
+    assertEquals("host3:" + YarnConfiguration.DEFAULT_RM_WEBAPP_HTTPS_PORT,
+        proxyHosts.get(1));
+    assertEquals("host4:" + YarnConfiguration.DEFAULT_RM_WEBAPP_HTTPS_PORT,
+        proxyHosts.get(2));
   }
 
   class MockAmFilterInitializer extends AmFilterInitializer {


[27/50] [abbrv] hadoop git commit: HDFS-8615. Correct HTTP method in WebHDFS document. Contributed by Brahma Reddy Battula.

Posted by ar...@apache.org.
HDFS-8615. Correct HTTP method in WebHDFS document. 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/1a169a26
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1a169a26
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1a169a26

Branch: refs/heads/HDFS-7240
Commit: 1a169a26bcc4e4bab7697965906cb9ca7ef8329e
Parents: 45ced38
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Jun 17 17:13:02 2015 -0700
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Jun 17 17:13:02 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a169a26/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index ef3530f3..8f563de 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -924,6 +924,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8551. Fix hdfs datanode CLI usage message.
     (Brahma Reddy Battula via xyao)
 
+    HDFS-8615. Correct HTTP method in WebHDFS document.
+    (Brahma Reddy Battula via aajisaka)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a169a26/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
index d0231a3..e8f5fee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
@@ -691,7 +691,7 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).setAcl
 
 * Submit a HTTP GET request.
 
-        curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETACLSTATUS"
+        curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETACLSTATUS"
 
     The client receives a response with a [`AclStatus` JSON object](#ACL_Status_JSON_Schema):
 
@@ -718,7 +718,7 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getAclSta
 
 * Submit a HTTP GET request.
 
-        curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=CHECKACCESS
+        curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=CHECKACCESS
                                       &fsaction=<FSACTION>
 
     The client receives a response with zero content length:
@@ -764,7 +764,7 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).removeXAt
 
 * Submit a HTTP GET request.
 
-        curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETXATTRS
+        curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETXATTRS
                                       &xattr.name=<XATTRNAME>&encoding=<ENCODING>"
 
     The client receives a response with a [`XAttrs` JSON object](#XAttrs_JSON_Schema):
@@ -788,7 +788,7 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getXAttr
 
 * Submit a HTTP GET request.
 
-        curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETXATTRS
+        curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETXATTRS
                                       &xattr.name=<XATTRNAME1>&xattr.name=<XATTRNAME2>
                                       &encoding=<ENCODING>"
 
@@ -817,7 +817,7 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getXAttrs
 
 * Submit a HTTP GET request.
 
-        curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETXATTRS
+        curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETXATTRS
                                       &encoding=<ENCODING>"
 
     The client receives a response with a [`XAttrs` JSON object](#XAttrs_JSON_Schema):
@@ -849,7 +849,7 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getXAttrs
 
 * Submit a HTTP GET request.
 
-        curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=LISTXATTRS"
+        curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=LISTXATTRS"
 
     The client receives a response with a [`XAttrNames` JSON object](#XAttrNames_JSON_Schema):
 


[44/50] [abbrv] hadoop git commit: Move HDFS-7546 to release 2.7.1 in CHANGES.txt.

Posted by ar...@apache.org.
Move HDFS-7546 to release 2.7.1 in CHANGES.txt.


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

Branch: refs/heads/HDFS-7240
Commit: bcb3c40bed572a2dd95ce7201d893d7bf59240d5
Parents: 20c03c9
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Sat Jun 20 00:39:31 2015 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Sat Jun 20 00:41:35 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bcb3c40b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 4a633fe..882bf3f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -148,9 +148,6 @@ Trunk (Unreleased)
     HDFS-7591. hdfs classpath command should support same options as hadoop
     classpath (Varun Saxena via Arpit Agarwal)
 
-    HDFS-7546. Document, and set an accepting default for
-    dfs.namenode.kerberos.principal.pattern (Harsh J via aw)
-
     HDFS-316. Balancer should run for a configurable # of iterations (Xiaoyu
     Yao via aw)
 
@@ -967,6 +964,9 @@ Release 2.7.1 - UNRELEASED
 
     HDFS-8361. Choose SSD over DISK in block placement.  (szetszwo)
 
+    HDFS-7546. Document, and set an accepting default for
+    dfs.namenode.kerberos.principal.pattern (Harsh J via aw)
+
   OPTIMIZATIONS
 
   BUG FIXES


[35/50] [abbrv] hadoop git commit: YARN-3802. Two RMNodes for the same NodeId are used in RM sometimes after NM is reconnected. Contributed by zhihai xu

Posted by ar...@apache.org.
YARN-3802. Two RMNodes for the same NodeId are used in RM sometimes
after NM is reconnected. Contributed by zhihai xu


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

Branch: refs/heads/HDFS-7240
Commit: 5b5bb8dcdc888ba1ebc7e4eba0fa0e7e79edda9a
Parents: 6e0a9f9
Author: Xuan <xg...@apache.org>
Authored: Thu Jun 18 14:37:49 2015 -0700
Committer: Xuan <xg...@apache.org>
Committed: Thu Jun 18 14:37:49 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../resourcemanager/rmnode/RMNodeImpl.java      |  8 ++-
 .../resourcetracker/TestNMReconnect.java        | 67 +++++++++++++++++++-
 3 files changed, 74 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5b5bb8dc/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index f00170e..d89c285 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -539,6 +539,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3824. Fix two minor nits in member variable properties
     of YarnConfiguration. (Ray Chiang via devaraj)
 
+    YARN-3802. Two RMNodes for the same NodeId are used in RM sometimes
+    after NM is reconnected. (zhihai xu via xgong)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5b5bb8dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
index 8a810cb..d1e6190 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
@@ -597,10 +597,14 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
         if (rmNode.getHttpPort() == newNode.getHttpPort()) {
           // Reset heartbeat ID since node just restarted.
           rmNode.getLastNodeHeartBeatResponse().setResponseId(0);
+          if (!rmNode.getTotalCapability().equals(
+              newNode.getTotalCapability())) {
+            rmNode.totalCapability = newNode.getTotalCapability();
+          }
           if (rmNode.getState().equals(NodeState.RUNNING)) {
-            // Only add new node if old state is RUNNING
+            // Only add old node if old state is RUNNING
             rmNode.context.getDispatcher().getEventHandler().handle(
-                new NodeAddedSchedulerEvent(newNode));
+                new NodeAddedSchedulerEvent(rmNode));
           }
         } else {
           // Reconnected node differs, so replace old node and start new node

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5b5bb8dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMReconnect.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/resourcetracker/TestNMReconnect.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMReconnect.java
index d16d551..b525efc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMReconnect.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMReconnect.java
@@ -25,6 +25,9 @@ import org.junit.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.ConfigurationProvider;
+import org.apache.hadoop.yarn.conf.ConfigurationProviderFactory;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.event.InlineDispatcher;
@@ -32,6 +35,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.NMLivelinessMonitor;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.NodesListManager;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
@@ -39,10 +43,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.NodeEventDi
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -51,6 +58,8 @@ public class TestNMReconnect {
       RecordFactoryProvider.getRecordFactory(null);
 
   private List<RMNodeEvent> rmNodeEvents = new ArrayList<RMNodeEvent>();
+  private Dispatcher dispatcher;
+  private RMContextImpl context;
 
   private class TestRMNodeEventDispatcher implements
       EventHandler<RMNodeEvent> {
@@ -68,12 +77,12 @@ public class TestNMReconnect {
   public void setUp() {
     Configuration conf = new Configuration();
     // Dispatcher that processes events inline
-    Dispatcher dispatcher = new InlineDispatcher();
+    dispatcher = new InlineDispatcher();
 
     dispatcher.register(RMNodeEventType.class,
         new TestRMNodeEventDispatcher());
 
-    RMContext context = new RMContextImpl(dispatcher, null,
+    context = new RMContextImpl(dispatcher, null,
         null, null, null, null, null, null, null, null);
     dispatcher.register(SchedulerEventType.class,
         new InlineDispatcher.EmptyEventHandler());
@@ -99,6 +108,11 @@ public class TestNMReconnect {
     resourceTrackerService.start();
   }
 
+  @After
+  public void tearDown() {
+    resourceTrackerService.stop();
+  }
+
   @Test
   public void testReconnect() throws Exception {
     String hostname1 = "localhost1";
@@ -126,4 +140,53 @@ public class TestNMReconnect {
     Assert.assertEquals(RMNodeEventType.RECONNECTED,
         rmNodeEvents.get(0).getType());
   }
+
+  @Test
+  public void testCompareRMNodeAfterReconnect() throws Exception {
+    Configuration yarnConf = new YarnConfiguration();
+    CapacityScheduler scheduler = new CapacityScheduler();
+    scheduler.setConf(yarnConf);
+    ConfigurationProvider configurationProvider =
+        ConfigurationProviderFactory.getConfigurationProvider(yarnConf);
+    configurationProvider.init(yarnConf);
+    context.setConfigurationProvider(configurationProvider);
+    RMNodeLabelsManager nlm = new RMNodeLabelsManager();
+    nlm.init(yarnConf);
+    nlm.start();
+    context.setNodeLabelManager(nlm);
+    scheduler.setRMContext(context);
+    scheduler.init(yarnConf);
+    scheduler.start();
+    dispatcher.register(SchedulerEventType.class, scheduler);
+
+    String hostname1 = "localhost1";
+    Resource capability = BuilderUtils.newResource(4096, 4);
+
+    RegisterNodeManagerRequest request1 = recordFactory
+        .newRecordInstance(RegisterNodeManagerRequest.class);
+    NodeId nodeId1 = NodeId.newInstance(hostname1, 0);
+    request1.setNodeId(nodeId1);
+    request1.setHttpPort(0);
+    request1.setResource(capability);
+    resourceTrackerService.registerNodeManager(request1);
+    Assert.assertNotNull(context.getRMNodes().get(nodeId1));
+    // verify Scheduler and RMContext use same RMNode reference.
+    Assert.assertTrue(scheduler.getSchedulerNode(nodeId1).getRMNode() ==
+        context.getRMNodes().get(nodeId1));
+    Assert.assertEquals(context.getRMNodes().get(nodeId1).
+        getTotalCapability(), capability);
+    Resource capability1 = BuilderUtils.newResource(2048, 2);
+    request1.setResource(capability1);
+    resourceTrackerService.registerNodeManager(request1);
+    Assert.assertNotNull(context.getRMNodes().get(nodeId1));
+    // verify Scheduler and RMContext use same RMNode reference
+    // after reconnect.
+    Assert.assertTrue(scheduler.getSchedulerNode(nodeId1).getRMNode() ==
+        context.getRMNodes().get(nodeId1));
+    // verify RMNode's capability is changed.
+    Assert.assertEquals(context.getRMNodes().get(nodeId1).
+        getTotalCapability(), capability1);
+    nlm.stop();
+    scheduler.stop();
+  }
 }


[42/50] [abbrv] hadoop git commit: HDFS-8337. Accessing httpfs via webhdfs doesn't work from a jar with kerberos. Contributed by Yongjun Zhang.

Posted by ar...@apache.org.
HDFS-8337. Accessing httpfs via webhdfs doesn't work from a jar with kerberos. 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/971dc838
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/971dc838
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/971dc838

Branch: refs/heads/HDFS-7240
Commit: 971dc838ecf8bf55a9bd64128ce2447e4613ea76
Parents: 311a417
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Fri Jun 19 23:39:13 2015 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Fri Jun 19 23:39:13 2015 -0700

----------------------------------------------------------------------
 .../hadoop/fs/http/server/HttpFSAuthenticationFilter.java       | 5 +++++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                     | 3 +++
 2 files changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/971dc838/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSAuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSAuthenticationFilter.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSAuthenticationFilter.java
index 004ff38..a68ed3b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSAuthenticationFilter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSAuthenticationFilter.java
@@ -20,8 +20,10 @@ package org.apache.hadoop.fs.http.server;
 import org.apache.commons.io.Charsets;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.web.WebHdfsConstants;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationFilter;
+import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticationHandler;
 
 import javax.servlet.FilterConfig;
 import javax.servlet.ServletException;
@@ -92,6 +94,9 @@ public class HttpFSAuthenticationFilter
     } catch (IOException ex) {
       throw new RuntimeException("Could not read HttpFS signature secret file: " + signatureSecretFile);
     }
+    setAuthHandlerClass(props);
+    props.setProperty(KerberosDelegationTokenAuthenticationHandler.TOKEN_KIND,
+        WebHdfsConstants.WEBHDFS_TOKEN_KIND.toString());
     return props;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/971dc838/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 527f6ca..78360a7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1071,6 +1071,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8626. Reserved RBW space is not released if creation of RBW File
     fails. (kanaka kumar avvaru via Arpit Agarwal)
 
+    HDFS-8337. Accessing httpfs via webhdfs doesn't work from a jar with
+    kerberos. (Yongjun Zhang)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES


[28/50] [abbrv] hadoop git commit: HDFS-6249. Output AclEntry in PBImageXmlWriter. Contributed by surendra singh lilhore.

Posted by ar...@apache.org.
HDFS-6249. Output AclEntry in PBImageXmlWriter. 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/cc432885
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/cc432885
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/cc432885

Branch: refs/heads/HDFS-7240
Commit: cc432885adb0182c2c5b3bf92edde12231fd567c
Parents: 1a169a2
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Jun 17 17:41:10 2015 -0700
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Jun 17 17:41:10 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../offlineImageViewer/PBImageXmlWriter.java    | 20 +++++++++++++---
 .../TestOfflineImageViewerForAcl.java           | 24 ++++++++++++++++++++
 3 files changed, 44 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc432885/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 8f563de..2545bcf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -653,6 +653,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8589. Fix unused imports in BPServiceActor and BlockReportLeaseManager
     (cmccabe)
 
+    HDFS-6249. Output AclEntry in PBImageXmlWriter.
+    (surendra singh lilhore via aajisaka)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc432885/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java
index f3fe886..4415c5d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java
@@ -29,6 +29,7 @@ import java.util.Comparator;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
@@ -41,6 +42,7 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FilesUnderConstructionSection.FileUnderConstructionEntry;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeDirectorySection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.AclFeatureProto;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeSymlink;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeReferenceSection;
@@ -51,7 +53,7 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.StringTableSection;
 import org.apache.hadoop.hdfs.util.XMLUtils;
 import org.apache.hadoop.util.LimitInputStream;
-
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 /**
@@ -188,7 +190,7 @@ public final class PBImageXmlWriter {
   private void dumpINodeDirectory(INodeDirectory d) {
     o("mtime", d.getModificationTime()).o("permission",
         dumpPermission(d.getPermission()));
-
+    dumpAcls(d.getAcl());
     if (d.hasDsQuota() && d.hasNsQuota()) {
       o("nsquota", d.getNsQuota()).o("dsquota", d.getDsQuota());
     }
@@ -242,7 +244,7 @@ public final class PBImageXmlWriter {
         .o("atime", f.getAccessTime())
         .o("perferredBlockSize", f.getPreferredBlockSize())
         .o("permission", dumpPermission(f.getPermission()));
-
+    dumpAcls(f.getAcl());
     if (f.getBlocksCount() > 0) {
       out.print("<blocks>");
       for (BlockProto b : f.getBlocksList()) {
@@ -263,6 +265,18 @@ public final class PBImageXmlWriter {
     }
   }
 
+  private void dumpAcls(AclFeatureProto aclFeatureProto) {
+    ImmutableList<AclEntry> aclEntryList = FSImageFormatPBINode.Loader
+        .loadAclEntries(aclFeatureProto, stringTable);
+    if (aclEntryList.size() > 0) {
+      out.print("<acls>");
+      for (AclEntry aclEntry : aclEntryList) {
+        o("acl", aclEntry.toString());
+      }
+      out.print("</acls>");
+    }
+  }
+
   private void dumpINodeSection(InputStream in) throws IOException {
     INodeSection s = INodeSection.parseDelimitedFrom(in);
     out.print("<INodeSection>");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc432885/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java
index 56963a6..7104ba1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java
@@ -17,14 +17,23 @@
  */
 package org.apache.hadoop.hdfs.tools.offlineImageViewer;
 
+import java.io.ByteArrayOutputStream;
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.PrintStream;
+import java.io.RandomAccessFile;
+import java.io.StringReader;
 import java.net.HttpURLConnection;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.util.HashMap;
 
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.parsers.SAXParser;
+import javax.xml.parsers.SAXParserFactory;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -42,6 +51,9 @@ import org.apache.hadoop.net.NetUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.xml.sax.InputSource;
+import org.xml.sax.SAXException;
+import org.xml.sax.helpers.DefaultHandler;
 
 import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
 import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT;
@@ -202,4 +214,16 @@ public class TestOfflineImageViewerForAcl {
       viewer.close();
     }
   }
+
+  @Test
+  public void testPBImageXmlWriterForAcl() throws Exception{
+    ByteArrayOutputStream output = new ByteArrayOutputStream();
+    PrintStream o = new PrintStream(output);
+    PBImageXmlWriter v = new PBImageXmlWriter(new Configuration(), o);
+    v.visit(new RandomAccessFile(originalFsimage, "r"));
+    SAXParserFactory spf = SAXParserFactory.newInstance();
+    SAXParser parser = spf.newSAXParser();
+    final String xml = output.toString();
+    parser.parse(new InputSource(new StringReader(xml)), new DefaultHandler());
+  }
 }


[34/50] [abbrv] hadoop git commit: HADOOP-12100. ImmutableFsPermission should not override applyUmask since that method doesn't modify the FsPermission (Bibin A. Chundatt via Colin P. McCabe)

Posted by ar...@apache.org.
HADOOP-12100. ImmutableFsPermission should not override applyUmask since that method doesn't modify the FsPermission (Bibin A. Chundatt via Colin P. McCabe)


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

Branch: refs/heads/HDFS-7240
Commit: 6e0a9f92fe0052d39b95a605174b3f6423c6aae7
Parents: 1c13519
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Thu Jun 18 11:30:29 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Thu Jun 18 11:39:05 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt               | 4 ++++
 .../java/org/apache/hadoop/fs/permission/FsPermission.java    | 7 ++-----
 2 files changed, 6 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e0a9f92/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 3430da6..2f5eda3 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -914,6 +914,10 @@ Release 2.7.1 - UNRELEASED
     HADOOP-12078. The default retry policy does not handle RetriableException
     correctly. (Arpit Agarwal)
 
+    HADOOP-12100. ImmutableFsPermission should not override applyUmask since
+    that method doesn't modify the FsPermission (Bibin A Chundatt via Colin P.
+    McCabe)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e0a9f92/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
index 264a095..0258293 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
@@ -375,13 +375,10 @@ public class FsPermission implements Writable {
     public ImmutableFsPermission(short permission) {
       super(permission);
     }
-    @Override
-    public FsPermission applyUMask(FsPermission umask) {
-      throw new UnsupportedOperationException();
-    }
+
     @Override
     public void readFields(DataInput in) throws IOException {
       throw new UnsupportedOperationException();
-    }    
+    }
   }
 }


[24/50] [abbrv] hadoop git commit: YARN-3804. Both RM are on standBy state when kerberos user not in yarn.admin.acl. Contributed by Varun Saxena

Posted by ar...@apache.org.
YARN-3804. Both RM are on standBy state when kerberos user not in yarn.admin.acl. Contributed by Varun Saxena


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

Branch: refs/heads/HDFS-7240
Commit: a826d432f9b45550cc5ab79ef63ca39b176dabb2
Parents: 2de586f
Author: Xuan <xg...@apache.org>
Authored: Wed Jun 17 16:23:27 2015 -0700
Committer: Xuan <xg...@apache.org>
Committed: Wed Jun 17 16:23:27 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../server/resourcemanager/AdminService.java    | 19 +++++---
 .../resourcemanager/TestRMAdminService.java     | 49 +++++++++++++++++++-
 3 files changed, 63 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a826d432/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index afe76bd..243edb3 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -681,6 +681,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3764. CapacityScheduler should forbid moving LeafQueue from one parent
     to another. (Wangda Tan via jianhe)
 
+    YARN-3804. Both RM are on standBy state when kerberos user not in yarn.admin.acl
+    (Varun Saxena via xgong)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a826d432/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
index 1ee8b3b..e5bb6e5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
@@ -112,6 +112,8 @@ public class AdminService extends CompositeService implements
   private final RecordFactory recordFactory = 
     RecordFactoryProvider.getRecordFactory(null);
 
+  private UserGroupInformation daemonUser;
+
   @VisibleForTesting
   boolean isDistributedNodeLabelConfiguration = false;
 
@@ -138,10 +140,9 @@ public class AdminService extends CompositeService implements
         YarnConfiguration.RM_ADMIN_ADDRESS,
         YarnConfiguration.DEFAULT_RM_ADMIN_ADDRESS,
         YarnConfiguration.DEFAULT_RM_ADMIN_PORT);
+    daemonUser = UserGroupInformation.getCurrentUser();
     authorizer = YarnAuthorizationProvider.getInstance(conf);
-    authorizer.setAdmins(new AccessControlList(conf.get(
-      YarnConfiguration.YARN_ADMIN_ACL,
-        YarnConfiguration.DEFAULT_YARN_ADMIN_ACL)), UserGroupInformation
+    authorizer.setAdmins(getAdminAclList(conf), UserGroupInformation
         .getCurrentUser());
     rmId = conf.get(YarnConfiguration.RM_HA_ID);
 
@@ -151,6 +152,14 @@ public class AdminService extends CompositeService implements
     super.serviceInit(conf);
   }
 
+  private AccessControlList getAdminAclList(Configuration conf) {
+    AccessControlList aclList = new AccessControlList(conf.get(
+        YarnConfiguration.YARN_ADMIN_ACL,
+        YarnConfiguration.DEFAULT_YARN_ADMIN_ACL));
+    aclList.addUser(daemonUser.getShortUserName());
+    return aclList;
+  }
+
   @Override
   protected void serviceStart() throws Exception {
     startServer();
@@ -470,9 +479,7 @@ public class AdminService extends CompositeService implements
     Configuration conf =
         getConfiguration(new Configuration(false),
             YarnConfiguration.YARN_SITE_CONFIGURATION_FILE);
-    authorizer.setAdmins(new AccessControlList(conf.get(
-      YarnConfiguration.YARN_ADMIN_ACL,
-        YarnConfiguration.DEFAULT_YARN_ADMIN_ACL)), UserGroupInformation
+    authorizer.setAdmins(getAdminAclList(conf), UserGroupInformation
         .getCurrentUser());
     RMAuditLogger.logSuccess(user.getShortUserName(), argName,
         "AdminService");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a826d432/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.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/TestRMAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
index fe0b8a8..0a05c91 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
@@ -38,12 +38,14 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.GroupMappingServiceProvider;
 import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
+import org.apache.hadoop.yarn.LocalConfigurationProvider;
 import org.apache.hadoop.yarn.api.records.DecommissionType;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.HAUtil;
@@ -208,7 +210,8 @@ public class TestRMAdminService {
         rm.adminService.getAccessControlList().getAclString().trim();
 
     Assert.assertTrue(!aclStringAfter.equals(aclStringBefore));
-    Assert.assertEquals(aclStringAfter, "world:anyone:rwcda");
+    Assert.assertEquals(aclStringAfter, "world:anyone:rwcda," +
+        UserGroupInformation.getCurrentUser().getShortUserName());
   }
 
   @Test
@@ -695,7 +698,8 @@ public class TestRMAdminService {
       String aclStringAfter =
           resourceManager.adminService.getAccessControlList()
               .getAclString().trim();
-      Assert.assertEquals(aclStringAfter, "world:anyone:rwcda");
+      Assert.assertEquals(aclStringAfter, "world:anyone:rwcda," +
+          UserGroupInformation.getCurrentUser().getShortUserName());
 
       // validate values for queue configuration
       CapacityScheduler cs =
@@ -761,6 +765,47 @@ public class TestRMAdminService {
     }
   }
 
+  /* For verifying fix for YARN-3804 */
+  @Test
+  public void testRefreshAclWithDaemonUser() throws Exception {
+    String daemonUser =
+        UserGroupInformation.getCurrentUser().getShortUserName();
+    configuration.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,
+        "org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider");
+
+    uploadDefaultConfiguration();
+    YarnConfiguration yarnConf = new YarnConfiguration();
+    yarnConf.set(YarnConfiguration.YARN_ADMIN_ACL, daemonUser + "xyz");
+    uploadConfiguration(yarnConf, "yarn-site.xml");
+
+    try {
+      rm = new MockRM(configuration);
+      rm.init(configuration);
+      rm.start();
+    } catch(Exception ex) {
+      fail("Should not get any exceptions");
+    }
+
+    assertEquals(daemonUser + "xyz," + daemonUser,
+        rm.adminService.getAccessControlList().getAclString().trim());
+
+    yarnConf = new YarnConfiguration();
+    yarnConf.set(YarnConfiguration.YARN_ADMIN_ACL, daemonUser + "abc");
+    uploadConfiguration(yarnConf, "yarn-site.xml");
+    try {
+      rm.adminService.refreshAdminAcls(RefreshAdminAclsRequest.newInstance());
+    } catch (YarnException e) {
+      if (e.getCause() != null &&
+          e.getCause() instanceof AccessControlException) {
+        fail("Refresh should not have failed due to incorrect ACL");
+      }
+      throw e;
+    }
+
+    assertEquals(daemonUser + "abc," + daemonUser,
+        rm.adminService.getAccessControlList().getAclString().trim());
+  }
+
   @Test
   public void testModifyLabelsOnNodesWithDistributedConfigurationDisabled()
       throws IOException, YarnException {


[20/50] [abbrv] hadoop git commit: HDFS-8608. Merge HDFS-7912 to trunk and branch-2 (track BlockInfo instead of Block in UnderReplicatedBlocks and PendingReplicationBlocks). Contributed by Zhe Zhang.

Posted by ar...@apache.org.
HDFS-8608. Merge HDFS-7912 to trunk and branch-2 (track BlockInfo instead of Block in UnderReplicatedBlocks and PendingReplicationBlocks). Contributed by Zhe Zhang.


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

Branch: refs/heads/HDFS-7240
Commit: 6e3fcffe291faec40fa9214f4880a35a952836c4
Parents: ebb9a82
Author: Andrew Wang <wa...@apache.org>
Authored: Wed Jun 17 08:05:44 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Wed Jun 17 08:05:44 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  4 ++
 .../server/blockmanagement/BlockManager.java    | 54 ++++++++-------
 .../PendingReplicationBlocks.java               | 51 +++++++-------
 .../blockmanagement/UnderReplicatedBlocks.java  | 57 ++++++++--------
 .../hdfs/server/namenode/FSDirAttrOp.java       |  8 +--
 .../hdfs/server/namenode/FSNamesystem.java      | 21 +++---
 .../hadoop/hdfs/server/namenode/INode.java      | 12 ++--
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  4 +-
 .../hdfs/server/namenode/NamenodeFsck.java      | 58 ++++++++--------
 .../hadoop/hdfs/server/namenode/SafeMode.java   |  4 +-
 .../blockmanagement/BlockManagerTestUtil.java   |  5 +-
 .../blockmanagement/TestBlockManager.java       |  8 +--
 .../server/blockmanagement/TestNodeCount.java   |  3 +-
 .../TestOverReplicatedBlocks.java               |  5 +-
 .../blockmanagement/TestPendingReplication.java | 27 +++++---
 .../TestRBWBlockInvalidation.java               |  4 +-
 .../blockmanagement/TestReplicationPolicy.java  | 70 +++++++++++---------
 .../TestUnderReplicatedBlockQueues.java         | 16 +++--
 .../datanode/TestReadOnlySharedStorage.java     | 11 +--
 .../namenode/TestProcessCorruptBlocks.java      |  5 +-
 20 files changed, 231 insertions(+), 196 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e3fcffe/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 07cd4a8..a01446a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -640,6 +640,10 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-7164. Feature documentation for HDFS-6581. (Arpit Agarwal)
 
+    HDFS-9608. Merge HDFS-7912 to trunk and branch-2 (track BlockInfo instead
+    of Block in UnderReplicatedBlocks and PendingReplicationBlocks).
+    (Zhe Zhang via wang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e3fcffe/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 ebc9017..824801f 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
@@ -588,7 +588,7 @@ public class BlockManager {
   /**
    * @return true if the block has minimum replicas
    */
-  public boolean checkMinReplication(Block block) {
+  public boolean checkMinReplication(BlockInfo block) {
     return (countNodes(block).liveReplicas() >= minReplication);
   }
 
@@ -1310,7 +1310,7 @@ public class BlockManager {
    * @return number of blocks scheduled for replication during this iteration.
    */
   int computeReplicationWork(int blocksToProcess) {
-    List<List<Block>> blocksToReplicate = null;
+    List<List<BlockInfo>> blocksToReplicate = null;
     namesystem.writeLock();
     try {
       // Choose the blocks to be replicated
@@ -1328,7 +1328,7 @@ public class BlockManager {
    * @return the number of blocks scheduled for replication
    */
   @VisibleForTesting
-  int computeReplicationWorkForBlocks(List<List<Block>> blocksToReplicate) {
+  int computeReplicationWorkForBlocks(List<List<BlockInfo>> blocksToReplicate) {
     int requiredReplication, numEffectiveReplicas;
     List<DatanodeDescriptor> containingNodes;
     DatanodeDescriptor srcNode;
@@ -1342,7 +1342,7 @@ public class BlockManager {
     try {
       synchronized (neededReplications) {
         for (int priority = 0; priority < blocksToReplicate.size(); priority++) {
-          for (Block block : blocksToReplicate.get(priority)) {
+          for (BlockInfo block : blocksToReplicate.get(priority)) {
             // block should belong to a file
             bc = blocksMap.getBlockCollection(block);
             // abandoned block or block reopened for append
@@ -1426,7 +1426,7 @@ public class BlockManager {
         }
 
         synchronized (neededReplications) {
-          Block block = rw.block;
+          BlockInfo block = rw.block;
           int priority = rw.priority;
           // Recheck since global lock was released
           // block should belong to a file
@@ -1688,7 +1688,7 @@ public class BlockManager {
    * and put them back into the neededReplication queue
    */
   private void processPendingReplications() {
-    Block[] timedOutItems = pendingReplications.getTimedOutBlocks();
+    BlockInfo[] timedOutItems = pendingReplications.getTimedOutBlocks();
     if (timedOutItems != null) {
       namesystem.writeLock();
       try {
@@ -2895,13 +2895,13 @@ public class BlockManager {
   
   /** Set replication for the blocks. */
   public void setReplication(final short oldRepl, final short newRepl,
-      final String src, final Block... blocks) {
+      final String src, final BlockInfo... blocks) {
     if (newRepl == oldRepl) {
       return;
     }
 
     // update needReplication priority queues
-    for(Block b : blocks) {
+    for(BlockInfo b : blocks) {
       updateNeededReplications(b, 0, newRepl-oldRepl);
     }
       
@@ -2909,7 +2909,7 @@ public class BlockManager {
       // old replication > the new one; need to remove copies
       LOG.info("Decreasing replication from " + oldRepl + " to " + newRepl
           + " for " + src);
-      for(Block b : blocks) {
+      for(BlockInfo b : blocks) {
         processOverReplicatedBlock(b, newRepl, null, null);
       }
     } else { // replication factor is increased
@@ -3092,7 +3092,8 @@ public class BlockManager {
     blockLog.debug("BLOCK* removeStoredBlock: {} from {}", block, node);
     assert (namesystem.hasWriteLock());
     {
-      if (!blocksMap.removeNode(block, node)) {
+      BlockInfo storedBlock = getStoredBlock(block);
+      if (storedBlock == null || !blocksMap.removeNode(storedBlock, node)) {
         blockLog.debug("BLOCK* removeStoredBlock: {} has already been" +
             " removed from node {}", block, node);
         return;
@@ -3106,8 +3107,8 @@ public class BlockManager {
       //
       BlockCollection bc = blocksMap.getBlockCollection(block);
       if (bc != null) {
-        namesystem.decrementSafeBlockCount(block);
-        updateNeededReplications(block, -1, 0);
+        namesystem.decrementSafeBlockCount(storedBlock);
+        updateNeededReplications(storedBlock, -1, 0);
       }
 
       //
@@ -3181,7 +3182,10 @@ public class BlockManager {
     //
     // Modify the blocks->datanode map and node's map.
     //
-    pendingReplications.decrement(block, node);
+    BlockInfo storedBlock = getStoredBlock(block);
+    if (storedBlock != null) {
+      pendingReplications.decrement(getStoredBlock(block), node);
+    }
     processAndHandleReportedBlock(storageInfo, block, ReplicaState.FINALIZED,
         delHintNode);
   }
@@ -3293,7 +3297,7 @@ public class BlockManager {
    * Return the number of nodes hosting a given block, grouped
    * by the state of those replicas.
    */
-  public NumberReplicas countNodes(Block b) {
+  public NumberReplicas countNodes(BlockInfo b) {
     int decommissioned = 0;
     int decommissioning = 0;
     int live = 0;
@@ -3326,12 +3330,12 @@ public class BlockManager {
   }
 
   /** 
-   * Simpler, faster form of {@link #countNodes(Block)} that only returns the number
+   * Simpler, faster form of {@link #countNodes} that only returns the number
    * of live nodes.  If in startup safemode (or its 30-sec extension period),
    * then it gains speed by ignoring issues of excess replicas or nodes
    * that are decommissioned or in process of becoming decommissioned.
-   * If not in startup, then it calls {@link #countNodes(Block)} instead.
-   * 
+   * If not in startup, then it calls {@link #countNodes} instead.
+   *
    * @param b - the block being tested
    * @return count of live nodes for this block
    */
@@ -3360,10 +3364,10 @@ public class BlockManager {
     if (!namesystem.isPopulatingReplQueues()) {
       return;
     }
-    final Iterator<? extends Block> it = srcNode.getBlockIterator();
+    final Iterator<BlockInfo> it = srcNode.getBlockIterator();
     int numOverReplicated = 0;
     while(it.hasNext()) {
-      final Block block = it.next();
+      final BlockInfo block = it.next();
       BlockCollection bc = blocksMap.getBlockCollection(block);
       short expectedReplication = bc.getPreferredBlockReplication();
       NumberReplicas num = countNodes(block);
@@ -3427,7 +3431,7 @@ public class BlockManager {
     return blocksMap.size();
   }
 
-  public void removeBlock(Block block) {
+  public void removeBlock(BlockInfo block) {
     assert namesystem.hasWriteLock();
     // No need to ACK blocks that are being removed entirely
     // from the namespace, since the removal of the associated
@@ -3448,7 +3452,7 @@ public class BlockManager {
   }
 
   /** updates a block in under replication queue */
-  private void updateNeededReplications(final Block block,
+  private void updateNeededReplications(final BlockInfo block,
       final int curReplicasDelta, int expectedReplicasDelta) {
     namesystem.writeLock();
     try {
@@ -3480,7 +3484,7 @@ public class BlockManager {
    */
   public void checkReplication(BlockCollection bc) {
     final short expected = bc.getPreferredBlockReplication();
-    for (Block block : bc.getBlocks()) {
+    for (BlockInfo block : bc.getBlocks()) {
       final NumberReplicas n = countNodes(block);
       if (isNeededReplication(block, expected, n.liveReplicas())) { 
         neededReplications.add(block, n.liveReplicas(),
@@ -3682,7 +3686,7 @@ public class BlockManager {
   /**
    * Return an iterator over the set of blocks for which there are no replicas.
    */
-  public Iterator<Block> getCorruptReplicaBlockIterator() {
+  public Iterator<BlockInfo> getCorruptReplicaBlockIterator() {
     return neededReplications.iterator(
         UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
   }
@@ -3807,7 +3811,7 @@ public class BlockManager {
 
   private static class ReplicationWork {
 
-    private final Block block;
+    private final BlockInfo block;
     private final BlockCollection bc;
 
     private final DatanodeDescriptor srcNode;
@@ -3818,7 +3822,7 @@ public class BlockManager {
     private DatanodeStorageInfo targets[];
     private final int priority;
 
-    public ReplicationWork(Block block,
+    public ReplicationWork(BlockInfo block,
         BlockCollection bc,
         DatanodeDescriptor srcNode,
         List<DatanodeDescriptor> containingNodes,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e3fcffe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
index 796b878..04232cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
@@ -23,6 +23,7 @@ import java.io.PrintWriter;
 import java.sql.Time;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -46,8 +47,8 @@ import org.slf4j.Logger;
 class PendingReplicationBlocks {
   private static final Logger LOG = BlockManager.LOG;
 
-  private final Map<Block, PendingBlockInfo> pendingReplications;
-  private final ArrayList<Block> timedOutItems;
+  private final Map<BlockInfo, PendingBlockInfo> pendingReplications;
+  private final ArrayList<BlockInfo> timedOutItems;
   Daemon timerThread = null;
   private volatile boolean fsRunning = true;
 
@@ -62,8 +63,8 @@ class PendingReplicationBlocks {
     if ( timeoutPeriod > 0 ) {
       this.timeout = timeoutPeriod;
     }
-    pendingReplications = new HashMap<Block, PendingBlockInfo>();
-    timedOutItems = new ArrayList<Block>();
+    pendingReplications = new HashMap<>();
+    timedOutItems = new ArrayList<>();
   }
 
   void start() {
@@ -76,7 +77,7 @@ class PendingReplicationBlocks {
    * @param block The corresponding block
    * @param targets The DataNodes where replicas of the block should be placed
    */
-  void increment(Block block, DatanodeDescriptor[] targets) {
+  void increment(BlockInfo block, DatanodeDescriptor[] targets) {
     synchronized (pendingReplications) {
       PendingBlockInfo found = pendingReplications.get(block);
       if (found == null) {
@@ -93,9 +94,9 @@ class PendingReplicationBlocks {
    * Decrement the number of pending replication requests
    * for this block.
    * 
-   * @param The DataNode that finishes the replication
+   * @param dn The DataNode that finishes the replication
    */
-  void decrement(Block block, DatanodeDescriptor dn) {
+  void decrement(BlockInfo block, DatanodeDescriptor dn) {
     synchronized (pendingReplications) {
       PendingBlockInfo found = pendingReplications.get(block);
       if (found != null) {
@@ -115,7 +116,7 @@ class PendingReplicationBlocks {
    * @param block The given block whose pending replication requests need to be
    *              removed
    */
-  void remove(Block block) {
+  void remove(BlockInfo block) {
     synchronized (pendingReplications) {
       pendingReplications.remove(block);
     }
@@ -138,7 +139,7 @@ class PendingReplicationBlocks {
   /**
    * How many copies of this block is pending replication?
    */
-  int getNumReplicas(Block block) {
+  int getNumReplicas(BlockInfo block) {
     synchronized (pendingReplications) {
       PendingBlockInfo found = pendingReplications.get(block);
       if (found != null) {
@@ -153,13 +154,13 @@ class PendingReplicationBlocks {
    * replication requests. Returns null if no blocks have
    * timed out.
    */
-  Block[] getTimedOutBlocks() {
+  BlockInfo[] getTimedOutBlocks() {
     synchronized (timedOutItems) {
       if (timedOutItems.size() <= 0) {
         return null;
       }
-      Block[] blockList = timedOutItems.toArray(
-          new Block[timedOutItems.size()]);
+      BlockInfo[] blockList = timedOutItems.toArray(
+          new BlockInfo[timedOutItems.size()]);
       timedOutItems.clear();
       return blockList;
     }
@@ -179,7 +180,7 @@ class PendingReplicationBlocks {
     PendingBlockInfo(DatanodeDescriptor[] targets) {
       this.timeStamp = monotonicNow();
       this.targets = targets == null ? new ArrayList<DatanodeDescriptor>()
-          : new ArrayList<DatanodeDescriptor>(Arrays.asList(targets));
+          : new ArrayList<>(Arrays.asList(targets));
     }
 
     long getTimeStamp() {
@@ -192,9 +193,7 @@ class PendingReplicationBlocks {
 
     void incrementReplicas(DatanodeDescriptor... newTargets) {
       if (newTargets != null) {
-        for (DatanodeDescriptor dn : newTargets) {
-          targets.add(dn);
-        }
+        Collections.addAll(targets, newTargets);
       }
     }
 
@@ -232,17 +231,17 @@ class PendingReplicationBlocks {
      */
     void pendingReplicationCheck() {
       synchronized (pendingReplications) {
-        Iterator<Map.Entry<Block, PendingBlockInfo>> iter =
+        Iterator<Map.Entry<BlockInfo, PendingBlockInfo>> iter =
                                     pendingReplications.entrySet().iterator();
         long now = monotonicNow();
         if(LOG.isDebugEnabled()) {
           LOG.debug("PendingReplicationMonitor checking Q");
         }
         while (iter.hasNext()) {
-          Map.Entry<Block, PendingBlockInfo> entry = iter.next();
+          Map.Entry<BlockInfo, PendingBlockInfo> entry = iter.next();
           PendingBlockInfo pendingBlock = entry.getValue();
           if (now > pendingBlock.getTimeStamp() + timeout) {
-            Block block = entry.getKey();
+            BlockInfo block = entry.getKey();
             synchronized (timedOutItems) {
               timedOutItems.add(block);
             }
@@ -275,16 +274,14 @@ class PendingReplicationBlocks {
     synchronized (pendingReplications) {
       out.println("Metasave: Blocks being replicated: " +
                   pendingReplications.size());
-      Iterator<Map.Entry<Block, PendingBlockInfo>> iter =
-                                  pendingReplications.entrySet().iterator();
-      while (iter.hasNext()) {
-        Map.Entry<Block, PendingBlockInfo> entry = iter.next();
+      for (Map.Entry<BlockInfo, PendingBlockInfo> entry :
+          pendingReplications.entrySet()) {
         PendingBlockInfo pendingBlock = entry.getValue();
         Block block = entry.getKey();
-        out.println(block + 
-                    " StartTime: " + new Time(pendingBlock.timeStamp) +
-                    " NumReplicaInProgress: " + 
-                    pendingBlock.getNumReplicas());
+        out.println(block +
+            " StartTime: " + new Time(pendingBlock.timeStamp) +
+            " NumReplicaInProgress: " +
+            pendingBlock.getNumReplicas());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e3fcffe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
index 1daa0ee..000416e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 
@@ -35,7 +34,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
  *
  * <p/>
  * The policy for choosing which priority to give added blocks
- * is implemented in {@link #getPriority(Block, int, int, int)}.
+ * is implemented in {@link #getPriority(int, int, int)}.
  * </p>
  * <p>The queue order is as follows:</p>
  * <ol>
@@ -62,7 +61,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
  *   blocks that are not corrupt higher priority.</li>
  * </ol>
  */
-class UnderReplicatedBlocks implements Iterable<Block> {
+class UnderReplicatedBlocks implements Iterable<BlockInfo> {
   /** The total number of queues : {@value} */
   static final int LEVEL = 5;
   /** The queue with the highest priority: {@value} */
@@ -78,8 +77,8 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   /** The queue for corrupt blocks: {@value} */
   static final int QUEUE_WITH_CORRUPT_BLOCKS = 4;
   /** the queues themselves */
-  private final List<LightWeightLinkedSet<Block>> priorityQueues
-      = new ArrayList<LightWeightLinkedSet<Block>>(LEVEL);
+  private final List<LightWeightLinkedSet<BlockInfo>> priorityQueues
+      = new ArrayList<>(LEVEL);
 
   /** The number of corrupt blocks with replication factor 1 */
   private int corruptReplOneBlocks = 0;
@@ -87,7 +86,7 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   /** Create an object. */
   UnderReplicatedBlocks() {
     for (int i = 0; i < LEVEL; i++) {
-      priorityQueues.add(new LightWeightLinkedSet<Block>());
+      priorityQueues.add(new LightWeightLinkedSet<BlockInfo>());
     }
   }
 
@@ -131,8 +130,8 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   }
 
   /** Check if a block is in the neededReplication queue */
-  synchronized boolean contains(Block block) {
-    for(LightWeightLinkedSet<Block> set : priorityQueues) {
+  synchronized boolean contains(BlockInfo block) {
+    for(LightWeightLinkedSet<BlockInfo> set : priorityQueues) {
       if (set.contains(block)) {
         return true;
       }
@@ -141,13 +140,11 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   }
 
   /** Return the priority of a block
-   * @param block a under replicated block
    * @param curReplicas current number of replicas of the block
    * @param expectedReplicas expected number of replicas of the block
    * @return the priority for the blocks, between 0 and ({@link #LEVEL}-1)
    */
-  private int getPriority(Block block,
-                          int curReplicas, 
+  private int getPriority(int curReplicas,
                           int decommissionedReplicas,
                           int expectedReplicas) {
     assert curReplicas >= 0 : "Negative replicas!";
@@ -183,12 +180,12 @@ class UnderReplicatedBlocks implements Iterable<Block> {
    * @param expectedReplicas expected number of replicas of the block
    * @return true if the block was added to a queue.
    */
-  synchronized boolean add(Block block,
-                           int curReplicas, 
+  synchronized boolean add(BlockInfo block,
+                           int curReplicas,
                            int decomissionedReplicas,
                            int expectedReplicas) {
     assert curReplicas >= 0 : "Negative replicas!";
-    int priLevel = getPriority(block, curReplicas, decomissionedReplicas,
+    int priLevel = getPriority(curReplicas, decomissionedReplicas,
                                expectedReplicas);
     if(priorityQueues.get(priLevel).add(block)) {
       if (priLevel == QUEUE_WITH_CORRUPT_BLOCKS &&
@@ -207,11 +204,11 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   }
 
   /** remove a block from a under replication queue */
-  synchronized boolean remove(Block block, 
-                              int oldReplicas, 
+  synchronized boolean remove(BlockInfo block,
+                              int oldReplicas,
                               int decommissionedReplicas,
                               int oldExpectedReplicas) {
-    int priLevel = getPriority(block, oldReplicas, 
+    int priLevel = getPriority(oldReplicas,
                                decommissionedReplicas,
                                oldExpectedReplicas);
     boolean removedBlock = remove(block, priLevel);
@@ -241,8 +238,8 @@ class UnderReplicatedBlocks implements Iterable<Block> {
    * @param priLevel expected privilege level
    * @return true if the block was found and removed from one of the priority queues
    */
-  boolean remove(Block block, int priLevel) {
-    if(priLevel >= 0 && priLevel < LEVEL 
+  boolean remove(BlockInfo block, int priLevel) {
+    if(priLevel >= 0 && priLevel < LEVEL
         && priorityQueues.get(priLevel).remove(block)) {
       NameNode.blockStateChangeLog.debug(
         "BLOCK* NameSystem.UnderReplicationBlock.remove: Removing block {}" +
@@ -279,14 +276,16 @@ class UnderReplicatedBlocks implements Iterable<Block> {
    * @param curReplicasDelta the change in the replicate count from before
    * @param expectedReplicasDelta the change in the expected replica count from before
    */
-  synchronized void update(Block block, int curReplicas,
+  synchronized void update(BlockInfo block, int curReplicas,
                            int decommissionedReplicas,
                            int curExpectedReplicas,
                            int curReplicasDelta, int expectedReplicasDelta) {
     int oldReplicas = curReplicas-curReplicasDelta;
     int oldExpectedReplicas = curExpectedReplicas-expectedReplicasDelta;
-    int curPri = getPriority(block, curReplicas, decommissionedReplicas, curExpectedReplicas);
-    int oldPri = getPriority(block, oldReplicas, decommissionedReplicas, oldExpectedReplicas);
+    int curPri = getPriority(curReplicas, decommissionedReplicas,
+        curExpectedReplicas);
+    int oldPri = getPriority(oldReplicas, decommissionedReplicas,
+        oldExpectedReplicas);
     if(NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("UnderReplicationBlocks.update " + 
         block +
@@ -336,12 +335,12 @@ class UnderReplicatedBlocks implements Iterable<Block> {
    * @return Return a list of block lists to be replicated. The block list index
    *         represents its replication priority.
    */
-  public synchronized List<List<Block>> chooseUnderReplicatedBlocks(
+  public synchronized List<List<BlockInfo>> chooseUnderReplicatedBlocks(
       int blocksToProcess) {
     // initialize data structure for the return value
-    List<List<Block>> blocksToReplicate = new ArrayList<List<Block>>(LEVEL);
+    List<List<BlockInfo>> blocksToReplicate = new ArrayList<>(LEVEL);
     for (int i = 0; i < LEVEL; i++) {
-      blocksToReplicate.add(new ArrayList<Block>());
+      blocksToReplicate.add(new ArrayList<BlockInfo>());
     }
 
     if (size() == 0) { // There are no blocks to collect.
@@ -364,7 +363,7 @@ class UnderReplicatedBlocks implements Iterable<Block> {
       // Loop through all remaining blocks in the list.
       while (blockCount < blocksToProcess
           && neededReplicationsIterator.hasNext()) {
-        Block block = neededReplicationsIterator.next();
+        BlockInfo block = neededReplicationsIterator.next();
         blocksToReplicate.get(priority).add(block);
         blockCount++;
       }
@@ -396,10 +395,10 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   /**
    * An iterator over blocks.
    */
-  class BlockIterator implements Iterator<Block> {
+  class BlockIterator implements Iterator<BlockInfo> {
     private int level;
     private boolean isIteratorForLevel = false;
-    private final List<Iterator<Block>> iterators = new ArrayList<Iterator<Block>>();
+    private final List<Iterator<BlockInfo>> iterators = new ArrayList<>();
 
     /**
      * Construct an iterator over all queues.
@@ -431,7 +430,7 @@ class UnderReplicatedBlocks implements Iterable<Block> {
     }
 
     @Override
-    public Block next() {
+    public BlockInfo next() {
       if (isIteratorForLevel) {
         return iterators.get(0).next();
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e3fcffe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
index 879738d..3b07320 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
@@ -25,12 +25,12 @@ import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.util.EnumCounters;
@@ -148,8 +148,8 @@ public class FSDirAttrOp {
       }
 
       final short[] blockRepls = new short[2]; // 0: old, 1: new
-      final Block[] blocks = unprotectedSetReplication(fsd, src, replication,
-                                                       blockRepls);
+      final BlockInfo[] blocks = unprotectedSetReplication(fsd, src,
+          replication, blockRepls);
       isFile = blocks != null;
       if (isFile) {
         fsd.getEditLog().logSetReplication(src, replication);
@@ -375,7 +375,7 @@ public class FSDirAttrOp {
     }
   }
 
-  static Block[] unprotectedSetReplication(
+  static BlockInfo[] unprotectedSetReplication(
       FSDirectory fsd, String src, short replication, short[] blockRepls)
       throws QuotaExceededException, UnresolvedLinkException,
              SnapshotAccessControlException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e3fcffe/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 40c71ea..b97776a 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
@@ -3170,8 +3170,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    *          of blocks that need to be removed from blocksMap
    */
   void removeBlocks(BlocksMapUpdateInfo blocks) {
-    List<Block> toDeleteList = blocks.getToDeleteList();
-    Iterator<Block> iter = toDeleteList.iterator();
+    List<BlockInfo> toDeleteList = blocks.getToDeleteList();
+    Iterator<BlockInfo> iter = toDeleteList.iterator();
     while (iter.hasNext()) {
       writeLock();
       try {
@@ -3227,12 +3227,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     boolean trackBlockCounts = isSafeModeTrackingBlocks();
     int numRemovedComplete = 0, numRemovedSafe = 0;
 
-    for (Block b : blocks.getToDeleteList()) {
+    for (BlockInfo b : blocks.getToDeleteList()) {
       if (trackBlockCounts) {
-        BlockInfo bi = getStoredBlock(b);
-        if (bi.isComplete()) {
+        if (b.isComplete()) {
           numRemovedComplete++;
-          if (bi.numNodes() >= blockManager.minReplication) {
+          if (blockManager.checkMinReplication(b)) {
             numRemovedSafe++;
           }
         }
@@ -4151,7 +4150,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       boolean changed = false;
       writeLock();
       try {
-        final Iterator<Block> it = blockManager.getCorruptReplicaBlockIterator();
+        final Iterator<BlockInfo> it =
+            blockManager.getCorruptReplicaBlockIterator();
 
         while (it.hasNext()) {
           Block b = it.next();
@@ -5093,7 +5093,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @Override
-  public void decrementSafeBlockCount(Block b) {
+  public void decrementSafeBlockCount(BlockInfo b) {
     // safeMode is volatile, and may be set to null at any time
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null) // mostly true
@@ -5918,7 +5918,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
       // print a limited # of corrupt files per call
 
-      final Iterator<Block> blkIterator = blockManager.getCorruptReplicaBlockIterator();
+      final Iterator<BlockInfo> blkIterator =
+          blockManager.getCorruptReplicaBlockIterator();
 
       int skip = getIntCookie(cookieTab[0]);
       for (int i = 0; i < skip && blkIterator.hasNext(); i++) {
@@ -5926,7 +5927,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
 
       while (blkIterator.hasNext()) {
-        Block blk = blkIterator.next();
+        BlockInfo blk = blkIterator.next();
         final INode inode = (INode)blockManager.getBlockCollection(blk);
         skip++;
         if (inode != null && blockManager.countNodes(blk).liveReplicas() == 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e3fcffe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
index cf38fa5..8a2d17a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
@@ -34,9 +34,9 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
@@ -950,8 +950,8 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     /**
      * The list of blocks that need to be removed from blocksMap
      */
-    private final List<Block> toDeleteList;
-    
+    private final List<BlockInfo> toDeleteList;
+
     public BlocksMapUpdateInfo() {
       toDeleteList = new ChunkedArrayList<>();
     }
@@ -959,7 +959,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     /**
      * @return The list of blocks that need to be removed from blocksMap
      */
-    public List<Block> getToDeleteList() {
+    public List<BlockInfo> getToDeleteList() {
       return toDeleteList;
     }
     
@@ -968,12 +968,12 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
      * {@link BlocksMapUpdateInfo#toDeleteList}
      * @param toDelete the to-be-deleted block
      */
-    public void addDeleteBlock(Block toDelete) {
+    public void addDeleteBlock(BlockInfo toDelete) {
       assert toDelete != null : "toDelete is null";
       toDeleteList.add(toDelete);
     }
 
-    public void removeDeleteBlock(Block block) {
+    public void removeDeleteBlock(BlockInfo block) {
       assert block != null : "block is null";
       toDeleteList.remove(block);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e3fcffe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 4590eec..48879d7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -904,8 +904,8 @@ public class INodeFile extends INodeWithAdditionalFields
         getDiffs().findEarlierSnapshotBlocks(snapshotId);
     if(snapshotBlocks == null)
       return;
-    List<Block> toDelete = collectedBlocks.getToDeleteList();
-    for(Block blk : snapshotBlocks) {
+    List<BlockInfo> toDelete = collectedBlocks.getToDeleteList();
+    for(BlockInfo blk : snapshotBlocks) {
       if(toDelete.contains(blk))
         collectedBlocks.removeDeleteBlock(blk);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e3fcffe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 39180b1..f67d25a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -109,7 +109,7 @@ import com.google.common.annotations.VisibleForTesting;
 @InterfaceAudience.Private
 public class NamenodeFsck implements DataEncryptionKeyFactory {
   public static final Log LOG = LogFactory.getLog(NameNode.class.getName());
-  
+
   // return string marking fsck status
   public static final String CORRUPT_STATUS = "is CORRUPT";
   public static final String HEALTHY_STATUS = "is HEALTHY";
@@ -117,7 +117,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
   public static final String DECOMMISSIONED_STATUS = "is DECOMMISSIONED";
   public static final String NONEXISTENT_STATUS = "does not exist";
   public static final String FAILURE_STATUS = "FAILED";
-  
+
   private final NameNode namenode;
   private final NetworkTopology networktopology;
   private final int totalDatanodes;
@@ -143,14 +143,14 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
    */
   private boolean internalError = false;
 
-  /** 
+  /**
    * True if the user specified the -move option.
    *
    * Whe this option is in effect, we will copy salvaged blocks into the lost
    * and found. */
   private boolean doMove = false;
 
-  /** 
+  /**
    * True if the user specified the -delete option.
    *
    * Whe this option is in effect, we will delete corrupted files.
@@ -183,7 +183,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
    * @param remoteAddress source address of the fsck request
    */
   NamenodeFsck(Configuration conf, NameNode namenode,
-      NetworkTopology networktopology, 
+      NetworkTopology networktopology,
       Map<String,String[]> pmap, PrintWriter out,
       int totalDatanodes, InetAddress remoteAddress) {
     this.conf = conf;
@@ -199,7 +199,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     this.staleInterval =
         conf.getLong(DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY,
           DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_DEFAULT);
-    
+
     for (Iterator<String> it = pmap.keySet().iterator(); it.hasNext();) {
       String key = it.next();
       if (key.equals("path")) { this.path = pmap.get("path")[0]; }
@@ -251,7 +251,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       }
       BlockCollection bc = bm.getBlockCollection(blockInfo);
       INode iNode = (INode) bc;
-      NumberReplicas numberReplicas= bm.countNodes(block);
+      NumberReplicas numberReplicas= bm.countNodes(blockInfo);
       out.println("Block Id: " + blockId);
       out.println("Block belongs to: "+iNode.getFullPathName());
       out.println("No. of Expected Replica: " +
@@ -350,7 +350,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
           listCorruptFileBlocks();
           return;
         }
-        
+
         if (this.showStoragePolcies) {
           storageTypeSummary = new StoragePolicySummary(
               namenode.getNamesystem().getBlockManager().getStoragePolicies());
@@ -380,7 +380,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
 
         // DFSck client scans for the string HEALTHY/CORRUPT to check the status
         // of file system and return appropriate code. Changing the output
-        // string might break testcases. Also note this must be the last line 
+        // string might break testcases. Also note this must be the last line
         // of the report.
         if (res.isHealthy()) {
           out.print("\n\nThe filesystem under path '" + path + "' " + HEALTHY_STATUS);
@@ -423,7 +423,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         + " CORRUPT files");
     out.println();
   }
-  
+
   @VisibleForTesting
   void check(String parent, HdfsFileStatus file, Result res) throws IOException {
     String path = file.getFullName(parent);
@@ -480,7 +480,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       throws IOException {
     long fileLen = file.getLen();
     LocatedBlocks blocks = null;
-    FSNamesystem fsn = namenode.getNamesystem();
+    final FSNamesystem fsn = namenode.getNamesystem();
     fsn.readLock();
     try {
       blocks = fsn.getBlockLocations(
@@ -539,8 +539,10 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       ExtendedBlock block = lBlk.getBlock();
       BlockManager bm = namenode.getNamesystem().getBlockManager();
 
+      final BlockInfo storedBlock = bm.getStoredBlock(
+          block.getLocalBlock());
       // count decommissionedReplicas / decommissioningReplicas
-      NumberReplicas numberReplicas = bm.countNodes(block.getLocalBlock());
+      NumberReplicas numberReplicas = bm.countNodes(storedBlock);
       int decommissionedReplicas = numberReplicas.decommissioned();;
       int decommissioningReplicas = numberReplicas.decommissioning();
       res.decommissionedReplicas +=  decommissionedReplicas;
@@ -608,7 +610,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
             out.println();
           out.print(path + ": ");
         }
-        out.println(" Replica placement policy is violated for " + 
+        out.println(" Replica placement policy is violated for " +
                     block + ". " + blockPlacementStatus.getErrorDescription());
       }
 
@@ -743,7 +745,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       return false;
     }
   }
-  
+
   private void copyBlocksToLostFound(String parent, HdfsFileStatus file,
         LocatedBlocks blocks) throws IOException {
     final DFSClient dfs = new DFSClient(NameNode.getAddress(conf), conf);
@@ -784,7 +786,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
           fos = dfs.create(target + "/" + chain, true);
           chain++;
         }
-        
+
         // copy the block. It's a pity it's not abstracted from DFSInputStream ...
         try {
           copyBlock(dfs, lblock, fos);
@@ -802,7 +804,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         LOG.warn("Fsck: there were errors copying the remains of the " +
           "corrupted file " + fullName + " to /lost+found");
       } else {
-        LOG.info("Fsck: copied the remains of the corrupted file " + 
+        LOG.info("Fsck: copied the remains of the corrupted file " +
           fullName + " to /lost+found");
       }
     } catch (Exception e) {
@@ -813,7 +815,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       dfs.close();
     }
   }
-      
+
   /*
    * XXX (ab) Bulk of this method is copied verbatim from {@link DFSClient}, which is
    * bad. Both places should be refactored to provide a method to copy blocks
@@ -824,12 +826,12 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     int failures = 0;
     InetSocketAddress targetAddr = null;
     TreeSet<DatanodeInfo> deadNodes = new TreeSet<DatanodeInfo>();
-    BlockReader blockReader = null; 
-    ExtendedBlock block = lblock.getBlock(); 
+    BlockReader blockReader = null;
+    ExtendedBlock block = lblock.getBlock();
 
     while (blockReader == null) {
       DatanodeInfo chosenNode;
-      
+
       try {
         chosenNode = bestNode(dfs, lblock.getLocations(), deadNodes);
         targetAddr = NetUtils.createSocketAddr(chosenNode.getXferAddr());
@@ -900,7 +902,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         bytesRead += cnt;
       }
       if ( bytesRead != block.getNumBytes() ) {
-        throw new IOException("Recorded block size is " + block.getNumBytes() + 
+        throw new IOException("Recorded block size is " + block.getNumBytes() +
                               ", but datanode returned " +bytesRead+" bytes");
       }
     } catch (Exception e) {
@@ -937,12 +939,12 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     } while (deadNodes.contains(chosenNode));
     return chosenNode;
   }
-  
+
   private void lostFoundInit(DFSClient dfs) {
     lfInited = true;
     try {
       String lfName = "/lost+found";
-      
+
       final HdfsFileStatus lfStatus = dfs.getFileInfo(lfName);
       if (lfStatus == null) { // not exists
         lfInitedOk = dfs.mkdirs(lfName, null, true);
@@ -997,21 +999,21 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
 
     final short replication;
     final int minReplication;
-    
+
     Result(Configuration conf) {
-      this.replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 
+      this.replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
                                             DFSConfigKeys.DFS_REPLICATION_DEFAULT);
       this.minReplication = (short)conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY,
                                             DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT);
     }
-    
+
     /**
      * DFS is considered healthy if there are no missing blocks.
      */
     boolean isHealthy() {
       return ((missingIds.size() == 0) && (corruptBlocks == 0));
     }
-    
+
     /** Add a missing block name, plus its size. */
     void addMissing(String id, long size) {
       missingIds.add(id);
@@ -1030,7 +1032,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         return 0.0f;
       return (float) (totalReplicas) / (float) totalBlocks;
     }
-    
+
     @Override
     public String toString() {
       StringBuilder res = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e3fcffe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
index 95fc06b..1428482 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
@@ -18,7 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 
 /** SafeMode related operations. */
 @InterfaceAudience.Private
@@ -49,5 +49,5 @@ public interface SafeMode {
   public void incrementSafeBlockCount(int replication);
 
   /** Decrement number of blocks that reached minimal replication. */
-  public void decrementSafeBlockCount(Block b);
+  public void decrementSafeBlockCount(BlockInfo b);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e3fcffe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
index 23e610f..148135b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
@@ -69,9 +69,10 @@ public class BlockManagerTestUtil {
     final BlockManager bm = namesystem.getBlockManager();
     namesystem.readLock();
     try {
+      final BlockInfo storedBlock = bm.getStoredBlock(b);
       return new int[]{getNumberOfRacks(bm, b),
-          bm.countNodes(b).liveReplicas(),
-          bm.neededReplications.contains(b) ? 1 : 0};
+          bm.countNodes(storedBlock).liveReplicas(),
+          bm.neededReplications.contains(storedBlock) ? 1 : 0};
     } finally {
       namesystem.readUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e3fcffe/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 5a82b15..396dff3 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
@@ -440,14 +440,14 @@ public class TestBlockManager {
     return blockInfo;
   }
 
-  private DatanodeStorageInfo[] scheduleSingleReplication(Block block) {
+  private DatanodeStorageInfo[] scheduleSingleReplication(BlockInfo block) {
     // list for priority 1
-    List<Block> list_p1 = new ArrayList<Block>();
+    List<BlockInfo> list_p1 = new ArrayList<>();
     list_p1.add(block);
 
     // list of lists for each priority
-    List<List<Block>> list_all = new ArrayList<List<Block>>();
-    list_all.add(new ArrayList<Block>()); // for priority 0
+    List<List<BlockInfo>> list_all = new ArrayList<>();
+    list_all.add(new ArrayList<BlockInfo>()); // for priority 0
     list_all.add(list_p1); // for priority 1
 
     assertEquals("Block not initially pending replication", 0,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e3fcffe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
index c3726f2..1c3f075 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
@@ -166,10 +166,11 @@ public class TestNodeCount {
 
   /* threadsafe read of the replication counts for this block */
   NumberReplicas countNodes(Block block, FSNamesystem namesystem) {
+    BlockManager blockManager = namesystem.getBlockManager();
     namesystem.readLock();
     try {
       lastBlock = block;
-      lastNum = namesystem.getBlockManager().countNodes(block);
+      lastNum = blockManager.countNodes(blockManager.getStoredBlock(block));
       return lastNum;
     }
     finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e3fcffe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
index a86b573..2d7bb44 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
@@ -117,7 +117,8 @@ public class TestOverReplicatedBlocks {
 
           // corrupt one won't be chosen to be excess one
           // without 4910 the number of live replicas would be 0: block gets lost
-          assertEquals(1, bm.countNodes(block.getLocalBlock()).liveReplicas());
+          assertEquals(1, bm.countNodes(
+              bm.getStoredBlock(block.getLocalBlock())).liveReplicas());
         }
       } finally {
         namesystem.writeUnlock();
@@ -219,7 +220,7 @@ public class TestOverReplicatedBlocks {
       out.close();
       ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, p);
       assertEquals("Expected only one live replica for the block", 1, bm
-          .countNodes(block.getLocalBlock()).liveReplicas());
+          .countNodes(bm.getStoredBlock(block.getLocalBlock())).liveReplicas());
     } finally {
       cluster.shutdown();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e3fcffe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
index 090c6e8..dae23d8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
@@ -53,6 +53,12 @@ public class TestPendingReplication {
   private static final int DFS_REPLICATION_INTERVAL = 1;
   // Number of datanodes in the cluster
   private static final int DATANODE_COUNT = 5;
+
+  private BlockInfo genBlockInfo(long id, long length, long gs) {
+    return new BlockInfoContiguous(new Block(id, length, gs),
+        (short) DATANODE_COUNT);
+  }
+
   @Test
   public void testPendingReplication() {
     PendingReplicationBlocks pendingReplications;
@@ -63,7 +69,7 @@ public class TestPendingReplication {
     //
     DatanodeStorageInfo[] storages = DFSTestUtil.createDatanodeStorageInfos(10);
     for (int i = 0; i < storages.length; i++) {
-      Block block = new Block(i, i, 0);
+      BlockInfo block = genBlockInfo(i, i, 0);
       DatanodeStorageInfo[] targets = new DatanodeStorageInfo[i];
       System.arraycopy(storages, 0, targets, 0, i);
       pendingReplications.increment(block,
@@ -76,7 +82,7 @@ public class TestPendingReplication {
     //
     // remove one item and reinsert it
     //
-    Block blk = new Block(8, 8, 0);
+    BlockInfo blk = genBlockInfo(8, 8, 0);
     pendingReplications.decrement(blk, storages[7].getDatanodeDescriptor()); // removes one replica
     assertEquals("pendingReplications.getNumReplicas ",
                  7, pendingReplications.getNumReplicas(blk));
@@ -96,7 +102,7 @@ public class TestPendingReplication {
     // are sane.
     //
     for (int i = 0; i < 10; i++) {
-      Block block = new Block(i, i, 0);
+      BlockInfo block = genBlockInfo(i, i, 0);
       int numReplicas = pendingReplications.getNumReplicas(block);
       assertTrue(numReplicas == i);
     }
@@ -115,7 +121,7 @@ public class TestPendingReplication {
     }
 
     for (int i = 10; i < 15; i++) {
-      Block block = new Block(i, i, 0);
+      BlockInfo block = genBlockInfo(i, i, 0);
       pendingReplications.increment(block,
           DatanodeStorageInfo.toDatanodeDescriptors(
               DFSTestUtil.createDatanodeStorageInfos(i)));
@@ -180,7 +186,7 @@ public class TestPendingReplication {
       block = new Block(1, 1, 0);
       blockInfo = new BlockInfoContiguous(block, (short) 3);
 
-      pendingReplications.increment(block,
+      pendingReplications.increment(blockInfo,
           DatanodeStorageInfo.toDatanodeDescriptors(
               DFSTestUtil.createDatanodeStorageInfos(1)));
       BlockCollection bc = Mockito.mock(BlockCollection.class);
@@ -195,7 +201,8 @@ public class TestPendingReplication {
       // Add a second block to pendingReplications that has no
       // corresponding entry in blocksmap
       block = new Block(2, 2, 0);
-      pendingReplications.increment(block,
+      blockInfo = new BlockInfoContiguous(block, (short) 3);
+      pendingReplications.increment(blockInfo,
           DatanodeStorageInfo.toDatanodeDescriptors(
               DFSTestUtil.createDatanodeStorageInfos(1)));
 
@@ -275,7 +282,7 @@ public class TestPendingReplication {
 
       assertEquals(1, blkManager.pendingReplications.size());
       INodeFile fileNode = fsn.getFSDirectory().getINode4Write(file).asFile();
-      Block[] blocks = fileNode.getBlocks();
+      BlockInfo[] blocks = fileNode.getBlocks();
       assertEquals(DATANODE_COUNT - 1,
           blkManager.pendingReplications.getNumReplicas(blocks[0]));
 
@@ -381,9 +388,9 @@ public class TestPendingReplication {
       BlockManagerTestUtil.computeAllPendingWork(bm);
       BlockManagerTestUtil.updateState(bm);
       assertEquals(bm.getPendingReplicationBlocksCount(), 1L);
-      assertEquals(bm.pendingReplications.getNumReplicas(block.getBlock()
-          .getLocalBlock()), 2);
-      
+      BlockInfo storedBlock = bm.getStoredBlock(block.getBlock().getLocalBlock());
+      assertEquals(bm.pendingReplications.getNumReplicas(storedBlock), 2);
+
       // 4. delete the file
       fs.delete(filePath, true);
       // retry at most 10 times, each time sleep for 1s. Note that 10s is much

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e3fcffe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
index 728934d..1a32892 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
@@ -58,7 +58,9 @@ public class TestRBWBlockInvalidation {
   
   private static NumberReplicas countReplicas(final FSNamesystem namesystem,
       ExtendedBlock block) {
-    return namesystem.getBlockManager().countNodes(block.getLocalBlock());
+    final BlockManager blockManager = namesystem.getBlockManager();
+    return blockManager.countNodes(blockManager.getStoredBlock(
+        block.getLocalBlock()));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e3fcffe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index 6e98538..2812957 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
@@ -830,7 +830,11 @@ public class TestReplicationPolicy {
     assertEquals(targets.length, 2);
     assertTrue(isOnSameRack(targets[0], dataNodes[2]));
   }
-  
+
+  private BlockInfo genBlockInfo(long id) {
+    return new BlockInfoContiguous(new Block(id), (short) 3);
+  }
+
   /**
    * Test for the high priority blocks are processed before the low priority
    * blocks.
@@ -849,16 +853,17 @@ public class TestReplicationPolicy {
           .getNamesystem().getBlockManager().neededReplications;
       for (int i = 0; i < 100; i++) {
         // Adding the blocks directly to normal priority
-        neededReplications.add(new Block(ThreadLocalRandom.current()
-            .nextLong()), 2, 0, 3);
+
+        neededReplications.add(genBlockInfo(ThreadLocalRandom.current().
+            nextLong()), 2, 0, 3);
       }
       // Lets wait for the replication interval, to start process normal
       // priority blocks
       Thread.sleep(DFS_NAMENODE_REPLICATION_INTERVAL);
       
       // Adding the block directly to high priority list
-      neededReplications.add(new Block(ThreadLocalRandom.current().nextLong()),
-          1, 0, 3);
+      neededReplications.add(genBlockInfo(ThreadLocalRandom.current().
+          nextLong()), 1, 0, 3);
 
       // Lets wait for the replication interval
       Thread.sleep(DFS_NAMENODE_REPLICATION_INTERVAL);
@@ -881,30 +886,31 @@ public class TestReplicationPolicy {
 
     for (int i = 0; i < 5; i++) {
       // Adding QUEUE_HIGHEST_PRIORITY block
-      underReplicatedBlocks.add(new Block(ThreadLocalRandom.current()
-          .nextLong()), 1, 0, 3);
+      underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+          nextLong()), 1, 0, 3);
 
       // Adding QUEUE_VERY_UNDER_REPLICATED block
-      underReplicatedBlocks.add(new Block(ThreadLocalRandom.current()
-          .nextLong()), 2, 0, 7);
+      underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+          nextLong()), 2, 0, 7);
 
       // Adding QUEUE_REPLICAS_BADLY_DISTRIBUTED block
-      underReplicatedBlocks.add(new Block(ThreadLocalRandom.current()
-          .nextLong()), 6, 0, 6);
+      underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+          nextLong()), 6, 0, 6);
 
       // Adding QUEUE_UNDER_REPLICATED block
-      underReplicatedBlocks.add(new Block(ThreadLocalRandom.current()
-          .nextLong()), 5, 0, 6);
+      underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+          nextLong()), 5, 0, 6);
 
       // Adding QUEUE_WITH_CORRUPT_BLOCKS block
-      underReplicatedBlocks.add(new Block(ThreadLocalRandom.current()
-          .nextLong()), 0, 0, 3);
+      underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+          nextLong()), 0, 0, 3);
     }
 
     // Choose 6 blocks from UnderReplicatedBlocks. Then it should pick 5 blocks
     // from
     // QUEUE_HIGHEST_PRIORITY and 1 block from QUEUE_VERY_UNDER_REPLICATED.
-    List<List<Block>> chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(6);
+    List<List<BlockInfo>> chosenBlocks =
+        underReplicatedBlocks.chooseUnderReplicatedBlocks(6);
     assertTheChosenBlocks(chosenBlocks, 5, 1, 0, 0, 0);
 
     // Choose 10 blocks from UnderReplicatedBlocks. Then it should pick 4 blocks from
@@ -914,8 +920,8 @@ public class TestReplicationPolicy {
     assertTheChosenBlocks(chosenBlocks, 0, 4, 5, 1, 0);
 
     // Adding QUEUE_HIGHEST_PRIORITY
-    underReplicatedBlocks.add(new Block(ThreadLocalRandom.current().nextLong()),
-        1, 0, 3);
+    underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+        nextLong()), 1, 0, 3);
 
     // Choose 10 blocks from UnderReplicatedBlocks. Then it should pick 1 block from
     // QUEUE_HIGHEST_PRIORITY, 4 blocks from QUEUE_REPLICAS_BADLY_DISTRIBUTED
@@ -933,7 +939,7 @@ public class TestReplicationPolicy {
   
   /** asserts the chosen blocks with expected priority blocks */
   private void assertTheChosenBlocks(
-      List<List<Block>> chosenBlocks, int firstPrioritySize,
+      List<List<BlockInfo>> chosenBlocks, int firstPrioritySize,
       int secondPrioritySize, int thirdPrioritySize, int fourthPrioritySize,
       int fifthPrioritySize) {
     assertEquals(
@@ -1107,9 +1113,9 @@ public class TestReplicationPolicy {
   public void testUpdateDoesNotCauseSkippedReplication() {
     UnderReplicatedBlocks underReplicatedBlocks = new UnderReplicatedBlocks();
 
-    Block block1 = new Block(ThreadLocalRandom.current().nextLong());
-    Block block2 = new Block(ThreadLocalRandom.current().nextLong());
-    Block block3 = new Block(ThreadLocalRandom.current().nextLong());
+    BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong());
+    BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong());
+    BlockInfo block3 = genBlockInfo(ThreadLocalRandom.current().nextLong());
 
     // Adding QUEUE_VERY_UNDER_REPLICATED block
     final int block1CurReplicas = 2;
@@ -1123,7 +1129,7 @@ public class TestReplicationPolicy {
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block3, 2, 0, 6);
 
-    List<List<Block>> chosenBlocks;
+    List<List<BlockInfo>> chosenBlocks;
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
     // from QUEUE_VERY_UNDER_REPLICATED.
@@ -1156,8 +1162,8 @@ public class TestReplicationPolicy {
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
 
-    Block block1 = new Block(ThreadLocalRandom.current().nextLong());
-    Block block2 = new Block(ThreadLocalRandom.current().nextLong());
+    BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong());
+    BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong());
 
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block1, 0, 1, 1);
@@ -1165,7 +1171,7 @@ public class TestReplicationPolicy {
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block2, 0, 1, 1);
 
-    List<List<Block>> chosenBlocks;
+    List<List<BlockInfo>> chosenBlocks;
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
     // from QUEUE_VERY_UNDER_REPLICATED.
@@ -1203,8 +1209,8 @@ public class TestReplicationPolicy {
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
 
-    Block block1 = new Block(ThreadLocalRandom.current().nextLong());
-    Block block2 = new Block(ThreadLocalRandom.current().nextLong());
+    BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong());
+    BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong());
 
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block1, 0, 1, 1);
@@ -1212,7 +1218,7 @@ public class TestReplicationPolicy {
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block2, 0, 1, 1);
 
-    List<List<Block>> chosenBlocks;
+    List<List<BlockInfo>> chosenBlocks;
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
     // from QUEUE_VERY_UNDER_REPLICATED.
@@ -1266,8 +1272,8 @@ public class TestReplicationPolicy {
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
 
-    Block block1 = new Block(ThreadLocalRandom.current().nextLong());
-    Block block2 = new Block(ThreadLocalRandom.current().nextLong());
+    BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong());
+    BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong());
 
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block1, 0, 1, 1);
@@ -1275,7 +1281,7 @@ public class TestReplicationPolicy {
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block2, 0, 1, 1);
 
-    List<List<Block>> chosenBlocks;
+    List<List<BlockInfo>> chosenBlocks;
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
     // from QUEUE_VERY_UNDER_REPLICATED.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e3fcffe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java
index e87a043..de36e07 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java
@@ -28,6 +28,10 @@ import static org.junit.Assert.fail;
 
 public class TestUnderReplicatedBlockQueues {
 
+  private BlockInfo genBlockInfo(long id) {
+    return new BlockInfoContiguous(new Block(id), (short) 3);
+  }
+
   /**
    * Test that adding blocks with different replication counts puts them
    * into different queues
@@ -36,11 +40,11 @@ public class TestUnderReplicatedBlockQueues {
   @Test
   public void testBlockPriorities() throws Throwable {
     UnderReplicatedBlocks queues = new UnderReplicatedBlocks();
-    Block block1 = new Block(1);
-    Block block2 = new Block(2);
-    Block block_very_under_replicated = new Block(3);
-    Block block_corrupt = new Block(4);
-    Block block_corrupt_repl_one = new Block(5);
+    BlockInfo block1 = genBlockInfo(1);
+    BlockInfo block2 = genBlockInfo(2);
+    BlockInfo block_very_under_replicated = genBlockInfo(3);
+    BlockInfo block_corrupt = genBlockInfo(4);
+    BlockInfo block_corrupt_repl_one = genBlockInfo(5);
 
     //add a block with a single entry
     assertAdded(queues, block1, 1, 0, 3);
@@ -82,7 +86,7 @@ public class TestUnderReplicatedBlockQueues {
   }
 
   private void assertAdded(UnderReplicatedBlocks queues,
-                           Block block,
+                           BlockInfo block,
                            int curReplicas,
                            int decomissionedReplicas,
                            int expectedReplicas) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e3fcffe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java
index 8f99afb..90eb7d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java
@@ -40,6 +40,7 @@ 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.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -81,6 +82,7 @@ public class TestReadOnlySharedStorage {
   private DatanodeInfo readOnlyDataNode;
   
   private Block block;
+  private BlockInfo storedBlock;
 
   private ExtendedBlock extendedBlock;
 
@@ -132,7 +134,8 @@ public class TestReadOnlySharedStorage {
     LocatedBlock locatedBlock = getLocatedBlock();
     extendedBlock = locatedBlock.getBlock();
     block = extendedBlock.getLocalBlock();
-    
+    storedBlock = blockManager.getStoredBlock(block);
+
     assertThat(locatedBlock.getLocations().length, is(1));
     normalDataNode = locatedBlock.getLocations()[0];
     readOnlyDataNode = datanodeManager.getDatanode(cluster.getDataNodes().get(RO_NODE_INDEX).getDatanodeId());
@@ -188,7 +191,7 @@ public class TestReadOnlySharedStorage {
   }
   
   private void validateNumberReplicas(int expectedReplicas) throws IOException {
-    NumberReplicas numberReplicas = blockManager.countNodes(block);
+    NumberReplicas numberReplicas = blockManager.countNodes(storedBlock);
     assertThat(numberReplicas.liveReplicas(), is(expectedReplicas));
     assertThat(numberReplicas.excessReplicas(), is(0));
     assertThat(numberReplicas.corruptReplicas(), is(0));
@@ -230,7 +233,7 @@ public class TestReadOnlySharedStorage {
         cluster.getNameNode(), normalDataNode.getXferAddr());
     
     // The live replica count should now be zero (since the NORMAL replica is offline)
-    NumberReplicas numberReplicas = blockManager.countNodes(block);
+    NumberReplicas numberReplicas = blockManager.countNodes(storedBlock);
     assertThat(numberReplicas.liveReplicas(), is(0));
     
     // The block should be reported as under-replicated
@@ -263,7 +266,7 @@ public class TestReadOnlySharedStorage {
     waitForLocations(1);
     
     // However, the corrupt READ_ONLY_SHARED replica should *not* affect the overall corrupt replicas count
-    NumberReplicas numberReplicas = blockManager.countNodes(block);
+    NumberReplicas numberReplicas = blockManager.countNodes(storedBlock);
     assertThat(numberReplicas.corruptReplicas(), is(0));
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e3fcffe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java
index 37abc5b..228a667 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.junit.Test;
@@ -260,7 +261,9 @@ public class TestProcessCorruptBlocks {
   }
 
   private static NumberReplicas countReplicas(final FSNamesystem namesystem, ExtendedBlock block) {
-    return namesystem.getBlockManager().countNodes(block.getLocalBlock());
+    final BlockManager blockManager = namesystem.getBlockManager();
+    return blockManager.countNodes(blockManager.getStoredBlock(
+        block.getLocalBlock()));
   }
 
   private void corruptBlock(MiniDFSCluster cluster, FileSystem fs, final Path fileName,


[46/50] [abbrv] hadoop git commit: MAPREDUCE-6305. AM/Task log page should be able to link back to the job. (Siqi Li via gera)

Posted by ar...@apache.org.
MAPREDUCE-6305. AM/Task log page should be able to link back to the job. (Siqi Li via gera)


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

Branch: refs/heads/HDFS-7240
Commit: 055cd5a9a11bd3e544a8a67c6c44b53efcb3a9ed
Parents: 63d40d5
Author: Gera Shegalov <ge...@apache.org>
Authored: Sat Jun 20 12:38:36 2015 -0700
Committer: Gera Shegalov <ge...@apache.org>
Committed: Sat Jun 20 12:38:36 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 +++
 .../mapreduce/v2/hs/webapp/HsController.java    | 28 ++++++++++++++++++--
 .../mapreduce/v2/hs/webapp/HsCountersPage.java  |  8 +-----
 .../mapreduce/v2/hs/webapp/HsLogsPage.java      | 11 +-------
 .../v2/hs/webapp/HsSingleCounterPage.java       |  8 +-----
 .../hadoop/mapreduce/v2/hs/webapp/HsView.java   | 13 +++++++++
 6 files changed, 45 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/055cd5a9/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 71934b7..d48fc5b 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -355,6 +355,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6316. Task Attempt List entries should link to the task overview.
     (Siqi Li via gera)
 
+    MAPREDUCE-6305. AM/Task log page should be able to link back to the job.
+    (Siqi Li via gera)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/055cd5a9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java
index b7bdfa4..2cc7ad5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java
@@ -18,9 +18,13 @@
 
 package org.apache.hadoop.mapreduce.v2.hs.webapp;
 
+import static org.apache.hadoop.yarn.webapp.YarnWebParams.ENTITY_STRING;
+
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.v2.app.webapp.App;
 import org.apache.hadoop.mapreduce.v2.app.webapp.AppController;
 import org.apache.hadoop.yarn.webapp.View;
@@ -32,8 +36,7 @@ import com.google.inject.Inject;
  * This class renders the various pages that the History Server WebApp supports
  */
 public class HsController extends AppController {
-  
-  
+
   @Inject HsController(App app, Configuration conf, RequestContext ctx) {
     super(app, conf, ctx, "History");
   }
@@ -175,6 +178,27 @@ public class HsController extends AppController {
    * Render the logs page.
    */
   public void logs() {
+    String logEntity = $(ENTITY_STRING);
+    JobID jid = null;
+    try {
+      jid = JobID.forName(logEntity);
+      set(JOB_ID, logEntity);
+      requireJob();
+    } catch (Exception e) {
+      // fall below
+    }
+
+    if (jid == null) {
+      try {
+        TaskAttemptID taskAttemptId = TaskAttemptID.forName(logEntity);
+        set(TASK_ID, taskAttemptId.getTaskID().toString());
+        set(JOB_ID, taskAttemptId.getJobID().toString());
+        requireTask();
+        requireJob();
+      } catch (Exception e) {
+        // fall below
+      }
+    }
     render(HsLogsPage.class);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/055cd5a9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsCountersPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsCountersPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsCountersPage.java
index 1bae8fd..e70a668 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsCountersPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsCountersPage.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.mapreduce.v2.hs.webapp;
 
-import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_ID;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
 
 import org.apache.hadoop.mapreduce.v2.app.webapp.CountersBlock;
@@ -35,12 +34,7 @@ public class HsCountersPage extends HsView {
    */
   @Override protected void preHead(Page.HTML<_> html) {
     commonPreHead(html);
-    String tid = $(TASK_ID);
-    String activeNav = "2";
-    if(tid == null || tid.isEmpty()) {
-      activeNav = "1";
-    }
-    set(initID(ACCORDION, "nav"), "{autoHeight:false, active:"+activeNav+"}");
+    setActiveNavColumnForTask();
     set(DATATABLES_SELECTOR, "#counters .dt-counters");
     set(initSelector(DATATABLES),
         "{bJQueryUI:true, sDom:'t', iDisplayLength:-1}");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/055cd5a9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsLogsPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsLogsPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsLogsPage.java
index 909a316..f483dc9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsLogsPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsLogsPage.java
@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.mapreduce.v2.hs.webapp;
 
-import static org.apache.hadoop.yarn.webapp.YarnWebParams.CONTAINER_ID;
-import static org.apache.hadoop.yarn.webapp.YarnWebParams.ENTITY_STRING;
-
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.log.AggregatedLogsBlock;
 
@@ -30,14 +27,8 @@ public class HsLogsPage extends HsView {
    * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
    */
   @Override protected void preHead(Page.HTML<_> html) {
-    String logEntity = $(ENTITY_STRING);
-    if (logEntity == null || logEntity.isEmpty()) {
-      logEntity = $(CONTAINER_ID);
-    }
-    if (logEntity == null || logEntity.isEmpty()) {
-      logEntity = "UNKNOWN";
-    }
     commonPreHead(html);
+    setActiveNavColumnForTask();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/055cd5a9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsSingleCounterPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsSingleCounterPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsSingleCounterPage.java
index 4e0036a..5f97b8f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsSingleCounterPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsSingleCounterPage.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.mapreduce.v2.hs.webapp;
 
-import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_ID;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
 
 import org.apache.hadoop.mapreduce.v2.app.webapp.SingleCounterBlock;
@@ -35,12 +34,7 @@ public class HsSingleCounterPage extends HsView {
    */
   @Override protected void preHead(Page.HTML<_> html) {
     commonPreHead(html);
-    String tid = $(TASK_ID);
-    String activeNav = "2";
-    if(tid == null || tid.isEmpty()) {
-      activeNav = "1";
-    }
-    set(initID(ACCORDION, "nav"), "{autoHeight:false, active:"+activeNav+"}");
+    setActiveNavColumnForTask();
     set(DATATABLES_ID, "singleCounter");
     set(initID(DATATABLES, "singleCounter"), counterTableInit());
     setTableStyles(html, "singleCounter");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/055cd5a9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsView.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsView.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsView.java
index 229bbb0..8e39087 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsView.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsView.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.mapreduce.v2.hs.webapp;
 
+import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_ID;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION_ID;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES;
@@ -55,6 +56,18 @@ public class HsView extends TwoColumnLayout {
     set(initID(ACCORDION, "nav"), "{autoHeight:false, active:0}");
   }
 
+  /**
+   * Determine which navigation column is active.
+   */
+  protected void setActiveNavColumnForTask() {
+    String tid = $(TASK_ID);
+    String activeNav = "2";
+    if((tid == null || tid.isEmpty())) {
+      activeNav = "1";
+    }
+    set(initID(ACCORDION, "nav"), "{autoHeight:false, active:"+activeNav+"}");
+  }
+
   /*
    * (non-Javadoc)
    * @see org.apache.hadoop.yarn.webapp.view.TwoColumnLayout#nav()


[31/50] [abbrv] hadoop git commit: HADOOP-7139. Allow appending to existing SequenceFiles (Contributed by kanaka kumar avvaru)

Posted by ar...@apache.org.
HADOOP-7139. Allow appending to existing SequenceFiles (Contributed by kanaka kumar avvaru)


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

Branch: refs/heads/HDFS-7240
Commit: 295d678be8853a52c3ec3da43d9265478d6632b3
Parents: 1babe50
Author: Vinayakumar B <vi...@apache.org>
Authored: Thu Jun 18 14:39:00 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Thu Jun 18 14:39:00 2015 +0530

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../java/org/apache/hadoop/io/SequenceFile.java |  85 ++++-
 .../hadoop/io/TestSequenceFileAppend.java       | 311 +++++++++++++++++++
 3 files changed, 394 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/295d678b/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 4b1b382..3430da6 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -651,6 +651,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11965. determine-flaky-tests needs a summary mode.
     (Yufei Gu via Yongjun Zhang)
 
+    HADOOP-7139. Allow appending to existing SequenceFiles
+    (kanaka kumar avvaru via vinayakumarb)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/295d678b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java
index 7a59149..e37e855 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java
@@ -837,7 +837,9 @@ public class SequenceFile {
     DataOutputStream deflateOut = null;
     Metadata metadata = null;
     Compressor compressor = null;
-    
+
+    private boolean appendMode = false;
+
     protected Serializer keySerializer;
     protected Serializer uncompressedValSerializer;
     protected Serializer compressedValSerializer;
@@ -909,6 +911,13 @@ public class SequenceFile {
       }
     }
 
+    static class AppendIfExistsOption extends Options.BooleanOption implements
+        Option {
+      AppendIfExistsOption(boolean value) {
+        super(value);
+      }
+    }
+
     static class KeyClassOption extends Options.ClassOption implements Option {
       KeyClassOption(Class<?> value) {
         super(value);
@@ -958,7 +967,7 @@ public class SequenceFile {
         return codec;
       }
     }
-    
+
     public static Option file(Path value) {
       return new FileOption(value);
     }
@@ -984,6 +993,10 @@ public class SequenceFile {
       return new ReplicationOption(value);
     }
     
+    public static Option appendIfExists(boolean value) {
+      return new AppendIfExistsOption(value);
+    }
+
     public static Option blockSize(long value) {
       return new BlockSizeOption(value);
     }
@@ -1030,6 +1043,8 @@ public class SequenceFile {
       ProgressableOption progressOption = 
         Options.getOption(ProgressableOption.class, opts);
       FileOption fileOption = Options.getOption(FileOption.class, opts);
+      AppendIfExistsOption appendIfExistsOption = Options.getOption(
+          AppendIfExistsOption.class, opts);
       FileSystemOption fsOption = Options.getOption(FileSystemOption.class, opts);
       StreamOption streamOption = Options.getOption(StreamOption.class, opts);
       KeyClassOption keyClassOption = 
@@ -1071,7 +1086,54 @@ public class SequenceFile {
           blockSizeOption.getValue();
         Progressable progress = progressOption == null ? null :
           progressOption.getValue();
-        out = fs.create(p, true, bufferSize, replication, blockSize, progress);
+
+        if (appendIfExistsOption != null && appendIfExistsOption.getValue()
+            && fs.exists(p)) {
+
+          // Read the file and verify header details
+          SequenceFile.Reader reader = new SequenceFile.Reader(conf,
+              SequenceFile.Reader.file(p), new Reader.OnlyHeaderOption());
+          try {
+
+            if (keyClassOption.getValue() != reader.getKeyClass()
+                || valueClassOption.getValue() != reader.getValueClass()) {
+              throw new IllegalArgumentException(
+                  "Key/value class provided does not match the file");
+            }
+
+            if (reader.getVersion() != VERSION[3]) {
+              throw new VersionMismatchException(VERSION[3],
+                  reader.getVersion());
+            }
+
+            if (metadataOption != null) {
+              LOG.info("MetaData Option is ignored during append");
+            }
+            metadataOption = (MetadataOption) SequenceFile.Writer
+                .metadata(reader.getMetadata());
+
+            CompressionOption readerCompressionOption = new CompressionOption(
+                reader.getCompressionType(), reader.getCompressionCodec());
+
+            if (readerCompressionOption.value != compressionTypeOption.value
+                || !readerCompressionOption.codec.getClass().getName()
+                    .equals(compressionTypeOption.codec.getClass().getName())) {
+              throw new IllegalArgumentException(
+                  "Compression option provided does not match the file");
+            }
+
+            sync = reader.getSync();
+
+          } finally {
+            reader.close();
+          }
+
+          out = fs.append(p, bufferSize, progress);
+          this.appendMode = true;
+        } else {
+          out = fs
+              .create(p, true, bufferSize, replication, blockSize, progress);
+        }
       } else {
         out = streamOption.getValue();
       }
@@ -1159,7 +1221,7 @@ public class SequenceFile {
       out.write(sync);                       // write the sync bytes
       out.flush();                           // flush header
     }
-    
+
     /** Initialize. */
     @SuppressWarnings("unchecked")
     void init(Configuration conf, FSDataOutputStream out, boolean ownStream,
@@ -1214,7 +1276,12 @@ public class SequenceFile {
         }
         this.compressedValSerializer.open(deflateOut);
       }
-      writeFileHeader();
+
+      if (appendMode) {
+        sync();
+      } else {
+        writeFileHeader();
+      }
     }
     
     /** Returns the class of keys in this file. */
@@ -2045,6 +2112,14 @@ public class SequenceFile {
     /** Returns the compression codec of data in this file. */
     public CompressionCodec getCompressionCodec() { return codec; }
     
+    private byte[] getSync() {
+      return sync;
+    }
+
+    private byte getVersion() {
+      return version;
+    }
+
     /**
      * Get the compression type for this file.
      * @return the compression type

http://git-wip-us.apache.org/repos/asf/hadoop/blob/295d678b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSequenceFileAppend.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSequenceFileAppend.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSequenceFileAppend.java
new file mode 100644
index 0000000..4576642
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSequenceFileAppend.java
@@ -0,0 +1,311 @@
+/**
+ * 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.io;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.hadoop.io.SequenceFile.Writer;
+import org.apache.hadoop.io.SequenceFile.Writer.Option;
+import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.io.serializer.JavaSerializationComparator;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestSequenceFileAppend {
+
+  private static Configuration conf;
+  private static FileSystem fs;
+  private static Path ROOT_PATH = new Path(System.getProperty(
+      "test.build.data", "build/test/data"));
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    conf = new Configuration();
+    conf.set("io.serializations",
+        "org.apache.hadoop.io.serializer.JavaSerialization");
+    conf.set("fs.file.impl", "org.apache.hadoop.fs.RawLocalFileSystem");
+    fs = FileSystem.get(conf);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    fs.close();
+  }
+
+  @Test(timeout = 30000)
+  public void testAppend() throws Exception {
+
+    Path file = new Path(ROOT_PATH, "testseqappend.seq");
+    fs.delete(file, true);
+
+    Text key1 = new Text("Key1");
+    Text value1 = new Text("Value1");
+    Text value2 = new Text("Updated");
+
+    SequenceFile.Metadata metadata = new SequenceFile.Metadata();
+    metadata.set(key1, value1);
+    Writer.Option metadataOption = Writer.metadata(metadata);
+
+    Writer writer = SequenceFile.createWriter(conf,
+        SequenceFile.Writer.file(file),
+        SequenceFile.Writer.keyClass(Long.class),
+        SequenceFile.Writer.valueClass(String.class), metadataOption);
+
+    writer.append(1L, "one");
+    writer.append(2L, "two");
+    writer.close();
+
+    verify2Values(file);
+
+    metadata.set(key1, value2);
+
+    writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(file),
+        SequenceFile.Writer.keyClass(Long.class),
+        SequenceFile.Writer.valueClass(String.class),
+        SequenceFile.Writer.appendIfExists(true), metadataOption);
+
+    // Verify the Meta data is not changed
+    assertEquals(value1, writer.metadata.get(key1));
+
+    writer.append(3L, "three");
+    writer.append(4L, "four");
+
+    writer.close();
+
+    verifyAll4Values(file);
+
+    // Verify the Meta data readable after append
+    Reader reader = new Reader(conf, Reader.file(file));
+    assertEquals(value1, reader.getMetadata().get(key1));
+    reader.close();
+
+    // Verify failure if the compression details are different
+    try {
+      Option wrongCompressOption = Writer.compression(CompressionType.RECORD,
+          new GzipCodec());
+
+      writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(file),
+          SequenceFile.Writer.keyClass(Long.class),
+          SequenceFile.Writer.valueClass(String.class),
+          SequenceFile.Writer.appendIfExists(true), wrongCompressOption);
+      writer.close();
+      fail("Expected IllegalArgumentException for compression options");
+    } catch (IllegalArgumentException IAE) {
+      // Expected exception. Ignore it
+    }
+
+    try {
+      Option wrongCompressOption = Writer.compression(CompressionType.BLOCK,
+          new DefaultCodec());
+
+      writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(file),
+          SequenceFile.Writer.keyClass(Long.class),
+          SequenceFile.Writer.valueClass(String.class),
+          SequenceFile.Writer.appendIfExists(true), wrongCompressOption);
+      writer.close();
+      fail("Expected IllegalArgumentException for compression options");
+    } catch (IllegalArgumentException IAE) {
+      // Expected exception. Ignore it
+    }
+
+    fs.deleteOnExit(file);
+  }
+
+  @Test(timeout = 30000)
+  public void testAppendRecordCompression() throws Exception {
+
+    Path file = new Path(ROOT_PATH, "testseqappendblockcompr.seq");
+    fs.delete(file, true);
+
+    Option compressOption = Writer.compression(CompressionType.RECORD,
+        new GzipCodec());
+    Writer writer = SequenceFile.createWriter(conf,
+        SequenceFile.Writer.file(file),
+        SequenceFile.Writer.keyClass(Long.class),
+        SequenceFile.Writer.valueClass(String.class), compressOption);
+
+    writer.append(1L, "one");
+    writer.append(2L, "two");
+    writer.close();
+
+    verify2Values(file);
+
+    writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(file),
+        SequenceFile.Writer.keyClass(Long.class),
+        SequenceFile.Writer.valueClass(String.class),
+        SequenceFile.Writer.appendIfExists(true), compressOption);
+
+    writer.append(3L, "three");
+    writer.append(4L, "four");
+    writer.close();
+
+    verifyAll4Values(file);
+
+    fs.deleteOnExit(file);
+  }
+
+  @Test(timeout = 30000)
+  public void testAppendBlockCompression() throws Exception {
+
+    Path file = new Path(ROOT_PATH, "testseqappendblockcompr.seq");
+    fs.delete(file, true);
+
+    Option compressOption = Writer.compression(CompressionType.BLOCK,
+        new GzipCodec());
+    Writer writer = SequenceFile.createWriter(conf,
+        SequenceFile.Writer.file(file),
+        SequenceFile.Writer.keyClass(Long.class),
+        SequenceFile.Writer.valueClass(String.class), compressOption);
+
+    writer.append(1L, "one");
+    writer.append(2L, "two");
+    writer.close();
+
+    verify2Values(file);
+
+    writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(file),
+        SequenceFile.Writer.keyClass(Long.class),
+        SequenceFile.Writer.valueClass(String.class),
+        SequenceFile.Writer.appendIfExists(true), compressOption);
+
+    writer.append(3L, "three");
+    writer.append(4L, "four");
+    writer.close();
+
+    verifyAll4Values(file);
+
+    // Verify failure if the compression details are different or not Provided
+    try {
+      writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(file),
+          SequenceFile.Writer.keyClass(Long.class),
+          SequenceFile.Writer.valueClass(String.class),
+          SequenceFile.Writer.appendIfExists(true));
+      writer.close();
+      fail("Expected IllegalArgumentException for compression options");
+    } catch (IllegalArgumentException IAE) {
+      // Expected exception. Ignore it
+    }
+
+    // Verify failure if the compression details are different
+    try {
+      Option wrongCompressOption = Writer.compression(CompressionType.RECORD,
+          new GzipCodec());
+
+      writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(file),
+          SequenceFile.Writer.keyClass(Long.class),
+          SequenceFile.Writer.valueClass(String.class),
+          SequenceFile.Writer.appendIfExists(true), wrongCompressOption);
+      writer.close();
+      fail("Expected IllegalArgumentException for compression options");
+    } catch (IllegalArgumentException IAE) {
+      // Expected exception. Ignore it
+    }
+
+    try {
+      Option wrongCompressOption = Writer.compression(CompressionType.BLOCK,
+          new DefaultCodec());
+
+      writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(file),
+          SequenceFile.Writer.keyClass(Long.class),
+          SequenceFile.Writer.valueClass(String.class),
+          SequenceFile.Writer.appendIfExists(true), wrongCompressOption);
+      writer.close();
+      fail("Expected IllegalArgumentException for compression options");
+    } catch (IllegalArgumentException IAE) {
+      // Expected exception. Ignore it
+    }
+
+    fs.deleteOnExit(file);
+  }
+
+  @Test(timeout = 30000)
+  public void testAppendSort() throws Exception {
+    Path file = new Path(ROOT_PATH, "testseqappendSort.seq");
+    fs.delete(file, true);
+
+    Path sortedFile = new Path(ROOT_PATH, "testseqappendSort.seq.sort");
+    fs.delete(sortedFile, true);
+
+    SequenceFile.Sorter sorter = new SequenceFile.Sorter(fs,
+        new JavaSerializationComparator<Long>(), Long.class, String.class, conf);
+
+    Option compressOption = Writer.compression(CompressionType.BLOCK,
+        new GzipCodec());
+    Writer writer = SequenceFile.createWriter(conf,
+        SequenceFile.Writer.file(file),
+        SequenceFile.Writer.keyClass(Long.class),
+        SequenceFile.Writer.valueClass(String.class), compressOption);
+
+    writer.append(2L, "two");
+    writer.append(1L, "one");
+
+    writer.close();
+
+    writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(file),
+        SequenceFile.Writer.keyClass(Long.class),
+        SequenceFile.Writer.valueClass(String.class),
+        SequenceFile.Writer.appendIfExists(true), compressOption);
+
+    writer.append(4L, "four");
+    writer.append(3L, "three");
+    writer.close();
+
+    // Sort file after append
+    sorter.sort(file, sortedFile);
+    verifyAll4Values(sortedFile);
+
+    fs.deleteOnExit(file);
+    fs.deleteOnExit(sortedFile);
+  }
+
+  private void verify2Values(Path file) throws IOException {
+    Reader reader = new Reader(conf, Reader.file(file));
+    assertEquals(1L, reader.next((Object) null));
+    assertEquals("one", reader.getCurrentValue((Object) null));
+    assertEquals(2L, reader.next((Object) null));
+    assertEquals("two", reader.getCurrentValue((Object) null));
+    assertNull(reader.next((Object) null));
+    reader.close();
+  }
+
+  private void verifyAll4Values(Path file) throws IOException {
+    Reader reader = new Reader(conf, Reader.file(file));
+    assertEquals(1L, reader.next((Object) null));
+    assertEquals("one", reader.getCurrentValue((Object) null));
+    assertEquals(2L, reader.next((Object) null));
+    assertEquals("two", reader.getCurrentValue((Object) null));
+    assertEquals(3L, reader.next((Object) null));
+    assertEquals("three", reader.getCurrentValue((Object) null));
+    assertEquals(4L, reader.next((Object) null));
+    assertEquals("four", reader.getCurrentValue((Object) null));
+    assertNull(reader.next((Object) null));
+    reader.close();
+  }
+}


[47/50] [abbrv] hadoop git commit: HDFS-7164. Update CHANGES.txt

Posted by ar...@apache.org.
HDFS-7164. Update CHANGES.txt


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

Branch: refs/heads/HDFS-7240
Commit: 658b5c84ae03b38293bea069efd2e5fd2922d74a
Parents: 055cd5a
Author: Arpit Agarwal <ar...@apache.org>
Authored: Sat Jun 20 13:08:52 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Sat Jun 20 13:08:52 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/658b5c84/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 882bf3f..aad3c25 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -635,8 +635,6 @@ Release 2.8.0 - UNRELEASED
     HDFS-8606. Cleanup DFSOutputStream by removing unwanted changes
     from HDFS-8386. (Rakesh R via szetszwo)
 
-    HDFS-7164. Feature documentation for HDFS-6581. (Arpit Agarwal)
-
     HDFS-9608. Merge HDFS-7912 to trunk and branch-2 (track BlockInfo instead
     of Block in UnderReplicatedBlocks and PendingReplicationBlocks).
     (Zhe Zhang via wang)
@@ -967,6 +965,8 @@ Release 2.7.1 - UNRELEASED
     HDFS-7546. Document, and set an accepting default for
     dfs.namenode.kerberos.principal.pattern (Harsh J via aw)
 
+    HDFS-7164. Feature documentation for HDFS-6581. (Arpit Agarwal)
+
   OPTIMIZATIONS
 
   BUG FIXES


[22/50] [abbrv] hadoop git commit: HDFS-8238. Move ClientProtocol to the hdfs-client. Contributed by Takanobu Asanuma.

Posted by ar...@apache.org.
HDFS-8238. Move ClientProtocol to the hdfs-client. 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/b8327744
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b8327744
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b8327744

Branch: refs/heads/HDFS-7240
Commit: b8327744884bf86b01b8998849e2a42fb9e5c249
Parents: 6e3fcff
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Jun 17 16:16:36 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Wed Jun 17 16:16:36 2015 -0700

----------------------------------------------------------------------
 .../hdfs/client/HdfsClientConfigKeys.java       |    1 +
 .../hadoop/hdfs/protocol/ClientProtocol.java    | 1459 ++++++++++++++++++
 .../hdfs/server/namenode/SafeModeException.java |   38 +
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |    3 +-
 .../hadoop/hdfs/protocol/ClientProtocol.java    | 1459 ------------------
 .../hdfs/server/namenode/SafeModeException.java |   38 -
 6 files changed, 1500 insertions(+), 1498 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8327744/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
index 26283aa..e6d579b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
@@ -39,6 +39,7 @@ public interface HdfsClientConfigKeys {
   String  DFS_NAMENODE_HTTPS_ADDRESS_KEY = "dfs.namenode.https-address";
   String DFS_HA_NAMENODES_KEY_PREFIX = "dfs.ha.namenodes";
   int DFS_NAMENODE_RPC_PORT_DEFAULT = 8020;
+  String DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY = "dfs.namenode.kerberos.principal";
 
   /** dfs.client.retry configuration properties */
   interface Retry {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8327744/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
new file mode 100644
index 0000000..7e27078
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -0,0 +1,1459 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.crypto.CryptoProtocolVersion;
+import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
+import org.apache.hadoop.fs.CacheFlag;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.InvalidPathException;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Options.Rename;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.inotify.EventBatchList;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
+import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
+import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
+import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.retry.AtMostOnce;
+import org.apache.hadoop.io.retry.Idempotent;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.KerberosInfo;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenInfo;
+
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY;
+
+/**********************************************************************
+ * ClientProtocol is used by user code via the DistributedFileSystem class to
+ * communicate with the NameNode.  User code can manipulate the directory
+ * namespace, as well as open/close file streams, etc.
+ *
+ **********************************************************************/
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+@KerberosInfo(
+    serverPrincipal = DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY)
+@TokenInfo(DelegationTokenSelector.class)
+public interface ClientProtocol {
+
+  /**
+   * Until version 69, this class ClientProtocol served as both
+   * the client interface to the NN AND the RPC protocol used to 
+   * communicate with the NN.
+   * 
+   * This class is used by both the DFSClient and the 
+   * NN server side to insulate from the protocol serialization.
+   * 
+   * If you are adding/changing this interface then you need to 
+   * change both this class and ALSO related protocol buffer
+   * wire protocol definition in ClientNamenodeProtocol.proto.
+   * 
+   * For more details on protocol buffer wire protocol, please see 
+   * .../org/apache/hadoop/hdfs/protocolPB/overview.html
+   * 
+   * The log of historical changes can be retrieved from the svn).
+   * 69: Eliminate overloaded method names.
+   * 
+   * 69L is the last version id when this class was used for protocols
+   *  serialization. DO not update this version any further. 
+   */
+  public static final long versionID = 69L;
+  
+  ///////////////////////////////////////
+  // File contents
+  ///////////////////////////////////////
+  /**
+   * Get locations of the blocks of the specified file within the specified range.
+   * DataNode locations for each block are sorted by
+   * the proximity to the client.
+   * <p>
+   * Return {@link LocatedBlocks} which contains
+   * file length, blocks and their locations.
+   * DataNode locations for each block are sorted by
+   * the distance to the client's address.
+   * <p>
+   * The client will then have to contact 
+   * one of the indicated DataNodes to obtain the actual data.
+   * 
+   * @param src file name
+   * @param offset range start offset
+   * @param length range length
+   *
+   * @return file length and array of blocks with their locations
+   *
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> does not exist
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public LocatedBlocks getBlockLocations(String src,
+                                         long offset,
+                                         long length) 
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException;
+
+  /**
+   * Get server default values for a number of configuration params.
+   * @return a set of server default configuration values
+   * @throws IOException
+   */
+  @Idempotent
+  public FsServerDefaults getServerDefaults() throws IOException;
+
+  /**
+   * Create a new file entry in the namespace.
+   * <p>
+   * This will create an empty file specified by the source path.
+   * The path should reflect a full path originated at the root.
+   * The name-node does not have a notion of "current" directory for a client.
+   * <p>
+   * Once created, the file is visible and available for read to other clients.
+   * Although, other clients cannot {@link #delete(String, boolean)}, re-create or 
+   * {@link #rename(String, String)} it until the file is completed
+   * or explicitly as a result of lease expiration.
+   * <p>
+   * Blocks have a maximum size.  Clients that intend to create
+   * multi-block files must also use 
+   * {@link #addBlock}
+   *
+   * @param src path of the file being created.
+   * @param masked masked permission.
+   * @param clientName name of the current client.
+   * @param flag indicates whether the file should be 
+   * overwritten if it already exists or create if it does not exist or append.
+   * @param createParent create missing parent directory if true
+   * @param replication block replication factor.
+   * @param blockSize maximum block size.
+   * @param supportedVersions CryptoProtocolVersions supported by the client
+   * 
+   * @return the status of the created file, it could be null if the server
+   *           doesn't support returning the file status
+   * @throws AccessControlException If access is denied
+   * @throws AlreadyBeingCreatedException if the path does not exist.
+   * @throws DSQuotaExceededException If file creation violates disk space 
+   *           quota restriction
+   * @throws FileAlreadyExistsException If file <code>src</code> already exists
+   * @throws FileNotFoundException If parent of <code>src</code> does not exist
+   *           and <code>createParent</code> is false
+   * @throws ParentNotDirectoryException If parent of <code>src</code> is not a
+   *           directory.
+   * @throws NSQuotaExceededException If file creation violates name space 
+   *           quota restriction
+   * @throws SafeModeException create not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred
+   *
+   * RuntimeExceptions:
+   * @throws InvalidPathException Path <code>src</code> is invalid
+   * <p>
+   * <em>Note that create with {@link CreateFlag#OVERWRITE} is idempotent.</em>
+   */
+  @AtMostOnce
+  public HdfsFileStatus create(String src, FsPermission masked,
+      String clientName, EnumSetWritable<CreateFlag> flag,
+      boolean createParent, short replication, long blockSize, 
+      CryptoProtocolVersion[] supportedVersions)
+      throws AccessControlException, AlreadyBeingCreatedException,
+      DSQuotaExceededException, FileAlreadyExistsException,
+      FileNotFoundException, NSQuotaExceededException,
+      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+      SnapshotAccessControlException, IOException;
+
+  /**
+   * Append to the end of the file. 
+   * @param src path of the file being created.
+   * @param clientName name of the current client.
+   * @param flag indicates whether the data is appended to a new block.
+   * @return wrapper with information about the last partial block and file
+   *    status if any
+   * @throws AccessControlException if permission to append file is 
+   * denied by the system. As usually on the client side the exception will 
+   * be wrapped into {@link org.apache.hadoop.ipc.RemoteException}.
+   * Allows appending to an existing file if the server is
+   * configured with the parameter dfs.support.append set to true, otherwise
+   * throws an IOException.
+   * 
+   * @throws AccessControlException If permission to append to file is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws DSQuotaExceededException If append violates disk space quota 
+   *           restriction
+   * @throws SafeModeException append not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred.
+   *
+   * RuntimeExceptions:
+   * @throws UnsupportedOperationException if append is not supported
+   */
+  @AtMostOnce
+  public LastBlockWithStatus append(String src, String clientName,
+      EnumSetWritable<CreateFlag> flag) throws AccessControlException,
+      DSQuotaExceededException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+
+  /**
+   * Set replication for an existing file.
+   * <p>
+   * The NameNode sets replication to the new value and returns.
+   * The actual block replication is not expected to be performed during  
+   * this method call. The blocks will be populated or removed in the 
+   * background as the result of the routine block maintenance procedures.
+   * 
+   * @param src file name
+   * @param replication new replication
+   * 
+   * @return true if successful;
+   *         false if file does not exist or is a directory
+   *
+   * @throws AccessControlException If access is denied
+   * @throws DSQuotaExceededException If replication violates disk space 
+   *           quota restriction
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws SafeModeException not allowed in safemode
+   * @throws UnresolvedLinkException if <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public boolean setReplication(String src, short replication)
+      throws AccessControlException, DSQuotaExceededException,
+      FileNotFoundException, SafeModeException, UnresolvedLinkException,
+      SnapshotAccessControlException, IOException;
+
+  /**
+   * Get all the available block storage policies.
+   * @return All the in-use block storage policies currently.
+   */
+  @Idempotent
+  public BlockStoragePolicy[] getStoragePolicies() throws IOException;
+
+  /**
+   * Set the storage policy for a file/directory
+   * @param src Path of an existing file/directory. 
+   * @param policyName The name of the storage policy
+   * @throws SnapshotAccessControlException If access is denied
+   * @throws UnresolvedLinkException if <code>src</code> contains a symlink
+   * @throws FileNotFoundException If file/dir <code>src</code> is not found
+   * @throws QuotaExceededException If changes violate the quota restriction
+   */
+  @Idempotent
+  public void setStoragePolicy(String src, String policyName)
+      throws SnapshotAccessControlException, UnresolvedLinkException,
+      FileNotFoundException, QuotaExceededException, IOException;
+
+  /**
+   * Set permissions for an existing file/directory.
+   * 
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws SafeModeException not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public void setPermission(String src, FsPermission permission)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+
+  /**
+   * Set Owner of a path (i.e. a file or a directory).
+   * The parameters username and groupname cannot both be null.
+   * @param src file path
+   * @param username If it is null, the original username remains unchanged.
+   * @param groupname If it is null, the original groupname remains unchanged.
+   *
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws SafeModeException not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public void setOwner(String src, String username, String groupname)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+
+  /**
+   * The client can give up on a block by calling abandonBlock().
+   * The client can then either obtain a new block, or complete or abandon the 
+   * file.
+   * Any partial writes to the block will be discarded.
+   * 
+   * @param b         Block to abandon
+   * @param fileId    The id of the file where the block resides.  Older clients
+   *                    will pass GRANDFATHER_INODE_ID here.
+   * @param src       The path of the file where the block resides.
+   * @param holder    Lease holder.
+   *
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException file <code>src</code> is not found
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public void abandonBlock(ExtendedBlock b, long fileId,
+      String src, String holder)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException;
+
+  /**
+   * A client that wants to write an additional block to the 
+   * indicated filename (which must currently be open for writing)
+   * should call addBlock().  
+   *
+   * addBlock() allocates a new block and datanodes the block data
+   * should be replicated to.
+   * 
+   * addBlock() also commits the previous block by reporting
+   * to the name-node the actual generation stamp and the length
+   * of the block that the client has transmitted to data-nodes.
+   *
+   * @param src the file being created
+   * @param clientName the name of the client that adds the block
+   * @param previous  previous block
+   * @param excludeNodes a list of nodes that should not be
+   * allocated for the current block
+   * @param fileId the id uniquely identifying a file
+   * @param favoredNodes the list of nodes where the client wants the blocks.
+   *          Nodes are identified by either host name or address.
+   *
+   * @return LocatedBlock allocated block information.
+   *
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws NotReplicatedYetException previous blocks of the file are not
+   *           replicated yet. Blocks cannot be added until replication
+   *           completes.
+   * @throws SafeModeException create not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public LocatedBlock addBlock(String src, String clientName,
+      ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId, 
+      String[] favoredNodes)
+      throws AccessControlException, FileNotFoundException,
+      NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
+      IOException;
+
+  /** 
+   * Get a datanode for an existing pipeline.
+   * 
+   * @param src the file being written
+   * @param fileId the ID of the file being written
+   * @param blk the block being written
+   * @param existings the existing nodes in the pipeline
+   * @param excludes the excluded nodes
+   * @param numAdditionalNodes number of additional datanodes
+   * @param clientName the name of the client
+   * 
+   * @return the located block.
+   * 
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws SafeModeException create not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public LocatedBlock getAdditionalDatanode(final String src,
+      final long fileId, final ExtendedBlock blk,
+      final DatanodeInfo[] existings,
+      final String[] existingStorageIDs,
+      final DatanodeInfo[] excludes,
+      final int numAdditionalNodes, final String clientName
+      ) throws AccessControlException, FileNotFoundException,
+          SafeModeException, UnresolvedLinkException, IOException;
+
+  /**
+   * The client is done writing data to the given filename, and would 
+   * like to complete it.  
+   *
+   * The function returns whether the file has been closed successfully.
+   * If the function returns false, the caller should try again.
+   * 
+   * close() also commits the last block of file by reporting
+   * to the name-node the actual generation stamp and the length
+   * of the block that the client has transmitted to data-nodes.
+   *
+   * A call to complete() will not return true until all the file's
+   * blocks have been replicated the minimum number of times.  Thus,
+   * DataNode failures may cause a client to call complete() several
+   * times before succeeding.
+   *
+   * @param src the file being created
+   * @param clientName the name of the client that adds the block
+   * @param last the last block info
+   * @param fileId the id uniquely identifying a file
+   *
+   * @return true if all file blocks are minimally replicated or false otherwise
+   *
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws SafeModeException create not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink 
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public boolean complete(String src, String clientName,
+                          ExtendedBlock last, long fileId)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException;
+
+  /**
+   * The client wants to report corrupted blocks (blocks with specified
+   * locations on datanodes).
+   * @param blocks Array of located blocks to report
+   */
+  @Idempotent
+  public void reportBadBlocks(LocatedBlock[] blocks) throws IOException;
+
+  ///////////////////////////////////////
+  // Namespace management
+  ///////////////////////////////////////
+  /**
+   * Rename an item in the file system namespace.
+   * @param src existing file or directory name.
+   * @param dst new name.
+   * @return true if successful, or false if the old name does not exist
+   * or if the new name already belongs to the namespace.
+   * 
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException an I/O error occurred 
+   */
+  @AtMostOnce
+  public boolean rename(String src, String dst) 
+      throws UnresolvedLinkException, SnapshotAccessControlException, IOException;
+
+  /**
+   * Moves blocks from srcs to trg and delete srcs
+   * 
+   * @param trg existing file
+   * @param srcs - list of existing files (same block size, same replication)
+   * @throws IOException if some arguments are invalid
+   * @throws UnresolvedLinkException if <code>trg</code> or <code>srcs</code>
+   *           contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   */
+  @AtMostOnce
+  public void concat(String trg, String[] srcs) 
+      throws IOException, UnresolvedLinkException, SnapshotAccessControlException;
+
+  /**
+   * Rename src to dst.
+   * <ul>
+   * <li>Fails if src is a file and dst is a directory.
+   * <li>Fails if src is a directory and dst is a file.
+   * <li>Fails if the parent of dst does not exist or is a file.
+   * </ul>
+   * <p>
+   * Without OVERWRITE option, rename fails if the dst already exists.
+   * With OVERWRITE option, rename overwrites the dst, if it is a file 
+   * or an empty directory. Rename fails if dst is a non-empty directory.
+   * <p>
+   * This implementation of rename is atomic.
+   * <p>
+   * @param src existing file or directory name.
+   * @param dst new name.
+   * @param options Rename options
+   * 
+   * @throws AccessControlException If access is denied
+   * @throws DSQuotaExceededException If rename violates disk space 
+   *           quota restriction
+   * @throws FileAlreadyExistsException If <code>dst</code> already exists and
+   *           <code>options</options> has {@link Rename#OVERWRITE} option
+   *           false.
+   * @throws FileNotFoundException If <code>src</code> does not exist
+   * @throws NSQuotaExceededException If rename violates namespace 
+   *           quota restriction
+   * @throws ParentNotDirectoryException If parent of <code>dst</code> 
+   *           is not a directory
+   * @throws SafeModeException rename not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> or
+   *           <code>dst</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred
+   */
+  @AtMostOnce
+  public void rename2(String src, String dst, Options.Rename... options)
+      throws AccessControlException, DSQuotaExceededException,
+      FileAlreadyExistsException, FileNotFoundException,
+      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+
+  /**
+   * Truncate file src to new size.
+   * <ul>
+   * <li>Fails if src is a directory.
+   * <li>Fails if src does not exist.
+   * <li>Fails if src is not closed.
+   * <li>Fails if new size is greater than current size.
+   * </ul>
+   * <p>
+   * This implementation of truncate is purely a namespace operation if truncate
+   * occurs at a block boundary. Requires DataNode block recovery otherwise.
+   * <p>
+   * @param src  existing file
+   * @param newLength  the target size
+   *
+   * @return true if client does not need to wait for block recovery,
+   * false if client needs to wait for block recovery.
+   *
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws SafeModeException truncate not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public boolean truncate(String src, long newLength, String clientName)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+
+  /**
+   * Delete the given file or directory from the file system.
+   * <p>
+   * same as delete but provides a way to avoid accidentally 
+   * deleting non empty directories programmatically. 
+   * @param src existing name
+   * @param recursive if true deletes a non empty directory recursively,
+   * else throws an exception.
+   * @return true only if the existing file or directory was actually removed 
+   * from the file system.
+   * 
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws SafeModeException create not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred
+   */
+  @AtMostOnce
+  public boolean delete(String src, boolean recursive)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+  
+  /**
+   * Create a directory (or hierarchy of directories) with the given
+   * name and permission.
+   *
+   * @param src The path of the directory being created
+   * @param masked The masked permission of the directory being created
+   * @param createParent create missing parent directory if true
+   *
+   * @return True if the operation success.
+   *
+   * @throws AccessControlException If access is denied
+   * @throws FileAlreadyExistsException If <code>src</code> already exists
+   * @throws FileNotFoundException If parent of <code>src</code> does not exist
+   *           and <code>createParent</code> is false
+   * @throws NSQuotaExceededException If file creation violates quota restriction
+   * @throws ParentNotDirectoryException If parent of <code>src</code> 
+   *           is not a directory
+   * @throws SafeModeException create not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred.
+   *
+   * RunTimeExceptions:
+   * @throws InvalidPathException If <code>src</code> is invalid
+   */
+  @Idempotent
+  public boolean mkdirs(String src, FsPermission masked, boolean createParent)
+      throws AccessControlException, FileAlreadyExistsException,
+      FileNotFoundException, NSQuotaExceededException,
+      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+      SnapshotAccessControlException, IOException;
+
+  /**
+   * Get a partial listing of the indicated directory
+   *
+   * @param src the directory name
+   * @param startAfter the name to start listing after encoded in java UTF8
+   * @param needLocation if the FileStatus should contain block locations
+   *
+   * @return a partial listing starting after startAfter
+   *
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>src</code> is not found
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public DirectoryListing getListing(String src,
+                                     byte[] startAfter,
+                                     boolean needLocation)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException;
+  
+  /**
+   * Get listing of all the snapshottable directories
+   * 
+   * @return Information about all the current snapshottable directory
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+      throws IOException;
+
+  ///////////////////////////////////////
+  // System issues and management
+  ///////////////////////////////////////
+
+  /**
+   * Client programs can cause stateful changes in the NameNode
+   * that affect other clients.  A client may obtain a file and 
+   * neither abandon nor complete it.  A client might hold a series
+   * of locks that prevent other clients from proceeding.
+   * Clearly, it would be bad if a client held a bunch of locks
+   * that it never gave up.  This can happen easily if the client
+   * dies unexpectedly.
+   * <p>
+   * So, the NameNode will revoke the locks and live file-creates
+   * for clients that it thinks have died.  A client tells the
+   * NameNode that it is still alive by periodically calling
+   * renewLease().  If a certain amount of time passes since
+   * the last call to renewLease(), the NameNode assumes the
+   * client has died.
+   *
+   * @throws AccessControlException permission denied
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public void renewLease(String clientName) throws AccessControlException,
+      IOException;
+
+  /**
+   * Start lease recovery.
+   * Lightweight NameNode operation to trigger lease recovery
+   * 
+   * @param src path of the file to start lease recovery
+   * @param clientName name of the current client
+   * @return true if the file is already closed
+   * @throws IOException
+   */
+  @Idempotent
+  public boolean recoverLease(String src, String clientName) throws IOException;
+
+  public int GET_STATS_CAPACITY_IDX = 0;
+  public int GET_STATS_USED_IDX = 1;
+  public int GET_STATS_REMAINING_IDX = 2;
+  public int GET_STATS_UNDER_REPLICATED_IDX = 3;
+  public int GET_STATS_CORRUPT_BLOCKS_IDX = 4;
+  public int GET_STATS_MISSING_BLOCKS_IDX = 5;
+  public int GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX = 6;
+  
+  /**
+   * Get a set of statistics about the filesystem.
+   * Right now, only seven values are returned.
+   * <ul>
+   * <li> [0] contains the total storage capacity of the system, in bytes.</li>
+   * <li> [1] contains the total used space of the system, in bytes.</li>
+   * <li> [2] contains the available storage of the system, in bytes.</li>
+   * <li> [3] contains number of under replicated blocks in the system.</li>
+   * <li> [4] contains number of blocks with a corrupt replica. </li>
+   * <li> [5] contains number of blocks without any good replicas left. </li>
+   * <li> [6] contains number of blocks which have replication factor
+   *          1 and have lost the only replica. </li>
+   * </ul>
+   * Use public constants like {@link #GET_STATS_CAPACITY_IDX} in place of 
+   * actual numbers to index into the array.
+   */
+  @Idempotent
+  public long[] getStats() throws IOException;
+
+  /**
+   * Get a report on the system's current datanodes.
+   * One DatanodeInfo object is returned for each DataNode.
+   * Return live datanodes if type is LIVE; dead datanodes if type is DEAD;
+   * otherwise all datanodes if type is ALL.
+   */
+  @Idempotent
+  public DatanodeInfo[] getDatanodeReport(HdfsConstants.DatanodeReportType type)
+      throws IOException;
+
+  /**
+   * Get a report on the current datanode storages.
+   */
+  @Idempotent
+  public DatanodeStorageReport[] getDatanodeStorageReport(
+      HdfsConstants.DatanodeReportType type) throws IOException;
+
+  /**
+   * Get the block size for the given file.
+   * @param filename The name of the file
+   * @return The number of bytes in each block
+   * @throws IOException
+   * @throws UnresolvedLinkException if the path contains a symlink. 
+   */
+  @Idempotent
+  public long getPreferredBlockSize(String filename) 
+      throws IOException, UnresolvedLinkException;
+
+  /**
+   * Enter, leave or get safe mode.
+   * <p>
+   * Safe mode is a name node state when it
+   * <ol><li>does not accept changes to name space (read-only), and</li>
+   * <li>does not replicate or delete blocks.</li></ol>
+   * 
+   * <p>
+   * Safe mode is entered automatically at name node startup.
+   * Safe mode can also be entered manually using
+   * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean) setSafeMode(SafeModeAction.SAFEMODE_ENTER,false)}.
+   * <p>
+   * At startup the name node accepts data node reports collecting
+   * information about block locations.
+   * In order to leave safe mode it needs to collect a configurable
+   * percentage called threshold of blocks, which satisfy the minimal 
+   * replication condition.
+   * The minimal replication condition is that each block must have at least
+   * <tt>dfs.namenode.replication.min</tt> replicas.
+   * When the threshold is reached the name node extends safe mode
+   * for a configurable amount of time
+   * to let the remaining data nodes to check in before it
+   * will start replicating missing blocks.
+   * Then the name node leaves safe mode.
+   * <p>
+   * If safe mode is turned on manually using
+   * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean) setSafeMode(SafeModeAction.SAFEMODE_ENTER,false)}
+   * then the name node stays in safe mode until it is manually turned off
+   * using {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean) setSafeMode(SafeModeAction.SAFEMODE_LEAVE,false)}.
+   * Current state of the name node can be verified using
+   * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean) setSafeMode(SafeModeAction.SAFEMODE_GET,false)}
+   * <h4>Configuration parameters:</h4>
+   * <tt>dfs.safemode.threshold.pct</tt> is the threshold parameter.<br>
+   * <tt>dfs.safemode.extension</tt> is the safe mode extension parameter.<br>
+   * <tt>dfs.namenode.replication.min</tt> is the minimal replication parameter.
+   * 
+   * <h4>Special cases:</h4>
+   * The name node does not enter safe mode at startup if the threshold is 
+   * set to 0 or if the name space is empty.<br>
+   * If the threshold is set to 1 then all blocks need to have at least 
+   * minimal replication.<br>
+   * If the threshold value is greater than 1 then the name node will not be 
+   * able to turn off safe mode automatically.<br>
+   * Safe mode can always be turned off manually.
+   * 
+   * @param action  <ul> <li>0 leave safe mode;</li>
+   *                <li>1 enter safe mode;</li>
+   *                <li>2 get safe mode state.</li></ul>
+   * @param isChecked If true then action will be done only in ActiveNN.
+   * 
+   * @return <ul><li>0 if the safe mode is OFF or</li> 
+   *         <li>1 if the safe mode is ON.</li></ul>
+   *                   
+   * @throws IOException
+   */
+  @Idempotent
+  public boolean setSafeMode(HdfsConstants.SafeModeAction action, boolean isChecked) 
+      throws IOException;
+
+  /**
+   * Save namespace image.
+   * <p>
+   * Saves current namespace into storage directories and reset edits log.
+   * Requires superuser privilege and safe mode.
+   *
+   * @param timeWindow NameNode does a checkpoint if the latest checkpoint was
+   *                   done beyond the given time period (in seconds).
+   * @param txGap NameNode does a checkpoint if the gap between the latest
+   *              checkpoint and the latest transaction id is greater this gap.
+   * @return whether an extra checkpoint has been done
+   *
+   * @throws IOException if image creation failed.
+   */
+  @AtMostOnce
+  public boolean saveNamespace(long timeWindow, long txGap) throws IOException;
+
+  
+  /**
+   * Roll the edit log.
+   * Requires superuser privileges.
+   * 
+   * @throws AccessControlException if the superuser privilege is violated
+   * @throws IOException if log roll fails
+   * @return the txid of the new segment
+   */
+  @Idempotent
+  public long rollEdits() throws AccessControlException, IOException;
+
+  /**
+   * Enable/Disable restore failed storage.
+   * <p>
+   * sets flag to enable restore of failed storage replicas
+   * 
+   * @throws AccessControlException if the superuser privilege is violated.
+   */
+  @Idempotent
+  public boolean restoreFailedStorage(String arg) 
+      throws AccessControlException, IOException;
+
+  /**
+   * Tells the namenode to reread the hosts and exclude files. 
+   * @throws IOException
+   */
+  @Idempotent
+  public void refreshNodes() throws IOException;
+
+  /**
+   * Finalize previous upgrade.
+   * Remove file system state saved during the upgrade.
+   * The upgrade will become irreversible.
+   * 
+   * @throws IOException
+   */
+  @Idempotent
+  public void finalizeUpgrade() throws IOException;
+
+  /**
+   * Rolling upgrade operations.
+   * @param action either query, prepare or finalize.
+   * @return rolling upgrade information.
+   */
+  @Idempotent
+  public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
+      throws IOException;
+
+  /**
+   * @return CorruptFileBlocks, containing a list of corrupt files (with
+   *         duplicates if there is more than one corrupt block in a file)
+   *         and a cookie
+   * @throws IOException
+   *
+   * Each call returns a subset of the corrupt files in the system. To obtain
+   * all corrupt files, call this method repeatedly and each time pass in the
+   * cookie returned from the previous call.
+   */
+  @Idempotent
+  public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
+      throws IOException;
+  
+  /**
+   * Dumps namenode data structures into specified file. If the file
+   * already exists, then append.
+   *
+   * @throws IOException
+   */
+  @Idempotent
+  public void metaSave(String filename) throws IOException;
+
+  /**
+   * Tell all datanodes to use a new, non-persistent bandwidth value for
+   * dfs.balance.bandwidthPerSec.
+   *
+   * @param bandwidth Blanacer bandwidth in bytes per second for this datanode.
+   * @throws IOException
+   */
+  @Idempotent
+  public void setBalancerBandwidth(long bandwidth) throws IOException;
+  
+  /**
+   * Get the file info for a specific file or directory.
+   * @param src The string representation of the path to the file
+   *
+   * @return object containing information regarding the file
+   *         or null if file not found
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>src</code> is not found
+   * @throws UnresolvedLinkException if the path contains a symlink. 
+   * @throws IOException If an I/O error occurred        
+   */
+  @Idempotent
+  public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException;
+  
+  /**
+   * Get the close status of a file
+   * @param src The string representation of the path to the file
+   *
+   * @return return true if file is closed
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>src</code> is not found
+   * @throws UnresolvedLinkException if the path contains a symlink.
+   * @throws IOException If an I/O error occurred     
+   */
+  @Idempotent
+  public boolean isFileClosed(String src) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException;
+  
+  /**
+   * Get the file info for a specific file or directory. If the path 
+   * refers to a symlink then the FileStatus of the symlink is returned.
+   * @param src The string representation of the path to the file
+   *
+   * @return object containing information regarding the file
+   *         or null if file not found
+   *
+   * @throws AccessControlException permission denied
+   * @throws UnresolvedLinkException if <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred        
+   */
+  @Idempotent
+  public HdfsFileStatus getFileLinkInfo(String src)
+      throws AccessControlException, UnresolvedLinkException, IOException;
+  
+  /**
+   * Get {@link ContentSummary} rooted at the specified directory.
+   * @param path The string representation of the path
+   *
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>path</code> is not found
+   * @throws UnresolvedLinkException if <code>path</code> contains a symlink. 
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public ContentSummary getContentSummary(String path)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException;
+
+  /**
+   * Set the quota for a directory.
+   * @param path  The string representation of the path to the directory
+   * @param namespaceQuota Limit on the number of names in the tree rooted 
+   *                       at the directory
+   * @param storagespaceQuota Limit on storage space occupied all the files under
+   *                       this directory.
+   * @param type StorageType that the space quota is intended to be set on.
+   *             It may be null when called by traditional space/namespace quota.
+   *             When type is is not null, the storagespaceQuota parameter is for
+   *             type specified and namespaceQuota must be
+   *             {@link HdfsConstants#QUOTA_DONT_SET}.
+   *
+   * <br><br>
+   *                       
+   * The quota can have three types of values : (1) 0 or more will set 
+   * the quota to that value, (2) {@link HdfsConstants#QUOTA_DONT_SET}  implies
+   * the quota will not be changed, and (3) {@link HdfsConstants#QUOTA_RESET}
+   * implies the quota will be reset. Any other value is a runtime error.
+   * 
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>path</code> is not found
+   * @throws QuotaExceededException if the directory size 
+   *           is greater than the given quota
+   * @throws UnresolvedLinkException if the <code>path</code> contains a symlink. 
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
+      StorageType type) throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+
+  /**
+   * Write all metadata for this file into persistent storage.
+   * The file must be currently open for writing.
+   * @param src The string representation of the path
+   * @param inodeId The inode ID, or GRANDFATHER_INODE_ID if the client is
+   *                too old to support fsync with inode IDs.
+   * @param client The string representation of the client
+   * @param lastBlockLength The length of the last block (under construction) 
+   *                        to be reported to NameNode 
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>src</code> is not found
+   * @throws UnresolvedLinkException if <code>src</code> contains a symlink. 
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public void fsync(String src, long inodeId, String client,
+                    long lastBlockLength)
+      throws AccessControlException, FileNotFoundException, 
+      UnresolvedLinkException, IOException;
+
+  /**
+   * Sets the modification and access time of the file to the specified time.
+   * @param src The string representation of the path
+   * @param mtime The number of milliseconds since Jan 1, 1970.
+   *              Setting mtime to -1 means that modification time should not be set
+   *              by this call.
+   * @param atime The number of milliseconds since Jan 1, 1970.
+   *              Setting atime to -1 means that access time should not be set
+   *              by this call.
+   *              
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>src</code> is not found
+   * @throws UnresolvedLinkException if <code>src</code> contains a symlink. 
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public void setTimes(String src, long mtime, long atime)
+      throws AccessControlException, FileNotFoundException, 
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+
+  /**
+   * Create symlink to a file or directory.
+   * @param target The path of the destination that the
+   *               link points to.
+   * @param link The path of the link being created.
+   * @param dirPerm permissions to use when creating parent directories
+   * @param createParent - if true then missing parent dirs are created
+   *                       if false then parent must exist
+   *
+   * @throws AccessControlException permission denied
+   * @throws FileAlreadyExistsException If file <code>link</code> already exists
+   * @throws FileNotFoundException If parent of <code>link</code> does not exist
+   *           and <code>createParent</code> is false
+   * @throws ParentNotDirectoryException If parent of <code>link</code> is not a
+   *           directory.
+   * @throws UnresolvedLinkException if <code>link</target> contains a symlink. 
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred
+   */
+  @AtMostOnce
+  public void createSymlink(String target, String link, FsPermission dirPerm,
+      boolean createParent) throws AccessControlException,
+      FileAlreadyExistsException, FileNotFoundException,
+      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+      SnapshotAccessControlException, IOException;
+
+  /**
+   * Return the target of the given symlink. If there is an intermediate
+   * symlink in the path (ie a symlink leading up to the final path component)
+   * then the given path is returned with this symlink resolved.
+   *
+   * @param path The path with a link that needs resolution.
+   * @return The path after resolving the first symbolic link in the path.
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException If <code>path</code> does not exist
+   * @throws IOException If the given path does not refer to a symlink
+   *           or an I/O error occurred
+   */
+  @Idempotent
+  public String getLinkTarget(String path) throws AccessControlException,
+      FileNotFoundException, IOException; 
+  
+  /**
+   * Get a new generation stamp together with an access token for 
+   * a block under construction
+   * 
+   * This method is called only when a client needs to recover a failed
+   * pipeline or set up a pipeline for appending to a block.
+   * 
+   * @param block a block
+   * @param clientName the name of the client
+   * @return a located block with a new generation stamp and an access token
+   * @throws IOException if any error occurs
+   */
+  @Idempotent
+  public LocatedBlock updateBlockForPipeline(ExtendedBlock block,
+      String clientName) throws IOException;
+
+  /**
+   * Update a pipeline for a block under construction
+   * 
+   * @param clientName the name of the client
+   * @param oldBlock the old block
+   * @param newBlock the new block containing new generation stamp and length
+   * @param newNodes datanodes in the pipeline
+   * @throws IOException if any error occurs
+   */
+  @AtMostOnce
+  public void updatePipeline(String clientName, ExtendedBlock oldBlock, 
+      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
+      throws IOException;
+
+  /**
+   * Get a valid Delegation Token.
+   * 
+   * @param renewer the designated renewer for the token
+   * @return Token<DelegationTokenIdentifier>
+   * @throws IOException
+   */
+  @Idempotent
+  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer) 
+      throws IOException;
+
+  /**
+   * Renew an existing delegation token.
+   * 
+   * @param token delegation token obtained earlier
+   * @return the new expiration time
+   * @throws IOException
+   */
+  @Idempotent
+  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
+      throws IOException;
+  
+  /**
+   * Cancel an existing delegation token.
+   * 
+   * @param token delegation token
+   * @throws IOException
+   */
+  @Idempotent
+  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+      throws IOException;
+  
+  /**
+   * @return encryption key so a client can encrypt data sent via the
+   *         DataTransferProtocol to/from DataNodes.
+   * @throws IOException
+   */
+  @Idempotent
+  public DataEncryptionKey getDataEncryptionKey() throws IOException;
+  
+  /**
+   * Create a snapshot
+   * @param snapshotRoot the path that is being snapshotted
+   * @param snapshotName name of the snapshot created
+   * @return the snapshot path.
+   * @throws IOException
+   */
+  @AtMostOnce
+  public String createSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException;
+
+  /**
+   * Delete a specific snapshot of a snapshottable directory
+   * @param snapshotRoot  The snapshottable directory
+   * @param snapshotName Name of the snapshot for the snapshottable directory
+   * @throws IOException
+   */
+  @AtMostOnce
+  public void deleteSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException;
+  
+  /**
+   * Rename a snapshot
+   * @param snapshotRoot the directory path where the snapshot was taken 
+   * @param snapshotOldName old name of the snapshot
+   * @param snapshotNewName new name of the snapshot
+   * @throws IOException
+   */
+  @AtMostOnce
+  public void renameSnapshot(String snapshotRoot, String snapshotOldName,
+      String snapshotNewName) throws IOException;
+  
+  /**
+   * Allow snapshot on a directory.
+   * @param snapshotRoot the directory to be snapped
+   * @throws IOException on error
+   */
+  @Idempotent
+  public void allowSnapshot(String snapshotRoot)
+      throws IOException;
+    
+  /**
+   * Disallow snapshot on a directory.
+   * @param snapshotRoot the directory to disallow snapshot
+   * @throws IOException on error
+   */
+  @Idempotent
+  public void disallowSnapshot(String snapshotRoot)
+      throws IOException;
+  
+  /**
+   * Get the difference between two snapshots, or between a snapshot and the
+   * current tree of a directory.
+   * 
+   * @param snapshotRoot
+   *          full path of the directory where snapshots are taken
+   * @param fromSnapshot
+   *          snapshot name of the from point. Null indicates the current
+   *          tree
+   * @param toSnapshot
+   *          snapshot name of the to point. Null indicates the current
+   *          tree.
+   * @return The difference report represented as a {@link SnapshotDiffReport}.
+   * @throws IOException on error
+   */
+  @Idempotent
+  public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
+      String fromSnapshot, String toSnapshot) throws IOException;
+
+  /**
+   * Add a CacheDirective to the CacheManager.
+   * 
+   * @param directive A CacheDirectiveInfo to be added
+   * @param flags {@link CacheFlag}s to use for this operation.
+   * @return A CacheDirectiveInfo associated with the added directive
+   * @throws IOException if the directive could not be added
+   */
+  @AtMostOnce
+  public long addCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException;
+
+  /**
+   * Modify a CacheDirective in the CacheManager.
+   * 
+   * @param flags {@link CacheFlag}s to use for this operation.
+   * @throws IOException if the directive could not be modified
+   */
+  @AtMostOnce
+  public void modifyCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException;
+
+  /**
+   * Remove a CacheDirectiveInfo from the CacheManager.
+   * 
+   * @param id of a CacheDirectiveInfo
+   * @throws IOException if the cache directive could not be removed
+   */
+  @AtMostOnce
+  public void removeCacheDirective(long id) throws IOException;
+
+  /**
+   * List the set of cached paths of a cache pool. Incrementally fetches results
+   * from the server.
+   * 
+   * @param prevId The last listed entry ID, or -1 if this is the first call to
+   *               listCacheDirectives.
+   * @param filter Parameters to use to filter the list results, 
+   *               or null to display all directives visible to us.
+   * @return A batch of CacheDirectiveEntry objects.
+   */
+  @Idempotent
+  public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(
+      long prevId, CacheDirectiveInfo filter) throws IOException;
+
+  /**
+   * Add a new cache pool.
+   * 
+   * @param info Description of the new cache pool
+   * @throws IOException If the request could not be completed.
+   */
+  @AtMostOnce
+  public void addCachePool(CachePoolInfo info) throws IOException;
+
+  /**
+   * Modify an existing cache pool.
+   *
+   * @param req
+   *          The request to modify a cache pool.
+   * @throws IOException 
+   *          If the request could not be completed.
+   */
+  @AtMostOnce
+  public void modifyCachePool(CachePoolInfo req) throws IOException;
+  
+  /**
+   * Remove a cache pool.
+   * 
+   * @param pool name of the cache pool to remove.
+   * @throws IOException if the cache pool did not exist, or could not be
+   *           removed.
+   */
+  @AtMostOnce
+  public void removeCachePool(String pool) throws IOException;
+
+  /**
+   * List the set of cache pools. Incrementally fetches results from the server.
+   * 
+   * @param prevPool name of the last pool listed, or the empty string if this is
+   *          the first invocation of listCachePools
+   * @return A batch of CachePoolEntry objects.
+   */
+  @Idempotent
+  public BatchedEntries<CachePoolEntry> listCachePools(String prevPool)
+      throws IOException;
+
+  /**
+   * Modifies ACL entries of files and directories.  This method can add new ACL
+   * entries or modify the permissions on existing ACL entries.  All existing
+   * ACL entries that are not specified in this call are retained without
+   * changes.  (Modifications are merged into the current ACL.)
+   */
+  @Idempotent
+  public void modifyAclEntries(String src, List<AclEntry> aclSpec)
+      throws IOException;
+
+  /**
+   * Removes ACL entries from files and directories.  Other ACL entries are
+   * retained.
+   */
+  @Idempotent
+  public void removeAclEntries(String src, List<AclEntry> aclSpec)
+      throws IOException;
+
+  /**
+   * Removes all default ACL entries from files and directories.
+   */
+  @Idempotent
+  public void removeDefaultAcl(String src) throws IOException;
+
+  /**
+   * Removes all but the base ACL entries of files and directories.  The entries
+   * for user, group, and others are retained for compatibility with permission
+   * bits.
+   */
+  @Idempotent
+  public void removeAcl(String src) throws IOException;
+
+  /**
+   * Fully replaces ACL of files and directories, discarding all existing
+   * entries.
+   */
+  @Idempotent
+  public void setAcl(String src, List<AclEntry> aclSpec) throws IOException;
+
+  /**
+   * Gets the ACLs of files and directories.
+   */
+  @Idempotent
+  public AclStatus getAclStatus(String src) throws IOException;
+  
+  /**
+   * Create an encryption zone
+   */
+  @AtMostOnce
+  public void createEncryptionZone(String src, String keyName)
+    throws IOException;
+
+  /**
+   * Get the encryption zone for a path.
+   */
+  @Idempotent
+  public EncryptionZone getEZForPath(String src)
+    throws IOException;
+
+  /**
+   * Used to implement cursor-based batched listing of {@EncryptionZone}s.
+   *
+   * @param prevId ID of the last item in the previous batch. If there is no
+   *               previous batch, a negative value can be used.
+   * @return Batch of encryption zones.
+   */
+  @Idempotent
+  public BatchedEntries<EncryptionZone> listEncryptionZones(
+      long prevId) throws IOException;
+
+  /**
+   * Set xattr of a file or directory.
+   * The name must be prefixed with the namespace followed by ".". For example,
+   * "user.attr".
+   * <p/>
+   * Refer to the HDFS extended attributes user documentation for details.
+   *
+   * @param src file or directory
+   * @param xAttr <code>XAttr</code> to set
+   * @param flag set flag
+   * @throws IOException
+   */
+  @AtMostOnce
+  public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag) 
+      throws IOException;
+  
+  /**
+   * Get xattrs of a file or directory. Values in xAttrs parameter are ignored.
+   * If xAttrs is null or empty, this is the same as getting all xattrs of the
+   * file or directory.  Only those xattrs for which the logged-in user has
+   * permissions to view are returned.
+   * <p/>
+   * Refer to the HDFS extended attributes user documentation for details.
+   *
+   * @param src file or directory
+   * @param xAttrs xAttrs to get
+   * @return List<XAttr> <code>XAttr</code> list 
+   * @throws IOException
+   */
+  @Idempotent
+  public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs) 
+      throws IOException;
+
+  /**
+   * List the xattrs names for a file or directory.
+   * Only the xattr names for which the logged in user has the permissions to
+   * access will be returned.
+   * <p/>
+   * Refer to the HDFS extended attributes user documentation for details.
+   *
+   * @param src file or directory
+   * @return List<XAttr> <code>XAttr</code> list
+   * @throws IOException
+   */
+  @Idempotent
+  public List<XAttr> listXAttrs(String src)
+      throws IOException;
+  
+  /**
+   * Remove xattr of a file or directory.Value in xAttr parameter is ignored.
+   * The name must be prefixed with the namespace followed by ".". For example,
+   * "user.attr".
+   * <p/>
+   * Refer to the HDFS extended attributes user documentation for details.
+   *
+   * @param src file or directory
+   * @param xAttr <code>XAttr</code> to remove
+   * @throws IOException
+   */
+  @AtMostOnce
+  public void removeXAttr(String src, XAttr xAttr) throws IOException;
+
+  /**
+   * Checks if the user can access a path.  The mode specifies which access
+   * checks to perform.  If the requested permissions are granted, then the
+   * method returns normally.  If access is denied, then the method throws an
+   * {@link AccessControlException}.
+   * In general, applications should avoid using this method, due to the risk of
+   * time-of-check/time-of-use race conditions.  The permissions on a file may
+   * change immediately after the access call returns.
+   *
+   * @param path Path to check
+   * @param mode type of access to check
+   * @throws AccessControlException if access is denied
+   * @throws FileNotFoundException if the path does not exist
+   * @throws IOException see specific implementation
+   */
+  @Idempotent
+  public void checkAccess(String path, FsAction mode) throws IOException;
+
+  /**
+   * Get the highest txid the NameNode knows has been written to the edit
+   * log, or -1 if the NameNode's edit log is not yet open for write. Used as
+   * the starting point for the inotify event stream.
+   */
+  @Idempotent
+  public long getCurrentEditLogTxid() throws IOException;
+
+  /**
+   * Get an ordered list of batches of events corresponding to the edit log
+   * transactions for txids equal to or greater than txid.
+   */
+  @Idempotent
+  public EventBatchList getEditsFromTxid(long txid) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8327744/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
new file mode 100644
index 0000000..8080bcf
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This exception is thrown when the name node is in safe mode.
+ * Client cannot modified namespace until the safe mode is off. 
+ * 
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class SafeModeException extends IOException {
+  private static final long serialVersionUID = 1L;
+  public SafeModeException(String msg) {
+    super(msg);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8327744/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 3f72608..5ce2863 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
@@ -490,7 +490,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS = "dfs.short.circuit.shared.memory.watcher.interrupt.check.ms";
   public static final int     DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT = 60000;
   public static final String  DFS_NAMENODE_KEYTAB_FILE_KEY = "dfs.namenode.keytab.file";
-  public static final String  DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY = "dfs.namenode.kerberos.principal";
+  public static final String  DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY =
+      HdfsClientConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY;
   @Deprecated
   public static final String  DFS_NAMENODE_USER_NAME_KEY = DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY;
   public static final String  DFS_NAMENODE_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY = "dfs.namenode.kerberos.internal.spnego.principal";


[04/50] [abbrv] hadoop git commit: HDFS-8361. Choose SSD over DISK in block placement.

Posted by ar...@apache.org.
HDFS-8361. Choose SSD over DISK in block placement.


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

Branch: refs/heads/HDFS-7240
Commit: 175e6d120fc34710048c2e9512dd270410172803
Parents: 1b6695a
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jun 15 17:12:01 2015 -0700
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Mon Jun 15 17:12:01 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/StorageType.java  |  7 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  2 +
 .../hadoop/hdfs/TestBlockStoragePolicy.java     | 75 +++++++++++++++++++-
 3 files changed, 80 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/175e6d12/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java
index 68069d7..0948801 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java
@@ -33,10 +33,11 @@ import org.apache.hadoop.util.StringUtils;
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
 public enum StorageType {
-  DISK(false),
+  // sorted by the speed of the storage types, from fast to slow
+  RAM_DISK(true),
   SSD(false),
-  ARCHIVE(false),
-  RAM_DISK(true);
+  DISK(false),
+  ARCHIVE(false);
 
   private final boolean isTransient;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/175e6d12/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 9822575..79e7820 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -920,6 +920,8 @@ Release 2.7.1 - UNRELEASED
     HDFS-8521. Add VisibleForTesting annotation to
     BlockPoolSlice#selectReplicaToDelete. (cmccabe)
 
+    HDFS-8361. Choose SSD over DISK in block placement.  (szetszwo)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/175e6d12/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
index ea69f97..0d59ded 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
@@ -26,6 +26,7 @@ import java.util.*;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockStoragePolicySpi;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -40,7 +41,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
-import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
@@ -1153,6 +1154,50 @@ public class TestBlockStoragePolicy {
   }
 
   @Test
+  public void testChooseSsdOverDisk() throws Exception {
+    BlockStoragePolicy policy = new BlockStoragePolicy((byte) 9, "TEST1",
+        new StorageType[]{StorageType.SSD, StorageType.DISK,
+            StorageType.ARCHIVE}, new StorageType[]{}, new StorageType[]{});
+
+    final String[] racks = {"/d1/r1", "/d1/r1", "/d1/r1"};
+    final String[] hosts = {"host1", "host2", "host3"};
+    final StorageType[] disks = {StorageType.DISK, StorageType.DISK, StorageType.DISK};
+
+    final DatanodeStorageInfo[] diskStorages
+        = DFSTestUtil.createDatanodeStorageInfos(3, racks, hosts, disks);
+    final DatanodeDescriptor[] dataNodes
+        = DFSTestUtil.toDatanodeDescriptor(diskStorages);
+    for(int i = 0; i < dataNodes.length; i++) {
+      BlockManagerTestUtil.updateStorage(dataNodes[i],
+          new DatanodeStorage("ssd" + i, DatanodeStorage.State.NORMAL,
+              StorageType.SSD));
+    }
+
+    FileSystem.setDefaultUri(conf, "hdfs://localhost:0");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
+    File baseDir = PathUtils.getTestDir(TestReplicationPolicy.class);
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+        new File(baseDir, "name").getPath());
+    DFSTestUtil.formatNameNode(conf);
+    NameNode namenode = new NameNode(conf);
+
+    final BlockManager bm = namenode.getNamesystem().getBlockManager();
+    BlockPlacementPolicy replicator = bm.getBlockPlacementPolicy();
+    NetworkTopology cluster = bm.getDatanodeManager().getNetworkTopology();
+    for (DatanodeDescriptor datanode : dataNodes) {
+      cluster.add(datanode);
+    }
+
+    DatanodeStorageInfo[] targets = replicator.chooseTarget("/foo", 3,
+        dataNodes[0], Collections.<DatanodeStorageInfo>emptyList(), false,
+        new HashSet<Node>(), 0, policy);
+    System.out.println(policy.getName() + ": " + Arrays.asList(targets));
+    Assert.assertEquals(2, targets.length);
+    Assert.assertEquals(StorageType.SSD, targets[0].getStorageType());
+    Assert.assertEquals(StorageType.DISK, targets[1].getStorageType());
+  }
+
+  @Test
   public void testGetFileStoragePolicyAfterRestartNN() throws Exception {
     //HDFS8219
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
@@ -1233,4 +1278,32 @@ public class TestBlockStoragePolicy {
       cluster.shutdown();
     }
   }
+
+  @Test
+  public void testStorageType() {
+    final EnumMap<StorageType, Integer> map = new EnumMap<>(StorageType.class);
+
+    //put storage type is reversed order
+    map.put(StorageType.ARCHIVE, 1);
+    map.put(StorageType.DISK, 1);
+    map.put(StorageType.SSD, 1);
+    map.put(StorageType.RAM_DISK, 1);
+
+    {
+      final Iterator<StorageType> i = map.keySet().iterator();
+      Assert.assertEquals(StorageType.RAM_DISK, i.next());
+      Assert.assertEquals(StorageType.SSD, i.next());
+      Assert.assertEquals(StorageType.DISK, i.next());
+      Assert.assertEquals(StorageType.ARCHIVE, i.next());
+    }
+
+    {
+      final Iterator<Map.Entry<StorageType, Integer>> i
+          = map.entrySet().iterator();
+      Assert.assertEquals(StorageType.RAM_DISK, i.next().getKey());
+      Assert.assertEquals(StorageType.SSD, i.next().getKey());
+      Assert.assertEquals(StorageType.DISK, i.next().getKey());
+      Assert.assertEquals(StorageType.ARCHIVE, i.next().getKey());
+    }
+  }
 }


[05/50] [abbrv] hadoop git commit: YARN-3711. Documentation of ResourceManager HA should explain configurations about listen addresses. Contributed by Masatake Iwasaki.

Posted by ar...@apache.org.
YARN-3711. Documentation of ResourceManager HA should explain configurations about listen addresses. Contributed by Masatake Iwasaki.


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

Branch: refs/heads/HDFS-7240
Commit: e8c514373f2d258663497a33ffb3b231d0743b57
Parents: 175e6d1
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Tue Jun 16 10:12:38 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Tue Jun 16 10:12:38 2015 +0900

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                     |  3 +++
 .../src/site/markdown/ResourceManagerHA.md          | 16 +++++++++++++++-
 2 files changed, 18 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e8c51437/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index f5780c9..e442244 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -544,6 +544,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3723. Need to clearly document primaryFilter and otherInfo value type.
     (Zhijie Shen via xgong)
 
+    YARN-3711. Documentation of ResourceManager HA should explain configurations
+    about listen addresses. (Masatake Iwasaki via ozawa)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e8c51437/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md
index 596cba7..49669a1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md
@@ -67,7 +67,13 @@ Most of the failover functionality is tunable using various configuration proper
 | `yarn.resourcemanager.zk-address` | Address of the ZK-quorum. Used both for the state-store and embedded leader-election. |
 | `yarn.resourcemanager.ha.enabled` | Enable RM HA. |
 | `yarn.resourcemanager.ha.rm-ids` | List of logical IDs for the RMs. e.g., "rm1,rm2". |
-| `yarn.resourcemanager.hostname.*rm-id*` | For each *rm-id*, specify the hostname the RM corresponds to. Alternately, one could set each of the RM's service addresses. |
+| `yarn.resourcemanager.hostname.`*rm-id* | For each *rm-id*, specify the hostname the RM corresponds to. Alternately, one could set each of the RM's service addresses. |
+| `yarn.resourcemanager.address.`*rm-id* | For each *rm-id*, specify host:port for clients to submit jobs. If set, overrides the hostname set in `yarn.resourcemanager.hostname.`*rm-id*. |
+| `yarn.resourcemanager.scheduler.address.`*rm-id* | For each *rm-id*, specify scheduler host:port for ApplicationMasters to obtain resources. If set, overrides the hostname set in `yarn.resourcemanager.hostname.`*rm-id*. |
+| `yarn.resourcemanager.resource-tracker.address.`*rm-id* | For each *rm-id*, specify host:port for NodeManagers to connect. If set, overrides the hostname set in `yarn.resourcemanager.hostname.`*rm-id*. |
+| `yarn.resourcemanager.admin.address.`*rm-id* | For each *rm-id*, specify host:port for administrative commands. If set, overrides the hostname set in `yarn.resourcemanager.hostname.`*rm-id*. |
+| `yarn.resourcemanager.webapp.address.`*rm-id* | For each *rm-id*, specify host:port of the RM web application corresponds to. You do not need this if you set `yarn.http.policy` to `HTTPS_ONLY`. If set, overrides the hostname set in `yarn.resourcemanager.hostname.`*rm-id*. |
+| `yarn.resourcemanager.webapp.https.address.`*rm-id* | For each *rm-id*, specify host:port of the RM https web application corresponds to. You do not need this if you set `yarn.http.policy` to `HTTP_ONLY`. If set, overrides the hostname set in `yarn.resourcemanager.hostname.`*rm-id*. |
 | `yarn.resourcemanager.ha.id` | Identifies the RM in the ensemble. This is optional; however, if set, admins have to ensure that all the RMs have their own IDs in the config. |
 | `yarn.resourcemanager.ha.automatic-failover.enabled` | Enable automatic failover; By default, it is enabled only when HA is enabled. |
 | `yarn.resourcemanager.ha.automatic-failover.embedded` | Use embedded leader-elector to pick the Active RM, when automatic failover is enabled. By default, it is enabled only when HA is enabled. |
@@ -105,6 +111,14 @@ Here is the sample of minimal setup for RM failover.
   <value>master2</value>
 </property>
 <property>
+  <name>yarn.resourcemanager.webapp.address.rm1</name>
+  <value>master1:8088</value>
+</property>
+<property>
+  <name>yarn.resourcemanager.webapp.address.rm2</name>
+  <value>master2:8088</value>
+</property>
+<property>
   <name>yarn.resourcemanager.zk-address</name>
   <value>zk1:2181,zk2:2181,zk3:2181</value>
 </property>


[19/50] [abbrv] hadoop git commit: YARN-3148. Allow CORS related headers to passthrough in WebAppProxyServlet. Contributed by Varun Saxena.

Posted by ar...@apache.org.
YARN-3148. Allow CORS related headers to passthrough in
WebAppProxyServlet. Contributed by Varun Saxena.


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

Branch: refs/heads/HDFS-7240
Commit: ebb9a82519c622bb898e1eec5798c2298c726694
Parents: 318d2cd
Author: Devaraj K <de...@apache.org>
Authored: Wed Jun 17 15:56:18 2015 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Wed Jun 17 15:56:18 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../server/webproxy/WebAppProxyServlet.java     |  5 +-
 .../server/webproxy/TestWebAppProxyServlet.java | 57 ++++++++++++++++++++
 3 files changed, 64 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ebb9a825/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index c793a31..afe76bd 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -309,6 +309,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3789. Improve logs for LeafQueue#activateApplications().
     (Bibin A Chundatt via devaraj)
 
+    YARN-3148. Allow CORS related headers to passthrough in WebAppProxyServlet.
+    (Varun Saxena via devaraj)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ebb9a825/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
index b1e355d..33f36f0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
@@ -81,7 +81,10 @@ public class WebAppProxyServlet extends HttpServlet {
         "Accept-Encoding",
         "Accept-Language",
         "Accept-Charset",
-        "Content-Type"));
+        "Content-Type",
+        "Origin",
+        "Access-Control-Request-Method",
+        "Access-Control-Request-Headers"));
 
   public static final String PROXY_USER_COOKIE_NAME = "proxy-user";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ebb9a825/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
index 2a2ca2c..8e68c38 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
@@ -31,6 +31,7 @@ import java.net.HttpCookie;
 import java.net.HttpURLConnection;
 import java.net.URI;
 import java.net.URL;
+import java.util.Enumeration;
 import java.util.List;
 import java.util.Map;
 
@@ -72,6 +73,10 @@ public class TestWebAppProxyServlet {
 
   private static Server server;
   private static int originalPort = 0;
+  private static int numberOfHeaders = 0;
+  private static final String UNKNOWN_HEADER = "Unknown-Header";
+  private static boolean hasUnknownHeader = false;
+
 
   /**
    * Simple http server. Server should send answer with status 200
@@ -88,6 +93,9 @@ public class TestWebAppProxyServlet {
     originalPort = server.getConnectors()[0].getLocalPort();
     LOG.info("Running embedded servlet container at: http://localhost:"
         + originalPort);
+    // This property needs to be set otherwise CORS Headers will be dropped
+    // by HttpUrlConnection
+    System.setProperty("sun.net.http.allowRestrictedHeaders", "true");
   }
 
   @SuppressWarnings("serial")
@@ -96,6 +104,18 @@ public class TestWebAppProxyServlet {
     @Override
     protected void doGet(HttpServletRequest req, HttpServletResponse resp)
         throws ServletException, IOException {
+      int numHeaders = 0;
+      hasUnknownHeader = false;
+      @SuppressWarnings("unchecked")
+      Enumeration<String> names = req.getHeaderNames();
+      while(names.hasMoreElements()) {
+        String headerName = names.nextElement();
+        if (headerName.equals(UNKNOWN_HEADER)) {
+          hasUnknownHeader = true;
+        }
+        ++numHeaders;
+      }
+      numberOfHeaders = numHeaders;
       resp.setStatus(HttpServletResponse.SC_OK);
     }
 
@@ -206,6 +226,43 @@ public class TestWebAppProxyServlet {
     }
   }
 
+  @Test(timeout=5000)
+  public void testWebAppProxyPassThroughHeaders() throws Exception {
+    Configuration configuration = new Configuration();
+    configuration.set(YarnConfiguration.PROXY_ADDRESS, "localhost:9091");
+    configuration.setInt("hadoop.http.max.threads", 5);
+    WebAppProxyServerForTest proxy = new WebAppProxyServerForTest();
+    proxy.init(configuration);
+    proxy.start();
+
+    int proxyPort = proxy.proxy.proxyServer.getConnectorAddress(0).getPort();
+
+    try {
+      URL url = new URL("http://localhost:" + proxyPort + "/proxy/application_00_1");
+      HttpURLConnection proxyConn = (HttpURLConnection) url.openConnection();
+      // set headers
+      proxyConn.addRequestProperty("Origin", "http://www.someurl.com");
+      proxyConn.addRequestProperty("Access-Control-Request-Method", "GET");
+      proxyConn.addRequestProperty(
+          "Access-Control-Request-Headers", "Authorization");
+      proxyConn.addRequestProperty(UNKNOWN_HEADER, "unknown");
+      // Verify if four headers mentioned above have been added
+      assertEquals(proxyConn.getRequestProperties().size(), 4);
+      proxyConn.connect();
+      assertEquals(HttpURLConnection.HTTP_OK, proxyConn.getResponseCode());
+      // Verify if number of headers received by end server is 8.
+      // Eight headers include Accept, Host, Connection, User-Agent, Cookie,
+      // Origin, Access-Control-Request-Method and
+      // Access-Control-Request-Headers. Pls note that Unknown-Header is dropped
+      // by proxy as it is not in the list of allowed headers.
+      assertEquals(numberOfHeaders, 8);
+      assertFalse(hasUnknownHeader);
+    } finally {
+      proxy.close();
+    }
+  }
+
+
   /**
    * Test main method of WebAppProxyServer
    */


[16/50] [abbrv] hadoop git commit: HDFS-8606. Cleanup DFSOutputStream by removing unwanted changes from HDFS-8386. Contributed by Rakesh R

Posted by ar...@apache.org.
HDFS-8606. Cleanup DFSOutputStream by removing unwanted changes from HDFS-8386. 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/d4929f44
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d4929f44
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d4929f44

Branch: refs/heads/HDFS-7240
Commit: d4929f448f95815af99100780a08b172e0262c17
Parents: fbf55dc
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Tue Jun 16 18:08:29 2015 -0700
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Tue Jun 16 18:08:29 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4929f44/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 2d4c062..a36e047 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -635,6 +635,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8540.  Mover should exit with NO_MOVE_BLOCK if no block can be moved.
     (surendra singh lilhore via szetszwo)
 
+    HDFS-8606. Cleanup DFSOutputStream by removing unwanted changes
+    from HDFS-8386. (Rakesh R via szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4929f44/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 695e6da..4622be6 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -899,16 +899,9 @@ public class DFSOutputStream extends FSOutputSummer
   }
 
   /**
-   * Set the data streamer object.
-   */
-  protected synchronized void setStreamer(DataStreamer streamer) {
-    this.streamer = streamer;
-  }
-
-  /**
    * Returns the data streamer object.
    */
-  protected synchronized DataStreamer getStreamer() {
+  protected DataStreamer getStreamer() {
     return streamer;
   }
 }


[41/50] [abbrv] hadoop git commit: MAPREDUCE-6405. NullPointerException in App Attempts page. Contributed by Siqi Li and Gera Shegalov.

Posted by ar...@apache.org.
MAPREDUCE-6405. NullPointerException in App Attempts page.  Contributed by
Siqi Li and Gera Shegalov.


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

Branch: refs/heads/HDFS-7240
Commit: 311a4179cc53a2754d1185b1cb10e6adb1b95d3d
Parents: d112d18
Author: Devaraj K <de...@apache.org>
Authored: Sat Jun 20 10:35:04 2015 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Sat Jun 20 10:35:04 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 +
 .../mapreduce/v2/app/webapp/TaskPage.java       | 82 +++++++++++---------
 2 files changed, 49 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/311a4179/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 7e57804..dfac028 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -483,6 +483,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6373. The logger reports total input paths but it is referring
     to input files. (Bibin A Chundatt via devaraj)
 
+    MAPREDUCE-6405. NullPointerException in App Attempts page.
+    (Siqi Li and Gera Shegalov via devaraj)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/311a4179/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
index 758b02c..d9f17c8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
@@ -24,11 +24,14 @@ import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES_ID;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.tableInit;
 
+import java.util.EnumSet;
 import java.util.Collection;
 
 import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.MRConfig;
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
 import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
 import org.apache.hadoop.mapreduce.v2.util.MRWebAppUtil;
@@ -48,7 +51,6 @@ public class TaskPage extends AppView {
   static class AttemptsBlock extends HtmlBlock {
     final App app;
     final boolean enableUIActions;
-    private String stateURLFormat;
 
     @Inject
     AttemptsBlock(App ctx, Configuration conf) {
@@ -66,37 +68,36 @@ public class TaskPage extends AppView {
         return;
       }
 
+      JobId jobId = app.getJob().getID();
       if (enableUIActions) {
         // Kill task attempt
-        String appID = app.getJob().getID().getAppId().toString();
-        String jobID = app.getJob().getID().toString();
-        String taskID = app.getTask().getID().toString();
-        stateURLFormat =
-            String.format("/proxy/%s/ws/v1/mapreduce/jobs/%s/tasks/%s/"
-                + "attempts", appID, jobID, taskID) + "/%s/state";
-
-        String current =
-            String.format("/proxy/%s/mapreduce/task/%s", appID, taskID);
 
         StringBuilder script = new StringBuilder();
-        script.append("function confirmAction(stateURL) {")
-            .append(" b = confirm(\"Are you sure?\");")
-            .append(" if (b == true) {")
-            .append(" $.ajax({")
-            .append(" type: 'PUT',")
-            .append(" url: stateURL,")
-            .append(" contentType: 'application/json',")
-            .append(" data: '{\"state\":\"KILLED\"}',")
-            .append(" dataType: 'json'")
-            .append(" }).done(function(data){")
-            .append(" setTimeout(function(){")
-            .append(" location.href = '").append(current).append("';")
-            .append(" }, 1000);")
-            .append(" }).fail(function(data){")
-            .append(" console.log(data);")
-            .append(" });")
-            .append(" }")
-            .append("}");
+        script
+            .append("function confirmAction(appID, jobID, taskID, attID) {\n")
+            .append("  var b = confirm(\"Are you sure?\");\n")
+            .append("  if (b == true) {\n")
+            .append("    var current = '/proxy/' + appID")
+            .append("      + '/mapreduce/task/' + taskID;\n")
+            .append("    var stateURL = '/proxy/' + appID")
+            .append("      + '/ws/v1/mapreduce/jobs/' + jobID")
+            .append("      + '/tasks/' + taskID")
+            .append("      + '/attempts/' + attID + '/state';\n")
+            .append("    $.ajax({\n")
+            .append("      type: 'PUT',\n")
+            .append("      url: stateURL,\n")
+            .append("      contentType: 'application/json',\n")
+            .append("      data: '{\"state\":\"KILLED\"}',\n")
+            .append("      dataType: 'json'\n")
+            .append("    }).done(function(data) {\n")
+            .append("         setTimeout(function() {\n")
+            .append("           location.href = current;\n")
+            .append("         }, 1000);\n")
+            .append("    }).fail(function(data) {\n")
+            .append("         console.log(data);\n")
+            .append("    });\n")
+            .append("  }\n")
+            .append("}\n");
 
         html.script().$type("text/javascript")._(script.toString())._();
       }
@@ -135,8 +136,8 @@ public class TaskPage extends AppView {
               StringEscapeUtils.escapeHtml(ta.getStatus()))).append("\",\"")
 
         .append(nodeHttpAddr == null ? "N/A" :
-          "<a class='nodelink' href='" + MRWebAppUtil.getYARNWebappScheme() + nodeHttpAddr + "'>"
-          + nodeHttpAddr + "</a>")
+            "<a class='nodelink' href='" + MRWebAppUtil.getYARNWebappScheme() + nodeHttpAddr + "'>"
+                + nodeHttpAddr + "</a>")
         .append("\",\"")
 
         .append(ta.getAssignedContainerId() == null ? "N/A" :
@@ -151,12 +152,21 @@ public class TaskPage extends AppView {
         .append(StringEscapeUtils.escapeJavaScript(StringEscapeUtils.escapeHtml(
           diag)));
         if (enableUIActions) {
-          attemptsTableData.append("\",\"")
-          .append("<a href=javascript:void(0) onclick=confirmAction('")
-          .append(String.format(stateURLFormat, ta.getId()))
-          .append("');>Kill</a>")
-          .append("\"],\n");
-        } else {
+          attemptsTableData.append("\",\"");
+          if (EnumSet.of(
+                  TaskAttemptState.SUCCEEDED,
+                  TaskAttemptState.FAILED,
+                  TaskAttemptState.KILLED).contains(attempt.getState())) {
+            attemptsTableData.append("N/A");
+          } else {
+            attemptsTableData
+              .append("<a href=javascript:void(0) onclick=confirmAction('")
+              .append(jobId.getAppId()).append("','")
+              .append(jobId).append("','")
+              .append(attempt.getID().getTaskId()).append("','")
+              .append(ta.getId())
+              .append("');>Kill</a>");
+          }
           attemptsTableData.append("\"],\n");
         }
       }