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 sj...@apache.org on 2016/10/18 23:44:47 UTC

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

Repository: hadoop
Updated Branches:
  refs/heads/HADOOP-13070 d84ab8a57 -> f0e56e364


HADOOP-13700. Remove unthrown IOException from TrashPolicy#initialize and #getInstance signatures.


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

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

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


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


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


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

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

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

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

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


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

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

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

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

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

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

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

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


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


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

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


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

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

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


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


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


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

Posted by sj...@apache.org.
HADOOP-13723. AliyunOSSInputStream#read() should update read bytes stat correctly. Contributed by Mingliang Liu


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

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

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


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


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


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

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


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

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

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


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


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


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

Posted by sj...@apache.org.
HDFS-11002. Fix broken attr/getfattr/setfattr links in ExtendedAttributes.md. Contributed by Mingliang Liu.


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

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

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


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


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


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

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


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

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

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


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


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


[37/50] [abbrv] hadoop git commit: HDFS-9390. Block management for maintenance states.

Posted by sj...@apache.org.
HDFS-9390. Block management for maintenance states.


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

Branch: refs/heads/HADOOP-13070
Commit: b61fb267b92b2736920b4bd0c673d31e7632ebb9
Parents: f5d9235
Author: Ming Ma <mi...@apache.org>
Authored: Mon Oct 17 17:45:41 2016 -0700
Committer: Ming Ma <mi...@apache.org>
Committed: Mon Oct 17 17:45:41 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   5 +
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |  53 +-
 .../hadoop/hdfs/server/balancer/Dispatcher.java |  11 +-
 .../server/blockmanagement/BlockManager.java    | 249 ++++--
 .../BlockPlacementPolicyDefault.java            |   4 +-
 .../CacheReplicationMonitor.java                |   2 +-
 .../blockmanagement/DatanodeDescriptor.java     |  35 +-
 .../server/blockmanagement/DatanodeManager.java |  47 +-
 .../blockmanagement/DecommissionManager.java    | 142 +++-
 .../blockmanagement/ErasureCodingWork.java      |  16 +-
 .../blockmanagement/HeartbeatManager.java       |  23 +-
 .../blockmanagement/LowRedundancyBlocks.java    |  47 +-
 .../server/blockmanagement/NumberReplicas.java  |  30 +-
 .../blockmanagement/StorageTypeStats.java       |   8 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   9 +-
 .../src/main/resources/hdfs-default.xml         |   7 +
 .../apache/hadoop/hdfs/AdminStatesBaseTest.java |  20 +-
 .../apache/hadoop/hdfs/TestDecommission.java    |   2 +-
 .../hadoop/hdfs/TestMaintenanceState.java       | 775 +++++++++++++++++--
 .../blockmanagement/TestBlockManager.java       |   8 +-
 .../namenode/TestDecommissioningStatus.java     |  57 +-
 .../namenode/TestNamenodeCapacityReport.java    |  78 +-
 .../hadoop/hdfs/util/HostsFileWriter.java       |   1 +
 23 files changed, 1240 insertions(+), 389 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/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 10c0ad6..d54c109 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
@@ -220,6 +220,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.namenode.reconstruction.pending.timeout-sec";
   public static final int     DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_DEFAULT = -1;
 
+  public static final String  DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_KEY =
+      "dfs.namenode.maintenance.replication.min";
+  public static final int     DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_DEFAULT
+      = 1;
+
   public static final String  DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY =
       HdfsClientConfigKeys.DeprecatedKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY;
   public static final int     DFS_NAMENODE_REPLICATION_MAX_STREAMS_DEFAULT = 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index 83870cf..23166e2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -124,48 +124,57 @@ public class DFSUtil {
   }
 
   /**
-   * Compartor for sorting DataNodeInfo[] based on decommissioned states.
-   * Decommissioned nodes are moved to the end of the array on sorting with
-   * this compartor.
+   * Comparator for sorting DataNodeInfo[] based on
+   * decommissioned and entering_maintenance states.
    */
-  public static final Comparator<DatanodeInfo> DECOM_COMPARATOR = 
-    new Comparator<DatanodeInfo>() {
-      @Override
-      public int compare(DatanodeInfo a, DatanodeInfo b) {
-        return a.isDecommissioned() == b.isDecommissioned() ? 0 : 
-          a.isDecommissioned() ? 1 : -1;
+  public static class ServiceComparator implements Comparator<DatanodeInfo> {
+    @Override
+    public int compare(DatanodeInfo a, DatanodeInfo b) {
+      // Decommissioned nodes will still be moved to the end of the list
+      if (a.isDecommissioned()) {
+        return b.isDecommissioned() ? 0 : 1;
+      } else if (b.isDecommissioned()) {
+        return -1;
       }
-    };
 
+      // ENTERING_MAINTENANCE nodes should be after live nodes.
+      if (a.isEnteringMaintenance()) {
+        return b.isEnteringMaintenance() ? 0 : 1;
+      } else if (b.isEnteringMaintenance()) {
+        return -1;
+      } else {
+        return 0;
+      }
+    }
+  }
 
   /**
-   * Comparator for sorting DataNodeInfo[] based on decommissioned/stale states.
-   * Decommissioned/stale nodes are moved to the end of the array on sorting
-   * with this comparator.
-   */ 
+   * Comparator for sorting DataNodeInfo[] based on
+   * stale, decommissioned and entering_maintenance states.
+   * Order: live -> stale -> entering_maintenance -> decommissioned
+   */
   @InterfaceAudience.Private 
-  public static class DecomStaleComparator implements Comparator<DatanodeInfo> {
+  public static class ServiceAndStaleComparator extends ServiceComparator {
     private final long staleInterval;
 
     /**
-     * Constructor of DecomStaleComparator
+     * Constructor of ServiceAndStaleComparator
      * 
      * @param interval
      *          The time interval for marking datanodes as stale is passed from
      *          outside, since the interval may be changed dynamically
      */
-    public DecomStaleComparator(long interval) {
+    public ServiceAndStaleComparator(long interval) {
       this.staleInterval = interval;
     }
 
     @Override
     public int compare(DatanodeInfo a, DatanodeInfo b) {
-      // Decommissioned nodes will still be moved to the end of the list
-      if (a.isDecommissioned()) {
-        return b.isDecommissioned() ? 0 : 1;
-      } else if (b.isDecommissioned()) {
-        return -1;
+      int ret = super.compare(a, b);
+      if (ret != 0) {
+        return ret;
       }
+
       // Stale nodes will be moved behind the normal nodes
       boolean aStale = a.isStale(staleInterval);
       boolean bStale = b.isStale(staleInterval);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
index aea0ae4..e5c5e53 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
@@ -989,20 +989,17 @@ public class Dispatcher {
   }
 
   private boolean shouldIgnore(DatanodeInfo dn) {
-    // ignore decommissioned nodes
-    final boolean decommissioned = dn.isDecommissioned();
-    // ignore decommissioning nodes
-    final boolean decommissioning = dn.isDecommissionInProgress();
+    // ignore out-of-service nodes
+    final boolean outOfService = !dn.isInService();
     // ignore nodes in exclude list
     final boolean excluded = Util.isExcluded(excludedNodes, dn);
     // ignore nodes not in the include list (if include list is not empty)
     final boolean notIncluded = !Util.isIncluded(includedNodes, dn);
 
-    if (decommissioned || decommissioning || excluded || notIncluded) {
+    if (outOfService || excluded || notIncluded) {
       if (LOG.isTraceEnabled()) {
         LOG.trace("Excluding datanode " + dn
-            + ": decommissioned=" + decommissioned
-            + ", decommissioning=" + decommissioning
+            + ": outOfService=" + outOfService
             + ", excluded=" + excluded
             + ", notIncluded=" + notIncluded);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 7b13add..03bdb7a 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
@@ -126,6 +126,29 @@ import org.slf4j.LoggerFactory;
 
 /**
  * Keeps information related to the blocks stored in the Hadoop cluster.
+ * For block state management, it tries to maintain the  safety
+ * property of "# of live replicas == # of expected redundancy" under
+ * any events such as decommission, namenode failover, datanode failure.
+ *
+ * The motivation of maintenance mode is to allow admins quickly repair nodes
+ * without paying the cost of decommission. Thus with maintenance mode,
+ * # of live replicas doesn't have to be equal to # of expected redundancy.
+ * If any of the replica is in maintenance mode, the safety property
+ * is extended as follows. These property still apply for the case of zero
+ * maintenance replicas, thus we can use these safe property for all scenarios.
+ * a. # of live replicas >= # of min replication for maintenance.
+ * b. # of live replicas <= # of expected redundancy.
+ * c. # of live replicas and maintenance replicas >= # of expected redundancy.
+ *
+ * For regular replication, # of min live replicas for maintenance is determined
+ * by DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_KEY. This number has to <=
+ * DFS_NAMENODE_REPLICATION_MIN_KEY.
+ * For erasure encoding, # of min live replicas for maintenance is
+ * BlockInfoStriped#getRealDataBlockNum.
+ *
+ * Another safety property is to satisfy the block placement policy. While the
+ * policy is configurable, the replicas the policy is applied to are the live
+ * replicas + maintenance replicas.
  */
 @InterfaceAudience.Private
 public class BlockManager implements BlockStatsMXBean {
@@ -341,6 +364,11 @@ public class BlockManager implements BlockStatsMXBean {
 
   private final BlockIdManager blockIdManager;
 
+  /** Minimum live replicas needed for the datanode to be transitioned
+   * from ENTERING_MAINTENANCE to IN_MAINTENANCE.
+   */
+  private final short minReplicationToBeInMaintenance;
+
   public BlockManager(final Namesystem namesystem, boolean haEnabled,
       final Configuration conf) throws IOException {
     this.namesystem = namesystem;
@@ -373,13 +401,13 @@ public class BlockManager implements BlockStatsMXBean {
     this.maxCorruptFilesReturned = conf.getInt(
       DFSConfigKeys.DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED_KEY,
       DFSConfigKeys.DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED);
-    this.defaultReplication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 
-                                          DFSConfigKeys.DFS_REPLICATION_DEFAULT);
+    this.defaultReplication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
+        DFSConfigKeys.DFS_REPLICATION_DEFAULT);
 
-    final int maxR = conf.getInt(DFSConfigKeys.DFS_REPLICATION_MAX_KEY, 
-                                 DFSConfigKeys.DFS_REPLICATION_MAX_DEFAULT);
+    final int maxR = conf.getInt(DFSConfigKeys.DFS_REPLICATION_MAX_KEY,
+        DFSConfigKeys.DFS_REPLICATION_MAX_DEFAULT);
     final int minR = conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY,
-                                 DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT);
+        DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT);
     if (minR <= 0)
       throw new IOException("Unexpected configuration parameters: "
           + DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY
@@ -407,7 +435,7 @@ public class BlockManager implements BlockStatsMXBean {
     this.blocksInvalidateWorkPct = DFSUtil.getInvalidateWorkPctPerIteration(conf);
     this.blocksReplWorkMultiplier = DFSUtil.getReplWorkMultiplier(conf);
 
-    this.replicationRecheckInterval = 
+    this.replicationRecheckInterval =
       conf.getTimeDuration(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY,
           DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT,
           TimeUnit.SECONDS) * 1000L;
@@ -428,7 +456,7 @@ public class BlockManager implements BlockStatsMXBean {
     this.encryptDataTransfer =
         conf.getBoolean(DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY,
             DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_DEFAULT);
-    
+
     this.maxNumBlocksToLog =
         conf.getLong(DFSConfigKeys.DFS_MAX_NUM_BLOCKS_TO_LOG_KEY,
             DFSConfigKeys.DFS_MAX_NUM_BLOCKS_TO_LOG_DEFAULT);
@@ -438,6 +466,25 @@ public class BlockManager implements BlockStatsMXBean {
     this.getBlocksMinBlockSize = conf.getLongBytes(
         DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY,
         DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_DEFAULT);
+
+    final int minMaintenanceR = conf.getInt(
+        DFSConfigKeys.DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_KEY,
+        DFSConfigKeys.DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_DEFAULT);
+
+    if (minMaintenanceR < 0) {
+      throw new IOException("Unexpected configuration parameters: "
+          + DFSConfigKeys.DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_KEY
+          + " = " + minMaintenanceR + " < 0");
+    }
+    if (minMaintenanceR > minR) {
+      throw new IOException("Unexpected configuration parameters: "
+          + DFSConfigKeys.DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_KEY
+          + " = " + minMaintenanceR + " > "
+          + DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY
+          + " = " + minR);
+    }
+    this.minReplicationToBeInMaintenance = (short)minMaintenanceR;
+
     this.blockReportLeaseManager = new BlockReportLeaseManager(conf);
 
     bmSafeMode = new BlockManagerSafeMode(this, namesystem, haEnabled, conf);
@@ -668,7 +715,7 @@ public class BlockManager implements BlockStatsMXBean {
     // Dump all datanodes
     getDatanodeManager().datanodeDump(out);
   }
-  
+
   /**
    * Dump the metadata for the given block in a human-readable
    * form.
@@ -697,12 +744,12 @@ public class BlockManager implements BlockStatsMXBean {
       out.print(fileName + ": ");
     }
     // l: == live:, d: == decommissioned c: == corrupt e: == excess
-    out.print(block + ((usableReplicas > 0)? "" : " MISSING") + 
+    out.print(block + ((usableReplicas > 0)? "" : " MISSING") +
               " (replicas:" +
               " l: " + numReplicas.liveReplicas() +
               " d: " + numReplicas.decommissionedAndDecommissioning() +
               " c: " + numReplicas.corruptReplicas() +
-              " e: " + numReplicas.excessReplicas() + ") "); 
+              " e: " + numReplicas.excessReplicas() + ") ");
 
     Collection<DatanodeDescriptor> corruptNodes = 
                                   corruptReplicas.getNodes(block);
@@ -750,6 +797,18 @@ public class BlockManager implements BlockStatsMXBean {
     }
   }
 
+  public short getMinReplicationToBeInMaintenance() {
+    return minReplicationToBeInMaintenance;
+  }
+
+  private short getMinMaintenanceStorageNum(BlockInfo block) {
+    if (block.isStriped()) {
+      return ((BlockInfoStriped) block).getRealDataBlockNum();
+    } else {
+      return minReplicationToBeInMaintenance;
+    }
+  }
+
   public boolean hasMinStorage(BlockInfo block) {
     return countNodes(block).liveReplicas() >= getMinStorageNum(block);
   }
@@ -942,7 +1001,7 @@ public class BlockManager implements BlockStatsMXBean {
     NumberReplicas replicas = countNodes(lastBlock);
     neededReconstruction.remove(lastBlock, replicas.liveReplicas(),
         replicas.readOnlyReplicas(),
-        replicas.decommissionedAndDecommissioning(), getRedundancy(lastBlock));
+        replicas.outOfServiceReplicas(), getExpectedRedundancyNum(lastBlock));
     pendingReconstruction.remove(lastBlock);
 
     // remove this block from the list of pending blocks to be deleted. 
@@ -1078,7 +1137,8 @@ public class BlockManager implements BlockStatsMXBean {
     } else {
       isCorrupt = numCorruptReplicas != 0 && numCorruptReplicas == numNodes;
     }
-    final int numMachines = isCorrupt ? numNodes: numNodes - numCorruptReplicas;
+    int numMachines = isCorrupt ? numNodes: numNodes - numCorruptReplicas;
+    numMachines -= numReplicas.maintenanceNotForReadReplicas();
     DatanodeStorageInfo[] machines = new DatanodeStorageInfo[numMachines];
     final byte[] blockIndices = blk.isStriped() ? new byte[numMachines] : null;
     int j = 0, i = 0;
@@ -1086,11 +1146,17 @@ public class BlockManager implements BlockStatsMXBean {
       final boolean noCorrupt = (numCorruptReplicas == 0);
       for(DatanodeStorageInfo storage : blocksMap.getStorages(blk)) {
         if (storage.getState() != State.FAILED) {
+          final DatanodeDescriptor d = storage.getDatanodeDescriptor();
+          // Don't pick IN_MAINTENANCE or dead ENTERING_MAINTENANCE states.
+          if (d.isInMaintenance()
+              || (d.isEnteringMaintenance() && !d.isAlive())) {
+            continue;
+          }
+
           if (noCorrupt) {
             machines[j++] = storage;
             i = setBlockIndices(blk, blockIndices, i, storage);
           } else {
-            final DatanodeDescriptor d = storage.getDatanodeDescriptor();
             final boolean replicaCorrupt = isReplicaCorrupt(blk, d);
             if (isCorrupt || !replicaCorrupt) {
               machines[j++] = storage;
@@ -1106,7 +1172,7 @@ public class BlockManager implements BlockStatsMXBean {
     }
 
     assert j == machines.length :
-      "isCorrupt: " + isCorrupt + 
+      "isCorrupt: " + isCorrupt +
       " numMachines: " + numMachines +
       " numNodes: " + numNodes +
       " numCorrupt: " + numCorruptNodes +
@@ -1700,8 +1766,11 @@ public class BlockManager implements BlockStatsMXBean {
     return scheduledWork;
   }
 
+  // Check if the number of live + pending replicas satisfies
+  // the expected redundancy.
   boolean hasEnoughEffectiveReplicas(BlockInfo block,
-      NumberReplicas numReplicas, int pendingReplicaNum, int required) {
+      NumberReplicas numReplicas, int pendingReplicaNum) {
+    int required = getExpectedLiveRedundancyNum(block, numReplicas);
     int numEffectiveReplicas = numReplicas.liveReplicas() + pendingReplicaNum;
     return (numEffectiveReplicas >= required) &&
         (pendingReplicaNum > 0 || isPlacementPolicySatisfied(block));
@@ -1716,8 +1785,6 @@ public class BlockManager implements BlockStatsMXBean {
       return null;
     }
 
-    short requiredRedundancy = getExpectedRedundancyNum(block);
-
     // get a source data-node
     List<DatanodeDescriptor> containingNodes = new ArrayList<>();
     List<DatanodeStorageInfo> liveReplicaNodes = new ArrayList<>();
@@ -1726,6 +1793,8 @@ public class BlockManager implements BlockStatsMXBean {
     final DatanodeDescriptor[] srcNodes = chooseSourceDatanodes(block,
         containingNodes, liveReplicaNodes, numReplicas,
         liveBlockIndices, priority);
+    short requiredRedundancy = getExpectedLiveRedundancyNum(block,
+        numReplicas);
     if(srcNodes == null || srcNodes.length == 0) {
       // block can not be reconstructed from any node
       LOG.debug("Block " + block + " cannot be reconstructed " +
@@ -1738,8 +1807,7 @@ public class BlockManager implements BlockStatsMXBean {
     assert liveReplicaNodes.size() >= numReplicas.liveReplicas();
 
     int pendingNum = pendingReconstruction.getNumReplicas(block);
-    if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum,
-        requiredRedundancy)) {
+    if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum)) {
       neededReconstruction.remove(block, priority);
       blockLog.debug("BLOCK* Removing {} from neededReconstruction as" +
           " it has enough replicas", block);
@@ -1763,9 +1831,11 @@ public class BlockManager implements BlockStatsMXBean {
 
       // should reconstruct all the internal blocks before scheduling
       // replication task for decommissioning node(s).
-      if (additionalReplRequired - numReplicas.decommissioning() > 0) {
-        additionalReplRequired = additionalReplRequired
-            - numReplicas.decommissioning();
+      if (additionalReplRequired - numReplicas.decommissioning() -
+          numReplicas.liveEnteringMaintenanceReplicas() > 0) {
+        additionalReplRequired = additionalReplRequired -
+            numReplicas.decommissioning() -
+            numReplicas.liveEnteringMaintenanceReplicas();
       }
       byte[] indices = new byte[liveBlockIndices.size()];
       for (int i = 0 ; i < liveBlockIndices.size(); i++) {
@@ -1807,11 +1877,11 @@ public class BlockManager implements BlockStatsMXBean {
     }
 
     // do not schedule more if enough replicas is already pending
-    final short requiredRedundancy = getExpectedRedundancyNum(block);
     NumberReplicas numReplicas = countNodes(block);
+    final short requiredRedundancy =
+        getExpectedLiveRedundancyNum(block, numReplicas);
     final int pendingNum = pendingReconstruction.getNumReplicas(block);
-    if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum,
-        requiredRedundancy)) {
+    if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum)) {
       neededReconstruction.remove(block, priority);
       rw.resetTargets();
       blockLog.debug("BLOCK* Removing {} from neededReplications as" +
@@ -1880,7 +1950,7 @@ public class BlockManager implements BlockStatsMXBean {
    * @throws IOException
    *           if the number of targets < minimum replication.
    * @see BlockPlacementPolicy#chooseTarget(String, int, Node,
-   *      Set, long, List, BlockStoragePolicy)
+   *      Set, long, List, BlockStoragePolicy, EnumSet)
    */
   public DatanodeStorageInfo[] chooseTarget4NewBlock(final String src,
       final int numOfReplicas, final Node client,
@@ -1987,13 +2057,15 @@ public class BlockManager implements BlockStatsMXBean {
         continue;
       }
 
-      // never use already decommissioned nodes or unknown state replicas
-      if (state == null || state == StoredReplicaState.DECOMMISSIONED) {
+      // never use already decommissioned nodes, maintenance node not
+      // suitable for read or unknown state replicas.
+      if (state == null || state == StoredReplicaState.DECOMMISSIONED
+          || state == StoredReplicaState.MAINTENANCE_NOT_FOR_READ) {
         continue;
       }
 
       if (priority != LowRedundancyBlocks.QUEUE_HIGHEST_PRIORITY
-          && !node.isDecommissionInProgress() 
+          && (!node.isDecommissionInProgress() && !node.isEnteringMaintenance())
           && node.getNumberOfBlocksToBeReplicated() >= maxReplicationStreams) {
         continue; // already reached replication limit
       }
@@ -2045,10 +2117,10 @@ public class BlockManager implements BlockStatsMXBean {
             continue;
           }
           NumberReplicas num = countNodes(timedOutItems[i]);
-          if (isNeededReconstruction(bi, num.liveReplicas())) {
+          if (isNeededReconstruction(bi, num)) {
             neededReconstruction.add(bi, num.liveReplicas(),
-                num.readOnlyReplicas(), num.decommissionedAndDecommissioning(),
-                getRedundancy(bi));
+                num.readOnlyReplicas(), num.outOfServiceReplicas(),
+                getExpectedRedundancyNum(bi));
           }
         }
       } finally {
@@ -3014,10 +3086,9 @@ public class BlockManager implements BlockStatsMXBean {
 
     // handle low redundancy/extra redundancy
     short fileRedundancy = getExpectedRedundancyNum(storedBlock);
-    if (!isNeededReconstruction(storedBlock, numCurrentReplica)) {
+    if (!isNeededReconstruction(storedBlock, num, pendingNum)) {
       neededReconstruction.remove(storedBlock, numCurrentReplica,
-          num.readOnlyReplicas(),
-          num.decommissionedAndDecommissioning(), fileRedundancy);
+          num.readOnlyReplicas(), num.outOfServiceReplicas(), fileRedundancy);
     } else {
       updateNeededReconstructions(storedBlock, curReplicaDelta, 0);
     }
@@ -3040,6 +3111,10 @@ public class BlockManager implements BlockStatsMXBean {
     return storedBlock;
   }
 
+  // If there is any maintenance replica, we don't have to restore
+  // the condition of live + maintenance == expected. We allow
+  // live + maintenance >= expected. The extra redundancy will be removed
+  // when the maintenance node changes to live.
   private boolean shouldProcessExtraRedundancy(NumberReplicas num,
       int expectedNum) {
     final int numCurrent = num.liveReplicas();
@@ -3255,9 +3330,9 @@ public class BlockManager implements BlockStatsMXBean {
     NumberReplicas num = countNodes(block);
     final int numCurrentReplica = num.liveReplicas();
     // add to low redundancy queue if need to be
-    if (isNeededReconstruction(block, numCurrentReplica)) {
+    if (isNeededReconstruction(block, num)) {
       if (neededReconstruction.add(block, numCurrentReplica,
-          num.readOnlyReplicas(), num.decommissionedAndDecommissioning(),
+          num.readOnlyReplicas(), num.outOfServiceReplicas(),
           expectedRedundancy)) {
         return MisReplicationResult.UNDER_REPLICATED;
       }
@@ -3290,9 +3365,9 @@ public class BlockManager implements BlockStatsMXBean {
 
     // update neededReconstruction priority queues
     b.setReplication(newRepl);
+    NumberReplicas num = countNodes(b);
     updateNeededReconstructions(b, 0, newRepl - oldRepl);
-
-    if (oldRepl > newRepl) {
+    if (shouldProcessExtraRedundancy(num, newRepl)) {
       processExtraRedundancyBlock(b, newRepl, null, null);
     }
   }
@@ -3318,14 +3393,14 @@ public class BlockManager implements BlockStatsMXBean {
       }
       final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
       if (storage.areBlockContentsStale()) {
-        LOG.trace("BLOCK* processOverReplicatedBlock: Postponing {}"
+        LOG.trace("BLOCK* processExtraRedundancyBlock: Postponing {}"
             + " since storage {} does not yet have up-to-date information.",
             block, storage);
         postponeBlock(block);
         return;
       }
       if (!isExcess(cur, block)) {
-        if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
+        if (cur.isInService()) {
           // exclude corrupt replicas
           if (corruptNodes == null || !corruptNodes.contains(cur)) {
             nonExcess.add(storage);
@@ -3766,7 +3841,7 @@ public class BlockManager implements BlockStatsMXBean {
     return countNodes(b, false);
   }
 
-  private NumberReplicas countNodes(BlockInfo b, boolean inStartupSafeMode) {
+  NumberReplicas countNodes(BlockInfo b, boolean inStartupSafeMode) {
     NumberReplicas numberReplicas = new NumberReplicas();
     Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b);
     if (b.isStriped()) {
@@ -3797,6 +3872,12 @@ public class BlockManager implements BlockStatsMXBean {
         s = StoredReplicaState.DECOMMISSIONING;
       } else if (node.isDecommissioned()) {
         s = StoredReplicaState.DECOMMISSIONED;
+      } else if (node.isMaintenance()) {
+        if (node.isInMaintenance() || !node.isAlive()) {
+          s = StoredReplicaState.MAINTENANCE_NOT_FOR_READ;
+        } else {
+          s = StoredReplicaState.MAINTENANCE_FOR_READ;
+        }
       } else if (isExcess(node, b)) {
         s = StoredReplicaState.EXCESS;
       } else {
@@ -3868,11 +3949,11 @@ public class BlockManager implements BlockStatsMXBean {
   }
   
   /**
-   * On stopping decommission, check if the node has excess replicas.
+   * On putting the node in service, check if the node has excess replicas.
    * If there are any excess replicas, call processExtraRedundancyBlock().
    * Process extra redundancy blocks only when active NN is out of safe mode.
    */
-  void processExtraRedundancyBlocksOnReCommission(
+  void processExtraRedundancyBlocksOnInService(
       final DatanodeDescriptor srcNode) {
     if (!isPopulatingReplQueues()) {
       return;
@@ -3881,7 +3962,7 @@ public class BlockManager implements BlockStatsMXBean {
     int numExtraRedundancy = 0;
     while(it.hasNext()) {
       final BlockInfo block = it.next();
-      int expectedReplication = this.getRedundancy(block);
+      int expectedReplication = this.getExpectedRedundancyNum(block);
       NumberReplicas num = countNodes(block);
       if (shouldProcessExtraRedundancy(num, expectedReplication)) {
         // extra redundancy block
@@ -3891,14 +3972,15 @@ public class BlockManager implements BlockStatsMXBean {
       }
     }
     LOG.info("Invalidated " + numExtraRedundancy
-        + " extra redundancy blocks on " + srcNode + " during recommissioning");
+        + " extra redundancy blocks on " + srcNode + " after it is in service");
   }
 
   /**
-   * Returns whether a node can be safely decommissioned based on its 
-   * liveness. Dead nodes cannot always be safely decommissioned.
+   * Returns whether a node can be safely decommissioned or in maintenance
+   * based on its liveness. Dead nodes cannot always be safely decommissioned
+   * or in maintenance.
    */
-  boolean isNodeHealthyForDecommission(DatanodeDescriptor node) {
+  boolean isNodeHealthyForDecommissionOrMaintenance(DatanodeDescriptor node) {
     if (!node.checkBlockReportReceived()) {
       LOG.info("Node {} hasn't sent its first block report.", node);
       return false;
@@ -3912,17 +3994,18 @@ public class BlockManager implements BlockStatsMXBean {
     if (pendingReconstructionBlocksCount == 0 &&
         lowRedundancyBlocksCount == 0) {
       LOG.info("Node {} is dead and there are no low redundancy" +
-          " blocks or blocks pending reconstruction. Safe to decommission.",
-          node);
+          " blocks or blocks pending reconstruction. Safe to decommission or",
+          " put in maintenance.", node);
       return true;
     }
 
     LOG.warn("Node {} is dead " +
-        "while decommission is in progress. Cannot be safely " +
-        "decommissioned since there is risk of reduced " +
-        "data durability or data loss. Either restart the failed node or" +
-        " force decommissioning by removing, calling refreshNodes, " +
-        "then re-adding to the excludes files.", node);
+        "while in {}. Cannot be safely " +
+        "decommissioned or be in maintenance since there is risk of reduced " +
+        "data durability or data loss. Either restart the failed node or " +
+        "force decommissioning or maintenance by removing, calling " +
+        "refreshNodes, then re-adding to the excludes or host config files.",
+        node, node.getAdminState());
     return false;
   }
 
@@ -3990,17 +4073,16 @@ public class BlockManager implements BlockStatsMXBean {
       }
       NumberReplicas repl = countNodes(block);
       int pendingNum = pendingReconstruction.getNumReplicas(block);
-      int curExpectedReplicas = getRedundancy(block);
-      if (!hasEnoughEffectiveReplicas(block, repl, pendingNum,
-          curExpectedReplicas)) {
+      int curExpectedReplicas = getExpectedRedundancyNum(block);
+      if (!hasEnoughEffectiveReplicas(block, repl, pendingNum)) {
         neededReconstruction.update(block, repl.liveReplicas() + pendingNum,
-            repl.readOnlyReplicas(), repl.decommissionedAndDecommissioning(),
+            repl.readOnlyReplicas(), repl.outOfServiceReplicas(),
             curExpectedReplicas, curReplicasDelta, expectedReplicasDelta);
       } else {
         int oldReplicas = repl.liveReplicas() + pendingNum - curReplicasDelta;
         int oldExpectedReplicas = curExpectedReplicas-expectedReplicasDelta;
         neededReconstruction.remove(block, oldReplicas, repl.readOnlyReplicas(),
-            repl.decommissionedAndDecommissioning(), oldExpectedReplicas);
+            repl.outOfServiceReplicas(), oldExpectedReplicas);
       }
     } finally {
       namesystem.writeUnlock();
@@ -4018,24 +4100,15 @@ public class BlockManager implements BlockStatsMXBean {
       short expected = getExpectedRedundancyNum(block);
       final NumberReplicas n = countNodes(block);
       final int pending = pendingReconstruction.getNumReplicas(block);
-      if (!hasEnoughEffectiveReplicas(block, n, pending, expected)) {
+      if (!hasEnoughEffectiveReplicas(block, n, pending)) {
         neededReconstruction.add(block, n.liveReplicas() + pending,
-            n.readOnlyReplicas(),
-            n.decommissionedAndDecommissioning(), expected);
+            n.readOnlyReplicas(), n.outOfServiceReplicas(), expected);
       } else if (shouldProcessExtraRedundancy(n, expected)) {
         processExtraRedundancyBlock(block, expected, null, null);
       }
     }
   }
 
-  /** 
-   * @return 0 if the block is not found;
-   *         otherwise, return the replication factor of the block.
-   */
-  private int getRedundancy(BlockInfo block) {
-    return getExpectedRedundancyNum(block);
-  }
-
   /**
    * Get blocks to invalidate for <i>nodeId</i>
    * in {@link #invalidateBlocks}.
@@ -4088,6 +4161,8 @@ public class BlockManager implements BlockStatsMXBean {
         .getNodes(storedBlock);
     for (DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock)) {
       final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
+      // Nodes under maintenance should be counted as valid replicas from
+      // rack policy point of view.
       if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()
           && ((corruptNodes == null) || !corruptNodes.contains(cur))) {
         liveNodes.add(cur);
@@ -4102,14 +4177,36 @@ public class BlockManager implements BlockStatsMXBean {
         .isPlacementPolicySatisfied();
   }
 
+  boolean isNeededReconstructionForMaintenance(BlockInfo storedBlock,
+      NumberReplicas numberReplicas) {
+    return storedBlock.isComplete() && (numberReplicas.liveReplicas() <
+        getMinMaintenanceStorageNum(storedBlock) ||
+        !isPlacementPolicySatisfied(storedBlock));
+  }
+
+  boolean isNeededReconstruction(BlockInfo storedBlock,
+      NumberReplicas numberReplicas) {
+    return isNeededReconstruction(storedBlock, numberReplicas, 0);
+  }
+
   /**
    * A block needs reconstruction if the number of redundancies is less than
    * expected or if it does not have enough racks.
    */
-  boolean isNeededReconstruction(BlockInfo storedBlock, int current) {
-    int expected = getExpectedRedundancyNum(storedBlock);
-    return storedBlock.isComplete()
-        && (current < expected || !isPlacementPolicySatisfied(storedBlock));
+  boolean isNeededReconstruction(BlockInfo storedBlock,
+      NumberReplicas numberReplicas, int pending) {
+    return storedBlock.isComplete() &&
+        !hasEnoughEffectiveReplicas(storedBlock, numberReplicas, pending);
+  }
+
+  // Exclude maintenance, but make sure it has minimal live replicas
+  // to satisfy the maintenance requirement.
+  public short getExpectedLiveRedundancyNum(BlockInfo block,
+      NumberReplicas numberReplicas) {
+    final short expectedRedundancy = getExpectedRedundancyNum(block);
+    return (short)Math.max(expectedRedundancy -
+        numberReplicas.maintenanceReplicas(),
+        getMinMaintenanceStorageNum(block));
   }
 
   public short getExpectedRedundancyNum(BlockInfo block) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
index 3958c73..0390546 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
@@ -833,8 +833,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                          List<DatanodeStorageInfo> results,
                          boolean avoidStaleNodes) {
     // check if the node is (being) decommissioned
-    if (node.isDecommissionInProgress() || node.isDecommissioned()) {
-      logNodeIsNotChosen(node, "the node is (being) decommissioned ");
+    if (!node.isInService()) {
+      logNodeIsNotChosen(node, "the node isn't in service.");
       return false;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
index ca8d72a..8563cf3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
@@ -682,7 +682,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
       if (datanode == null) {
         continue;
       }
-      if (datanode.isDecommissioned() || datanode.isDecommissionInProgress()) {
+      if (!datanode.isInService()) {
         continue;
       }
       if (corrupt != null && corrupt.contains(datanode)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 6d163ec..f7da52a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -146,8 +146,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
 
   // Stores status of decommissioning.
   // If node is not decommissioning, do not use this object for anything.
-  public final DecommissioningStatus decommissioningStatus =
-      new DecommissioningStatus();
+  private final LeavingServiceStatus leavingServiceStatus =
+      new LeavingServiceStatus();
 
   private final Map<String, DatanodeStorageInfo> storageMap =
       new HashMap<>();
@@ -276,6 +276,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
     this.needKeyUpdate = needKeyUpdate;
   }
 
+  public LeavingServiceStatus getLeavingServiceStatus() {
+    return leavingServiceStatus;
+  }
+
   @VisibleForTesting
   public DatanodeStorageInfo getStorageInfo(String storageID) {
     synchronized (storageMap) {
@@ -729,51 +733,54 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return (this == obj) || super.equals(obj);
   }
 
-  /** Decommissioning status */
-  public class DecommissioningStatus {
+  /** Leaving service status. */
+  public class LeavingServiceStatus {
     private int underReplicatedBlocks;
-    private int decommissionOnlyReplicas;
+    private int outOfServiceOnlyReplicas;
     private int underReplicatedInOpenFiles;
     private long startTime;
     
     synchronized void set(int underRep,
         int onlyRep, int underConstruction) {
-      if (!isDecommissionInProgress()) {
+      if (!isDecommissionInProgress() && !isEnteringMaintenance()) {
         return;
       }
       underReplicatedBlocks = underRep;
-      decommissionOnlyReplicas = onlyRep;
+      outOfServiceOnlyReplicas = onlyRep;
       underReplicatedInOpenFiles = underConstruction;
     }
 
     /** @return the number of under-replicated blocks */
     public synchronized int getUnderReplicatedBlocks() {
-      if (!isDecommissionInProgress()) {
+      if (!isDecommissionInProgress() && !isEnteringMaintenance()) {
         return 0;
       }
       return underReplicatedBlocks;
     }
-    /** @return the number of decommission-only replicas */
-    public synchronized int getDecommissionOnlyReplicas() {
-      if (!isDecommissionInProgress()) {
+    /** @return the number of blocks with out-of-service-only replicas */
+    public synchronized int getOutOfServiceOnlyReplicas() {
+      if (!isDecommissionInProgress() && !isEnteringMaintenance()) {
         return 0;
       }
-      return decommissionOnlyReplicas;
+      return outOfServiceOnlyReplicas;
     }
     /** @return the number of under-replicated blocks in open files */
     public synchronized int getUnderReplicatedInOpenFiles() {
-      if (!isDecommissionInProgress()) {
+      if (!isDecommissionInProgress() && !isEnteringMaintenance()) {
         return 0;
       }
       return underReplicatedInOpenFiles;
     }
     /** Set start time */
     public synchronized void setStartTime(long time) {
+      if (!isDecommissionInProgress() && !isEnteringMaintenance()) {
+        return;
+      }
       startTime = time;
     }
     /** @return start time */
     public synchronized long getStartTime() {
-      if (!isDecommissionInProgress()) {
+      if (!isDecommissionInProgress() && !isEnteringMaintenance()) {
         return 0;
       }
       return startTime;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 2d6547f..1a47835 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -388,8 +388,8 @@ public class DatanodeManager {
   public void sortLocatedBlocks(final String targetHost,
       final List<LocatedBlock> locatedBlocks) {
     Comparator<DatanodeInfo> comparator = avoidStaleDataNodesForRead ?
-        new DFSUtil.DecomStaleComparator(staleInterval) :
-        DFSUtil.DECOM_COMPARATOR;
+        new DFSUtil.ServiceAndStaleComparator(staleInterval) :
+        new DFSUtil.ServiceComparator();
     // sort located block
     for (LocatedBlock lb : locatedBlocks) {
       if (lb.isStriped()) {
@@ -632,9 +632,20 @@ public class DatanodeManager {
    * @param nodeInfo datanode descriptor.
    */
   private void removeDatanode(DatanodeDescriptor nodeInfo) {
+    removeDatanode(nodeInfo, true);
+  }
+
+  /**
+   * Remove a datanode descriptor.
+   * @param nodeInfo datanode descriptor.
+   */
+  private void removeDatanode(DatanodeDescriptor nodeInfo,
+      boolean removeBlocksFromBlocksMap) {
     assert namesystem.hasWriteLock();
     heartbeatManager.removeDatanode(nodeInfo);
-    blockManager.removeBlocksAssociatedTo(nodeInfo);
+    if (removeBlocksFromBlocksMap) {
+      blockManager.removeBlocksAssociatedTo(nodeInfo);
+    }
     networktopology.remove(nodeInfo);
     decrementVersionCount(nodeInfo.getSoftwareVersion());
     blockManager.getBlockReportLeaseManager().unregister(nodeInfo);
@@ -655,7 +666,7 @@ public class DatanodeManager {
     try {
       final DatanodeDescriptor descriptor = getDatanode(node);
       if (descriptor != null) {
-        removeDatanode(descriptor);
+        removeDatanode(descriptor, true);
       } else {
         NameNode.stateChangeLog.warn("BLOCK* removeDatanode: "
                                      + node + " does not exist");
@@ -666,7 +677,8 @@ public class DatanodeManager {
   }
 
   /** Remove a dead datanode. */
-  void removeDeadDatanode(final DatanodeID nodeID) {
+  void removeDeadDatanode(final DatanodeID nodeID,
+      boolean removeBlocksFromBlockMap) {
     DatanodeDescriptor d;
     try {
       d = getDatanode(nodeID);
@@ -675,8 +687,9 @@ public class DatanodeManager {
     }
     if (d != null && isDatanodeDead(d)) {
       NameNode.stateChangeLog.info(
-          "BLOCK* removeDeadDatanode: lost heartbeat from " + d);
-      removeDatanode(d);
+          "BLOCK* removeDeadDatanode: lost heartbeat from " + d
+              + ", removeBlocksFromBlockMap " + removeBlocksFromBlockMap);
+      removeDatanode(d, removeBlocksFromBlockMap);
     }
   }
 
@@ -1112,10 +1125,16 @@ public class DatanodeManager {
   }
   
   /**
-   * 1. Added to hosts  --> no further work needed here.
-   * 2. Removed from hosts --> mark AdminState as decommissioned. 
-   * 3. Added to exclude --> start decommission.
-   * 4. Removed from exclude --> stop decommission.
+   * Reload datanode membership and the desired admin operations from
+   * host files. If a node isn't allowed, hostConfigManager.isIncluded returns
+   * false and the node can't be used.
+   * If a node is allowed and the desired admin operation is defined,
+   * it will transition to the desired admin state.
+   * If a node is allowed and upgrade domain is defined,
+   * the upgrade domain will be set on the node.
+   * To use maintenance mode or upgrade domain, set
+   * DFS_NAMENODE_HOSTS_PROVIDER_CLASSNAME_KEY to
+   * CombinedHostFileManager.class.
    */
   private void refreshDatanodes() {
     final Map<String, DatanodeDescriptor> copy;
@@ -1125,17 +1144,17 @@ public class DatanodeManager {
     for (DatanodeDescriptor node : copy.values()) {
       // Check if not include.
       if (!hostConfigManager.isIncluded(node)) {
-        node.setDisallowed(true); // case 2.
+        node.setDisallowed(true);
       } else {
         long maintenanceExpireTimeInMS =
             hostConfigManager.getMaintenanceExpirationTimeInMS(node);
         if (node.maintenanceNotExpired(maintenanceExpireTimeInMS)) {
           decomManager.startMaintenance(node, maintenanceExpireTimeInMS);
         } else if (hostConfigManager.isExcluded(node)) {
-          decomManager.startDecommission(node); // case 3.
+          decomManager.startDecommission(node);
         } else {
           decomManager.stopMaintenance(node);
-          decomManager.stopDecommission(node); // case 4.
+          decomManager.stopDecommission(node);
         }
       }
       node.setUpgradeDomain(hostConfigManager.getUpgradeDomain(node));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/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 87b36da..b1cfd78 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
@@ -201,7 +201,7 @@ public class DecommissionManager {
           LOG.info("Starting decommission of {} {} with {} blocks",
               node, storage, storage.numBlocks());
         }
-        node.decommissioningStatus.setStartTime(monotonicNow());
+        node.getLeavingServiceStatus().setStartTime(monotonicNow());
         pendingNodes.add(node);
       }
     } else {
@@ -222,7 +222,7 @@ public class DecommissionManager {
       // extra redundancy blocks will be detected and processed when
       // the dead node comes back and send in its full block report.
       if (node.isAlive()) {
-        blockManager.processExtraRedundancyBlocksOnReCommission(node);
+        blockManager.processExtraRedundancyBlocksOnInService(node);
       }
       // Remove from tracking in DecommissionManager
       pendingNodes.remove(node);
@@ -246,6 +246,16 @@ public class DecommissionManager {
     if (!node.isMaintenance()) {
       // Update DN stats maintained by HeartbeatManager
       hbManager.startMaintenance(node);
+      // hbManager.startMaintenance will set dead node to IN_MAINTENANCE.
+      if (node.isEnteringMaintenance()) {
+        for (DatanodeStorageInfo storage : node.getStorageInfos()) {
+          LOG.info("Starting maintenance of {} {} with {} blocks",
+              node, storage, storage.numBlocks());
+        }
+        node.getLeavingServiceStatus().setStartTime(monotonicNow());
+      }
+      // Track the node regardless whether it is ENTERING_MAINTENANCE or
+      // IN_MAINTENANCE to support maintenance expiration.
       pendingNodes.add(node);
     } else {
       LOG.trace("startMaintenance: Node {} in {}, nothing to do." +
@@ -264,8 +274,34 @@ public class DecommissionManager {
       // Update DN stats maintained by HeartbeatManager
       hbManager.stopMaintenance(node);
 
-      // TODO HDFS-9390 remove replicas from block maps
-      // or handle over replicated blocks.
+      // extra redundancy blocks will be detected and processed when
+      // the dead node comes back and send in its full block report.
+      if (!node.isAlive()) {
+        // The node became dead when it was in maintenance, at which point
+        // the replicas weren't removed from block maps.
+        // When the node leaves maintenance, the replicas should be removed
+        // from the block maps to trigger the necessary replication to
+        // maintain the safety property of "# of live replicas + maintenance
+        // replicas" >= the expected redundancy.
+        blockManager.removeBlocksAssociatedTo(node);
+      } else {
+        // Even though putting nodes in maintenance node doesn't cause live
+        // replicas to match expected replication factor, it is still possible
+        // to have over replicated when the node leaves maintenance node.
+        // First scenario:
+        // a. Node became dead when it is at AdminStates.NORMAL, thus
+        //    block is replicated so that 3 replicas exist on other nodes.
+        // b. Admins put the dead node into maintenance mode and then
+        //    have the node rejoin the cluster.
+        // c. Take the node out of maintenance mode.
+        // Second scenario:
+        // a. With replication factor 3, set one replica to maintenance node,
+        //    thus block has 1 maintenance replica and 2 live replicas.
+        // b. Change the replication factor to 2. The block will still have
+        //    1 maintenance replica and 2 live replicas.
+        // c. Take the node out of maintenance mode.
+        blockManager.processExtraRedundancyBlocksOnInService(node);
+      }
 
       // Remove from tracking in DecommissionManager
       pendingNodes.remove(node);
@@ -281,6 +317,11 @@ public class DecommissionManager {
     LOG.info("Decommissioning complete for node {}", dn);
   }
 
+  private void setInMaintenance(DatanodeDescriptor dn) {
+    dn.setInMaintenance();
+    LOG.info("Node {} has entered maintenance mode.", dn);
+  }
+
   /**
    * Checks whether a block is sufficiently replicated/stored for
    * decommissioning. For replicated blocks or striped blocks, full-strength
@@ -288,20 +329,21 @@ public class DecommissionManager {
    * @return true if sufficient, else false.
    */
   private boolean isSufficient(BlockInfo block, BlockCollection bc,
-      NumberReplicas numberReplicas) {
-    final int numExpected = blockManager.getExpectedRedundancyNum(block);
-    final int numLive = numberReplicas.liveReplicas();
-    if (numLive >= numExpected
-        && blockManager.isPlacementPolicySatisfied(block)) {
+      NumberReplicas numberReplicas, boolean isDecommission) {
+    if (blockManager.hasEnoughEffectiveReplicas(block, numberReplicas, 0)) {
       // Block has enough replica, skip
       LOG.trace("Block {} does not need replication.", block);
       return true;
     }
 
+    final int numExpected = blockManager.getExpectedLiveRedundancyNum(block,
+        numberReplicas);
+    final int numLive = numberReplicas.liveReplicas();
+
     // Block is under-replicated
-    LOG.trace("Block {} numExpected={}, numLive={}", block, numExpected, 
+    LOG.trace("Block {} numExpected={}, numLive={}", block, numExpected,
         numLive);
-    if (numExpected > numLive) {
+    if (isDecommission && numExpected > numLive) {
       if (bc.isUnderConstruction() && block.equals(bc.getLastBlock())) {
         // Can decom a UC block as long as there will still be minReplicas
         if (blockManager.hasMinStorage(block, numLive)) {
@@ -346,11 +388,16 @@ public class DecommissionManager {
         + ", corrupt replicas: " + num.corruptReplicas()
         + ", decommissioned replicas: " + num.decommissioned()
         + ", decommissioning replicas: " + num.decommissioning()
+        + ", maintenance replicas: " + num.maintenanceReplicas()
+        + ", live entering maintenance replicas: "
+        + num.liveEnteringMaintenanceReplicas()
         + ", excess replicas: " + num.excessReplicas()
         + ", Is Open File: " + bc.isUnderConstruction()
         + ", Datanodes having this block: " + nodeList + ", Current Datanode: "
         + srcNode + ", Is current datanode decommissioning: "
-        + srcNode.isDecommissionInProgress());
+        + srcNode.isDecommissionInProgress() +
+        ", Is current datanode entering maintenance: "
+        + srcNode.isEnteringMaintenance());
   }
 
   @VisibleForTesting
@@ -424,7 +471,7 @@ public class DecommissionManager {
       numBlocksChecked = 0;
       numBlocksCheckedPerLock = 0;
       numNodesChecked = 0;
-      // Check decom progress
+      // Check decommission or maintenance progress.
       namesystem.writeLock();
       try {
         processPendingNodes();
@@ -464,15 +511,14 @@ public class DecommissionManager {
         final DatanodeDescriptor dn = entry.getKey();
         AbstractList<BlockInfo> blocks = entry.getValue();
         boolean fullScan = false;
-        if (dn.isMaintenance()) {
-          // TODO HDFS-9390 make sure blocks are minimally replicated
-          // before transitioning the node to IN_MAINTENANCE state.
-
+        if (dn.isMaintenance() && dn.maintenanceExpired()) {
           // If maintenance expires, stop tracking it.
-          if (dn.maintenanceExpired()) {
-            stopMaintenance(dn);
-            toRemove.add(dn);
-          }
+          stopMaintenance(dn);
+          toRemove.add(dn);
+          continue;
+        }
+        if (dn.isInMaintenance()) {
+          // The dn is IN_MAINTENANCE and the maintenance hasn't expired yet.
           continue;
         }
         if (blocks == null) {
@@ -487,7 +533,7 @@ public class DecommissionManager {
         } else {
           // This is a known datanode, check if its # of insufficiently 
           // replicated blocks has dropped to zero and if it can be decommed
-          LOG.debug("Processing decommission-in-progress node {}", dn);
+          LOG.debug("Processing {} node {}", dn.getAdminState(), dn);
           pruneReliableBlocks(dn, blocks);
         }
         if (blocks.size() == 0) {
@@ -506,22 +552,31 @@ public class DecommissionManager {
           // If the full scan is clean AND the node liveness is okay, 
           // we can finally mark as decommissioned.
           final boolean isHealthy =
-              blockManager.isNodeHealthyForDecommission(dn);
+              blockManager.isNodeHealthyForDecommissionOrMaintenance(dn);
           if (blocks.size() == 0 && isHealthy) {
-            setDecommissioned(dn);
-            toRemove.add(dn);
+            if (dn.isDecommissionInProgress()) {
+              setDecommissioned(dn);
+              toRemove.add(dn);
+            } else if (dn.isEnteringMaintenance()) {
+              // IN_MAINTENANCE node remains in the outOfServiceNodeBlocks to
+              // to track maintenance expiration.
+              setInMaintenance(dn);
+            } else {
+              Preconditions.checkState(false,
+                  "A node is in an invalid state!");
+            }
             LOG.debug("Node {} is sufficiently replicated and healthy, "
-                + "marked as decommissioned.", dn);
+                + "marked as {}.", dn.getAdminState());
           } else {
             LOG.debug("Node {} {} healthy."
                 + " It needs to replicate {} more blocks."
-                + " Decommissioning is still in progress.",
-                dn, isHealthy? "is": "isn't", blocks.size());
+                + " {} is still in progress.", dn,
+                isHealthy? "is": "isn't", blocks.size(), dn.getAdminState());
           }
         } else {
           LOG.debug("Node {} still has {} blocks to replicate "
-                  + "before it is a candidate to finish decommissioning.",
-              dn, blocks.size());
+              + "before it is a candidate to finish {}.",
+              dn, blocks.size(), dn.getAdminState());
         }
         iterkey = dn;
       }
@@ -539,7 +594,7 @@ public class DecommissionManager {
      */
     private void pruneReliableBlocks(final DatanodeDescriptor datanode,
         AbstractList<BlockInfo> blocks) {
-      processBlocksForDecomInternal(datanode, blocks.iterator(), null, true);
+      processBlocksInternal(datanode, blocks.iterator(), null, true);
     }
 
     /**
@@ -554,7 +609,7 @@ public class DecommissionManager {
     private AbstractList<BlockInfo> handleInsufficientlyStored(
         final DatanodeDescriptor datanode) {
       AbstractList<BlockInfo> insufficient = new ChunkedArrayList<>();
-      processBlocksForDecomInternal(datanode, datanode.getBlockIterator(),
+      processBlocksInternal(datanode, datanode.getBlockIterator(),
           insufficient, false);
       return insufficient;
     }
@@ -573,14 +628,14 @@ public class DecommissionManager {
      * @param pruneReliableBlocks         whether to remove blocks reliable
      *                                    enough from the iterator
      */
-    private void processBlocksForDecomInternal(
+    private void processBlocksInternal(
         final DatanodeDescriptor datanode,
         final Iterator<BlockInfo> it,
         final List<BlockInfo> insufficientList,
         boolean pruneReliableBlocks) {
       boolean firstReplicationLog = true;
       int lowRedundancyBlocks = 0;
-      int decommissionOnlyReplicas = 0;
+      int outOfServiceOnlyReplicas = 0;
       int lowRedundancyInOpenFiles = 0;
       while (it.hasNext()) {
         if (insufficientList == null
@@ -626,21 +681,25 @@ public class DecommissionManager {
 
         // Schedule low redundancy blocks for reconstruction if not already
         // pending
-        if (blockManager.isNeededReconstruction(block, liveReplicas)) {
+        boolean isDecommission = datanode.isDecommissionInProgress();
+        boolean neededReconstruction = isDecommission ?
+            blockManager.isNeededReconstruction(block, num) :
+            blockManager.isNeededReconstructionForMaintenance(block, num);
+        if (neededReconstruction) {
           if (!blockManager.neededReconstruction.contains(block) &&
               blockManager.pendingReconstruction.getNumReplicas(block) == 0 &&
               blockManager.isPopulatingReplQueues()) {
             // Process these blocks only when active NN is out of safe mode.
             blockManager.neededReconstruction.add(block,
                 liveReplicas, num.readOnlyReplicas(),
-                num.decommissionedAndDecommissioning(),
+                num.outOfServiceReplicas(),
                 blockManager.getExpectedRedundancyNum(block));
           }
         }
 
         // Even if the block is without sufficient redundancy,
         // it doesn't block decommission if has sufficient redundancy
-        if (isSufficient(block, bc, num)) {
+        if (isSufficient(block, bc, num, isDecommission)) {
           if (pruneReliableBlocks) {
             it.remove();
           }
@@ -662,14 +721,13 @@ public class DecommissionManager {
         if (bc.isUnderConstruction()) {
           lowRedundancyInOpenFiles++;
         }
-        if ((liveReplicas == 0) && (num.decommissionedAndDecommissioning() > 0)) {
-          decommissionOnlyReplicas++;
+        if ((liveReplicas == 0) && (num.outOfServiceReplicas() > 0)) {
+          outOfServiceOnlyReplicas++;
         }
       }
 
-      datanode.decommissioningStatus.set(lowRedundancyBlocks,
-          decommissionOnlyReplicas,
-          lowRedundancyInOpenFiles);
+      datanode.getLeavingServiceStatus().set(lowRedundancyBlocks,
+          outOfServiceOnlyReplicas, lowRedundancyInOpenFiles);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java
index 082e949..0ae6f0f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java
@@ -130,12 +130,14 @@ class ErasureCodingWork extends BlockReconstructionWork {
       // we only need to replicate one internal block to a new rack
       int sourceIndex = chooseSource4SimpleReplication();
       createReplicationWork(sourceIndex, targets[0]);
-    } else if (numberReplicas.decommissioning() > 0 && hasAllInternalBlocks()) {
-      List<Integer> decommissioningSources = findDecommissioningSources();
+    } else if ((numberReplicas.decommissioning() > 0 ||
+        numberReplicas.liveEnteringMaintenanceReplicas() > 0) &&
+        hasAllInternalBlocks()) {
+      List<Integer> leavingServiceSources = findLeavingServiceSources();
       // decommissioningSources.size() should be >= targets.length
-      final int num = Math.min(decommissioningSources.size(), targets.length);
+      final int num = Math.min(leavingServiceSources.size(), targets.length);
       for (int i = 0; i < num; i++) {
-        createReplicationWork(decommissioningSources.get(i), targets[i]);
+        createReplicationWork(leavingServiceSources.get(i), targets[i]);
       }
     } else {
       targets[0].getDatanodeDescriptor().addBlockToBeErasureCoded(
@@ -162,10 +164,12 @@ class ErasureCodingWork extends BlockReconstructionWork {
     }
   }
 
-  private List<Integer> findDecommissioningSources() {
+  private List<Integer> findLeavingServiceSources() {
     List<Integer> srcIndices = new ArrayList<>();
     for (int i = 0; i < getSrcNodes().length; i++) {
-      if (getSrcNodes()[i].isDecommissionInProgress()) {
+      if (getSrcNodes()[i].isDecommissionInProgress() ||
+          (getSrcNodes()[i].isEnteringMaintenance() &&
+          getSrcNodes()[i].isAlive())) {
         srcIndices.add(i);
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
index d728ee2..a72ad64 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
@@ -25,10 +25,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
-import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 import org.apache.hadoop.util.Daemon;
@@ -269,13 +266,19 @@ class HeartbeatManager implements DatanodeStatistics {
     if (!node.isAlive()) {
       LOG.info("Dead node {} is put in maintenance state immediately.", node);
       node.setInMaintenance();
-    } else if (node.isDecommissioned()) {
-      LOG.info("Decommissioned node " + node + " is put in maintenance state"
-          + " immediately.");
-      node.setInMaintenance();
     } else {
       stats.subtract(node);
-      node.startMaintenance();
+      if (node.isDecommissioned()) {
+        LOG.info("Decommissioned node " + node + " is put in maintenance state"
+            + " immediately.");
+        node.setInMaintenance();
+      } else if (blockManager.getMinReplicationToBeInMaintenance() == 0) {
+        LOG.info("MinReplicationToBeInMaintenance is set to zero. " + node +
+            " is put in maintenance state" + " immediately.");
+        node.setInMaintenance();
+      } else {
+        node.startMaintenance();
+      }
       stats.add(node);
     }
   }
@@ -352,7 +355,7 @@ class HeartbeatManager implements DatanodeStatistics {
     boolean allAlive = false;
     while (!allAlive) {
       // locate the first dead node.
-      DatanodeID dead = null;
+      DatanodeDescriptor dead = null;
 
       // locate the first failed storage that isn't on a dead node.
       DatanodeStorageInfo failedStorage = null;
@@ -401,7 +404,7 @@ class HeartbeatManager implements DatanodeStatistics {
         // acquire the fsnamesystem lock, and then remove the dead node.
         namesystem.writeLock();
         try {
-          dm.removeDeadDatanode(dead);
+          dm.removeDeadDatanode(dead, !dead.isMaintenance());
         } finally {
           namesystem.writeUnlock();
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java
index de8cf4e..3a26f4a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java
@@ -155,7 +155,7 @@ class LowRedundancyBlocks implements Iterable<BlockInfo> {
   private int getPriority(BlockInfo block,
                           int curReplicas,
                           int readOnlyReplicas,
-                          int decommissionedReplicas,
+                          int outOfServiceReplicas,
                           int expectedReplicas) {
     assert curReplicas >= 0 : "Negative replicas!";
     if (curReplicas >= expectedReplicas) {
@@ -164,20 +164,20 @@ class LowRedundancyBlocks implements Iterable<BlockInfo> {
     }
     if (block.isStriped()) {
       BlockInfoStriped sblk = (BlockInfoStriped) block;
-      return getPriorityStriped(curReplicas, decommissionedReplicas,
+      return getPriorityStriped(curReplicas, outOfServiceReplicas,
           sblk.getRealDataBlockNum(), sblk.getParityBlockNum());
     } else {
       return getPriorityContiguous(curReplicas, readOnlyReplicas,
-          decommissionedReplicas, expectedReplicas);
+          outOfServiceReplicas, expectedReplicas);
     }
   }
 
   private int getPriorityContiguous(int curReplicas, int readOnlyReplicas,
-      int decommissionedReplicas, int expectedReplicas) {
+      int outOfServiceReplicas, int expectedReplicas) {
     if (curReplicas == 0) {
       // If there are zero non-decommissioned replicas but there are
-      // some decommissioned replicas, then assign them highest priority
-      if (decommissionedReplicas > 0) {
+      // some out of service replicas, then assign them highest priority
+      if (outOfServiceReplicas > 0) {
         return QUEUE_HIGHEST_PRIORITY;
       }
       if (readOnlyReplicas > 0) {
@@ -201,11 +201,11 @@ class LowRedundancyBlocks implements Iterable<BlockInfo> {
     }
   }
 
-  private int getPriorityStriped(int curReplicas, int decommissionedReplicas,
+  private int getPriorityStriped(int curReplicas, int outOfServiceReplicas,
       short dataBlkNum, short parityBlkNum) {
     if (curReplicas < dataBlkNum) {
       // There are some replicas on decommissioned nodes so it's not corrupted
-      if (curReplicas + decommissionedReplicas >= dataBlkNum) {
+      if (curReplicas + outOfServiceReplicas >= dataBlkNum) {
         return QUEUE_HIGHEST_PRIORITY;
       }
       return QUEUE_WITH_CORRUPT_BLOCKS;
@@ -227,18 +227,15 @@ class LowRedundancyBlocks implements Iterable<BlockInfo> {
    *
    * @param block a low redundancy block
    * @param curReplicas current number of replicas of the block
-   * @param decomissionedReplicas the number of decommissioned replicas
+   * @param outOfServiceReplicas the number of out-of-service replicas
    * @param expectedReplicas expected number of replicas of the block
    * @return true if the block was added to a queue.
    */
   synchronized boolean add(BlockInfo block,
-                           int curReplicas,
-                           int readOnlyReplicas,
-                           int decomissionedReplicas,
-                           int expectedReplicas) {
-    assert curReplicas >= 0 : "Negative replicas!";
+      int curReplicas, int readOnlyReplicas,
+      int outOfServiceReplicas, int expectedReplicas) {
     final int priLevel = getPriority(block, curReplicas, readOnlyReplicas,
-        decomissionedReplicas, expectedReplicas);
+        outOfServiceReplicas, expectedReplicas);
     if(priorityQueues.get(priLevel).add(block)) {
       if (priLevel == QUEUE_WITH_CORRUPT_BLOCKS &&
           expectedReplicas == 1) {
@@ -257,12 +254,10 @@ class LowRedundancyBlocks implements Iterable<BlockInfo> {
 
   /** Remove a block from a low redundancy queue. */
   synchronized boolean remove(BlockInfo block,
-                              int oldReplicas,
-                              int oldReadOnlyReplicas,
-                              int decommissionedReplicas,
-                              int oldExpectedReplicas) {
+      int oldReplicas, int oldReadOnlyReplicas,
+      int outOfServiceReplicas, int oldExpectedReplicas) {
     final int priLevel = getPriority(block, oldReplicas, oldReadOnlyReplicas,
-        decommissionedReplicas, oldExpectedReplicas);
+        outOfServiceReplicas, oldExpectedReplicas);
     boolean removedBlock = remove(block, priLevel);
     if (priLevel == QUEUE_WITH_CORRUPT_BLOCKS &&
         oldExpectedReplicas == 1 &&
@@ -325,22 +320,22 @@ class LowRedundancyBlocks implements Iterable<BlockInfo> {
    * method call.
    * @param block a low redundancy block
    * @param curReplicas current number of replicas of the block
-   * @param decommissionedReplicas  the number of decommissioned replicas
+   * @param outOfServiceReplicas  the number of out-of-service replicas
    * @param curExpectedReplicas expected number of replicas of the 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(BlockInfo block, int curReplicas,
-                           int readOnlyReplicas, int decommissionedReplicas,
-                           int curExpectedReplicas,
-                           int curReplicasDelta, int expectedReplicasDelta) {
+      int readOnlyReplicas, int outOfServiceReplicas,
+      int curExpectedReplicas,
+      int curReplicasDelta, int expectedReplicasDelta) {
     int oldReplicas = curReplicas-curReplicasDelta;
     int oldExpectedReplicas = curExpectedReplicas-expectedReplicasDelta;
     int curPri = getPriority(block, curReplicas, readOnlyReplicas,
-        decommissionedReplicas, curExpectedReplicas);
+        outOfServiceReplicas, curExpectedReplicas);
     int oldPri = getPriority(block, oldReplicas, readOnlyReplicas,
-        decommissionedReplicas, oldExpectedReplicas);
+        outOfServiceReplicas, oldExpectedReplicas);
     if(NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("LowRedundancyBlocks.update " +
         block +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/NumberReplicas.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/NumberReplicas.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/NumberReplicas.java
index 0198bcc..be984f9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/NumberReplicas.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/NumberReplicas.java
@@ -24,9 +24,11 @@ import static org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas.Store
 import static org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas.StoredReplicaState.DECOMMISSIONING;
 import static org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas.StoredReplicaState.EXCESS;
 import static org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas.StoredReplicaState.LIVE;
+import static org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas.StoredReplicaState.MAINTENANCE_FOR_READ;
+import static org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas.StoredReplicaState.MAINTENANCE_NOT_FOR_READ;
+import static org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas.StoredReplicaState.READONLY;
 import static org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas.StoredReplicaState.REDUNDANT;
 import static org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas.StoredReplicaState.STALESTORAGE;
-import static org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas.StoredReplicaState.READONLY;
 
 /**
  * A immutable object that stores the number of live replicas and
@@ -41,6 +43,14 @@ public class NumberReplicas extends EnumCounters<NumberReplicas.StoredReplicaSta
     READONLY,
     DECOMMISSIONING,
     DECOMMISSIONED,
+    // We need live ENTERING_MAINTENANCE nodes to continue
+    // to serve read request while it is being transitioned to live
+    // IN_MAINTENANCE if these are the only replicas left.
+    // MAINTENANCE_NOT_FOR_READ == maintenanceReplicas -
+    // Live ENTERING_MAINTENANCE.
+    MAINTENANCE_NOT_FOR_READ,
+    // Live ENTERING_MAINTENANCE nodes to serve read requests.
+    MAINTENANCE_FOR_READ,
     CORRUPT,
     // excess replicas already tracked by blockmanager's excess map
     EXCESS,
@@ -106,4 +116,20 @@ public class NumberReplicas extends EnumCounters<NumberReplicas.StoredReplicaSta
   public int redundantInternalBlocks() {
     return (int) get(REDUNDANT);
   }
-} 
+
+  public int maintenanceNotForReadReplicas() {
+    return (int) get(MAINTENANCE_NOT_FOR_READ);
+  }
+
+  public int maintenanceReplicas() {
+    return (int) (get(MAINTENANCE_NOT_FOR_READ) + get(MAINTENANCE_FOR_READ));
+  }
+
+  public int outOfServiceReplicas() {
+    return maintenanceReplicas() + decommissionedAndDecommissioning();
+  }
+
+  public int liveEnteringMaintenanceReplicas() {
+    return (int)get(MAINTENANCE_FOR_READ);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/StorageTypeStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/StorageTypeStats.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/StorageTypeStats.java
index 45dcc8d..005e6d5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/StorageTypeStats.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/StorageTypeStats.java
@@ -81,7 +81,7 @@ public class StorageTypeStats {
       final DatanodeDescriptor node) {
     capacityUsed += info.getDfsUsed();
     blockPoolUsed += info.getBlockPoolUsed();
-    if (!(node.isDecommissionInProgress() || node.isDecommissioned())) {
+    if (node.isInService()) {
       capacityTotal += info.getCapacity();
       capacityRemaining += info.getRemaining();
     } else {
@@ -90,7 +90,7 @@ public class StorageTypeStats {
   }
 
   void addNode(final DatanodeDescriptor node) {
-    if (!(node.isDecommissionInProgress() || node.isDecommissioned())) {
+    if (node.isInService()) {
       nodesInService++;
     }
   }
@@ -99,7 +99,7 @@ public class StorageTypeStats {
       final DatanodeDescriptor node) {
     capacityUsed -= info.getDfsUsed();
     blockPoolUsed -= info.getBlockPoolUsed();
-    if (!(node.isDecommissionInProgress() || node.isDecommissioned())) {
+    if (node.isInService()) {
       capacityTotal -= info.getCapacity();
       capacityRemaining -= info.getRemaining();
     } else {
@@ -108,7 +108,7 @@ public class StorageTypeStats {
   }
 
   void subtractNode(final DatanodeDescriptor node) {
-    if (!(node.isDecommissionInProgress() || node.isDecommissioned())) {
+    if (node.isInService()) {
       nodesInService--;
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/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 563682f..eb870f8 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
@@ -5462,11 +5462,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           .<String, Object> builder()
           .put("xferaddr", node.getXferAddr())
           .put("underReplicatedBlocks",
-              node.decommissioningStatus.getUnderReplicatedBlocks())
+          node.getLeavingServiceStatus().getUnderReplicatedBlocks())
+           // TODO use another property name for outOfServiceOnlyReplicas.
           .put("decommissionOnlyReplicas",
-              node.decommissioningStatus.getDecommissionOnlyReplicas())
+          node.getLeavingServiceStatus().getOutOfServiceOnlyReplicas())
           .put("underReplicateInOpenFiles",
-              node.decommissioningStatus.getUnderReplicatedInOpenFiles())
+          node.getLeavingServiceStatus().getUnderReplicatedInOpenFiles())
           .build();
       info.put(node.getHostName() + ":" + node.getXferPort(), innerinfo);
     }
@@ -5528,7 +5529,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     blockManager.getDatanodeManager().fetchDatanodes(live, null, true);
     for (Iterator<DatanodeDescriptor> it = live.iterator(); it.hasNext();) {
       DatanodeDescriptor node = it.next();
-      if (node.isDecommissionInProgress() || node.isDecommissioned()) {
+      if (!node.isInService()) {
         it.remove();
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/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 84b51f6..483663e 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
@@ -548,6 +548,13 @@
 </property>
 
 <property>
+  <name>dfs.namenode.maintenance.replication.min</name>
+  <value>1</value>
+  <description>Minimal live block replication in existence of maintenance mode.
+  </description>
+</property>
+
+<property>
   <name>dfs.namenode.safemode.replication.min</name>
   <value></value>
   <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AdminStatesBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AdminStatesBaseTest.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AdminStatesBaseTest.java
index 0698628..534c5e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AdminStatesBaseTest.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AdminStatesBaseTest.java
@@ -102,6 +102,7 @@ public class AdminStatesBaseTest {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 1);
 
     hostsFileWriter.initialize(conf, "temp/admin");
+
   }
 
   @After
@@ -110,17 +111,22 @@ public class AdminStatesBaseTest {
     shutdownCluster();
   }
 
-  protected void writeFile(FileSystem fileSys, Path name, int repl)
+  static public FSDataOutputStream writeIncompleteFile(FileSystem fileSys,
+      Path name, short repl, short numOfBlocks) throws IOException {
+    return writeFile(fileSys, name, repl, numOfBlocks, false);
+  }
+
+  static protected void writeFile(FileSystem fileSys, Path name, int repl)
       throws IOException {
     writeFile(fileSys, name, repl, 2);
   }
 
-  protected void writeFile(FileSystem fileSys, Path name, int repl,
+  static protected void writeFile(FileSystem fileSys, Path name, int repl,
       int numOfBlocks) throws IOException {
     writeFile(fileSys, name, repl, numOfBlocks, true);
   }
 
-  protected FSDataOutputStream writeFile(FileSystem fileSys, Path name,
+  static protected FSDataOutputStream writeFile(FileSystem fileSys, Path name,
       int repl, int numOfBlocks, boolean completeFile)
     throws IOException {
     // create and write a file that contains two blocks of data
@@ -136,6 +142,7 @@ public class AdminStatesBaseTest {
       stm.close();
       return null;
     } else {
+      stm.flush();
       // Do not close stream, return it
       // so that it is not garbage collected
       return stm;
@@ -353,7 +360,7 @@ public class AdminStatesBaseTest {
 
   protected void shutdownCluster() {
     if (cluster != null) {
-      cluster.shutdown();
+      cluster.shutdown(true);
     }
   }
 
@@ -362,12 +369,13 @@ public class AdminStatesBaseTest {
         refreshNodes(conf);
   }
 
-  protected DatanodeDescriptor getDatanodeDesriptor(
+  static private DatanodeDescriptor getDatanodeDesriptor(
       final FSNamesystem ns, final String datanodeUuid) {
     return ns.getBlockManager().getDatanodeManager().getDatanode(datanodeUuid);
   }
 
-  protected void cleanupFile(FileSystem fileSys, Path name) throws IOException {
+  static public void cleanupFile(FileSystem fileSys, Path name)
+      throws IOException {
     assertTrue(fileSys.exists(name));
     fileSys.delete(name, true);
     assertTrue(!fileSys.exists(name));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/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 ddb8237..6ca1e79 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
@@ -484,7 +484,7 @@ public class TestDecommission extends AdminStatesBaseTest {
       shutdownCluster();
     }
   }
-  
+
   /**
    * Tests cluster storage statistics during decommissioning for non
    * federated cluster


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


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

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

This reverts commit fdce515091f0a61ffd6c9ae464a68447dedf1124.


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

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

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


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


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


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

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


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

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

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


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


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


[45/50] [abbrv] hadoop git commit: HADOOP-13560. S3ABlockOutputStream to support huge (many GB) file writes. Contributed by Steve Loughran

Posted by sj...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
new file mode 100644
index 0000000..a60d084
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
@@ -0,0 +1,412 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.scale;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.amazonaws.event.ProgressEvent;
+import com.amazonaws.event.ProgressEventType;
+import com.amazonaws.event.ProgressListener;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runners.MethodSorters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageStatistics;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.util.Progressable;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
+
+/**
+ * Scale test which creates a huge file.
+ *
+ * <b>Important:</b> the order in which these tests execute is fixed to
+ * alphabetical order. Test cases are numbered {@code test_123_} to impose
+ * an ordering based on the numbers.
+ *
+ * Having this ordering allows the tests to assume that the huge file
+ * exists. Even so: they should all have a {@link #assumeHugeFileExists()}
+ * check at the start, in case an individual test is executed.
+ */
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbstractSTestS3AHugeFiles.class);
+  public static final int DEFAULT_UPLOAD_BLOCKSIZE = 64 * _1KB;
+  public static final String DEFAULT_PARTITION_SIZE = "8M";
+  private Path scaleTestDir;
+  private Path hugefile;
+  private Path hugefileRenamed;
+
+  private int uploadBlockSize = DEFAULT_UPLOAD_BLOCKSIZE;
+  private int partitionSize;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+
+    final Path testPath = getTestPath();
+    scaleTestDir = new Path(testPath, "scale");
+    hugefile = new Path(scaleTestDir, "hugefile");
+    hugefileRenamed = new Path(scaleTestDir, "hugefileRenamed");
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    // do nothing. Specifically: do not delete the test dir
+  }
+
+  /**
+   * Note that this can get called before test setup.
+   * @return the configuration to use.
+   */
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    partitionSize = (int)getTestPropertyBytes(conf,
+        KEY_HUGE_PARTITION_SIZE,
+        DEFAULT_PARTITION_SIZE);
+    assertTrue("Partition size too small: " + partitionSize,
+        partitionSize > MULTIPART_MIN_SIZE);
+    conf.setLong(SOCKET_SEND_BUFFER, _1MB);
+    conf.setLong(SOCKET_RECV_BUFFER, _1MB);
+    conf.setLong(MIN_MULTIPART_THRESHOLD, partitionSize);
+    conf.setInt(MULTIPART_SIZE, partitionSize);
+    conf.set(USER_AGENT_PREFIX, "STestS3AHugeFileCreate");
+    conf.setBoolean(FAST_UPLOAD, true);
+    conf.set(FAST_UPLOAD_BUFFER, getBlockOutputBufferName());
+    return conf;
+  }
+
+  /**
+   * The name of the buffering mechanism to use.
+   * @return a buffering mechanism
+   */
+  protected abstract String getBlockOutputBufferName();
+
+  @Test
+  public void test_010_CreateHugeFile() throws IOException {
+    assertFalse("Please run this test sequentially to avoid timeouts" +
+            " and bandwidth problems", isParallelExecution());
+    long filesize = getTestPropertyBytes(getConf(), KEY_HUGE_FILESIZE,
+        DEFAULT_HUGE_FILESIZE);
+    long filesizeMB = filesize / _1MB;
+
+    // clean up from any previous attempts
+    deleteHugeFile();
+
+    describe("Creating file %s of size %d MB" +
+            " with partition size %d buffered by %s",
+        hugefile, filesizeMB, partitionSize, getBlockOutputBufferName());
+
+    // now do a check of available upload time, with a pessimistic bandwidth
+    // (that of remote upload tests). If the test times out then not only is
+    // the test outcome lost, as the follow-on tests continue, they will
+    // overlap with the ongoing upload test, for much confusion.
+    int timeout = getTestTimeoutSeconds();
+    // assume 1 MB/s upload bandwidth
+    int bandwidth = _1MB;
+    long uploadTime = filesize / bandwidth;
+    assertTrue(String.format("Timeout set in %s seconds is too low;" +
+            " estimating upload time of %d seconds at 1 MB/s." +
+            " Rerun tests with -D%s=%d",
+            timeout, uploadTime, KEY_TEST_TIMEOUT, uploadTime * 2),
+        uploadTime < timeout);
+    assertEquals("File size set in " + KEY_HUGE_FILESIZE + " = " + filesize
+            + " is not a multiple of " + uploadBlockSize,
+        0, filesize % uploadBlockSize);
+
+    byte[] data = new byte[uploadBlockSize];
+    for (int i = 0; i < uploadBlockSize; i++) {
+      data[i] = (byte) (i % 256);
+    }
+
+    long blocks = filesize / uploadBlockSize;
+    long blocksPerMB = _1MB / uploadBlockSize;
+
+    // perform the upload.
+    // there's lots of logging here, so that a tail -f on the output log
+    // can give a view of what is happening.
+    StorageStatistics storageStatistics = fs.getStorageStatistics();
+    String putRequests = Statistic.OBJECT_PUT_REQUESTS.getSymbol();
+    String putBytes = Statistic.OBJECT_PUT_BYTES.getSymbol();
+    Statistic putRequestsActive = Statistic.OBJECT_PUT_REQUESTS_ACTIVE;
+    Statistic putBytesPending = Statistic.OBJECT_PUT_BYTES_PENDING;
+
+    ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+
+    long blocksPer10MB = blocksPerMB * 10;
+    ProgressCallback progress = new ProgressCallback(timer);
+    try (FSDataOutputStream out = fs.create(hugefile,
+        true,
+        uploadBlockSize,
+        progress)) {
+
+      for (long block = 1; block <= blocks; block++) {
+        out.write(data);
+        long written = block * uploadBlockSize;
+        // every 10 MB and on file upload @ 100%, print some stats
+        if (block % blocksPer10MB == 0 || written == filesize) {
+          long percentage = written * 100 / filesize;
+          double elapsedTime = timer.elapsedTime() / 1.0e9;
+          double writtenMB = 1.0 * written / _1MB;
+          LOG.info(String.format("[%02d%%] Buffered %.2f MB out of %d MB;" +
+                  " PUT %d bytes (%d pending) in %d operations (%d active);" +
+                  " elapsedTime=%.2fs; write to buffer bandwidth=%.2f MB/s",
+              percentage,
+              writtenMB,
+              filesizeMB,
+              storageStatistics.getLong(putBytes),
+              gaugeValue(putBytesPending),
+              storageStatistics.getLong(putRequests),
+              gaugeValue(putRequestsActive),
+              elapsedTime,
+              writtenMB / elapsedTime));
+        }
+      }
+      // now close the file
+      LOG.info("Closing file and completing write operation");
+      ContractTestUtils.NanoTimer closeTimer
+          = new ContractTestUtils.NanoTimer();
+      out.close();
+      closeTimer.end("time to close() output stream");
+    }
+
+    timer.end("time to write %d MB in blocks of %d",
+        filesizeMB, uploadBlockSize);
+    logFSState();
+    bandwidth(timer, filesize);
+    long putRequestCount = storageStatistics.getLong(putRequests);
+    Long putByteCount = storageStatistics.getLong(putBytes);
+    LOG.info("PUT {} bytes in {} operations; {} MB/operation",
+        putByteCount, putRequestCount,
+        putByteCount / (putRequestCount * _1MB));
+    LOG.info("Time per PUT {} nS",
+        toHuman(timer.nanosPerOperation(putRequestCount)));
+    assertEquals("active put requests in \n" + fs,
+        0, gaugeValue(putRequestsActive));
+    ContractTestUtils.assertPathExists(fs, "Huge file", hugefile);
+    S3AFileStatus status = fs.getFileStatus(hugefile);
+    ContractTestUtils.assertIsFile(hugefile, status);
+    assertEquals("File size in " + status, filesize, status.getLen());
+    progress.verifyNoFailures("Put file " + hugefile + " of size " + filesize);
+  }
+
+  /**
+   * Progress callback from AWS. Likely to come in on a different thread.
+   */
+  private final class ProgressCallback implements Progressable,
+      ProgressListener {
+    private AtomicLong bytesTransferred = new AtomicLong(0);
+    private AtomicInteger failures = new AtomicInteger(0);
+    private final ContractTestUtils.NanoTimer timer;
+
+    private ProgressCallback(NanoTimer timer) {
+      this.timer = timer;
+    }
+
+    @Override
+    public void progress() {
+    }
+
+    @Override
+    public void progressChanged(ProgressEvent progressEvent) {
+      ProgressEventType eventType = progressEvent.getEventType();
+      if (eventType.isByteCountEvent()) {
+        bytesTransferred.addAndGet(progressEvent.getBytesTransferred());
+      }
+      switch (eventType) {
+      case TRANSFER_PART_FAILED_EVENT:
+        // failure
+        failures.incrementAndGet();
+        LOG.warn("Transfer failure");
+        break;
+      case TRANSFER_PART_COMPLETED_EVENT:
+        // completion
+        long elapsedTime = timer.elapsedTime();
+        double elapsedTimeS = elapsedTime / 1.0e9;
+        long written = bytesTransferred.get();
+        long writtenMB = written / _1MB;
+        LOG.info(String.format(
+            "Event %s; total uploaded=%d MB in %.1fs;" +
+                " effective upload bandwidth = %.2f MB/s",
+            progressEvent,
+            writtenMB, elapsedTimeS, writtenMB / elapsedTimeS));
+        break;
+      default:
+        if (eventType.isByteCountEvent()) {
+          LOG.debug("Event {}", progressEvent);
+        } else {
+          LOG.info("Event {}", progressEvent);
+        }
+        break;
+      }
+    }
+
+    @Override
+    public String toString() {
+      String sb = "ProgressCallback{"
+          + "bytesTransferred=" + bytesTransferred +
+          ", failures=" + failures +
+          '}';
+      return sb;
+    }
+
+    private void verifyNoFailures(String operation) {
+      assertEquals("Failures in " + operation +": " + this, 0, failures.get());
+    }
+  }
+
+  void assumeHugeFileExists() throws IOException {
+    ContractTestUtils.assertPathExists(fs, "huge file not created", hugefile);
+    ContractTestUtils.assertIsFile(fs, hugefile);
+  }
+
+  private void logFSState() {
+    LOG.info("File System state after operation:\n{}", fs);
+  }
+
+  @Test
+  public void test_040_PositionedReadHugeFile() throws Throwable {
+    assumeHugeFileExists();
+    final String encryption = getConf().getTrimmed(
+        SERVER_SIDE_ENCRYPTION_ALGORITHM);
+    boolean encrypted = encryption != null;
+    if (encrypted) {
+      LOG.info("File is encrypted with algorithm {}", encryption);
+    }
+    String filetype = encrypted ? "encrypted file" : "file";
+    describe("Positioned reads of %s %s", filetype, hugefile);
+    S3AFileStatus status = fs.getFileStatus(hugefile);
+    long filesize = status.getLen();
+    int ops = 0;
+    final int bufferSize = 8192;
+    byte[] buffer = new byte[bufferSize];
+    long eof = filesize - 1;
+
+    ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+    ContractTestUtils.NanoTimer readAtByte0, readAtByte0Again, readAtEOF;
+    try (FSDataInputStream in = fs.open(hugefile, uploadBlockSize)) {
+      readAtByte0 = new ContractTestUtils.NanoTimer();
+      in.readFully(0, buffer);
+      readAtByte0.end("time to read data at start of file");
+      ops++;
+
+      readAtEOF = new ContractTestUtils.NanoTimer();
+      in.readFully(eof - bufferSize, buffer);
+      readAtEOF.end("time to read data at end of file");
+      ops++;
+
+      readAtByte0Again = new ContractTestUtils.NanoTimer();
+      in.readFully(0, buffer);
+      readAtByte0Again.end("time to read data at start of file again");
+      ops++;
+      LOG.info("Final stream state: {}", in);
+    }
+    long mb = Math.max(filesize / _1MB, 1);
+
+    logFSState();
+    timer.end("time to performed positioned reads of %s of %d MB ",
+        filetype, mb);
+    LOG.info("Time per positioned read = {} nS",
+        toHuman(timer.nanosPerOperation(ops)));
+  }
+
+  @Test
+  public void test_050_readHugeFile() throws Throwable {
+    assumeHugeFileExists();
+    describe("Reading %s", hugefile);
+    S3AFileStatus status = fs.getFileStatus(hugefile);
+    long filesize = status.getLen();
+    long blocks = filesize / uploadBlockSize;
+    byte[] data = new byte[uploadBlockSize];
+
+    ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+    try (FSDataInputStream in = fs.open(hugefile, uploadBlockSize)) {
+      for (long block = 0; block < blocks; block++) {
+        in.readFully(data);
+      }
+      LOG.info("Final stream state: {}", in);
+    }
+
+    long mb = Math.max(filesize / _1MB, 1);
+    timer.end("time to read file of %d MB ", mb);
+    LOG.info("Time per MB to read = {} nS",
+        toHuman(timer.nanosPerOperation(mb)));
+    bandwidth(timer, filesize);
+    logFSState();
+  }
+
+  @Test
+  public void test_100_renameHugeFile() throws Throwable {
+    assumeHugeFileExists();
+    describe("renaming %s to %s", hugefile, hugefileRenamed);
+    S3AFileStatus status = fs.getFileStatus(hugefile);
+    long filesize = status.getLen();
+    fs.delete(hugefileRenamed, false);
+    ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+    fs.rename(hugefile, hugefileRenamed);
+    long mb = Math.max(filesize / _1MB, 1);
+    timer.end("time to rename file of %d MB", mb);
+    LOG.info("Time per MB to rename = {} nS",
+        toHuman(timer.nanosPerOperation(mb)));
+    bandwidth(timer, filesize);
+    logFSState();
+    S3AFileStatus destFileStatus = fs.getFileStatus(hugefileRenamed);
+    assertEquals(filesize, destFileStatus.getLen());
+
+    // rename back
+    ContractTestUtils.NanoTimer timer2 = new ContractTestUtils.NanoTimer();
+    fs.rename(hugefileRenamed, hugefile);
+    timer2.end("Renaming back");
+    LOG.info("Time per MB to rename = {} nS",
+        toHuman(timer2.nanosPerOperation(mb)));
+    bandwidth(timer2, filesize);
+  }
+
+  @Test
+  public void test_999_DeleteHugeFiles() throws IOException {
+    deleteHugeFile();
+    ContractTestUtils.NanoTimer timer2 = new ContractTestUtils.NanoTimer();
+
+    fs.delete(hugefileRenamed, false);
+    timer2.end("time to delete %s", hugefileRenamed);
+    ContractTestUtils.rm(fs, getTestPath(), true, true);
+  }
+
+  protected void deleteHugeFile() throws IOException {
+    describe("Deleting %s", hugefile);
+    NanoTimer timer = new NanoTimer();
+    fs.delete(hugefile, false);
+    timer.end("time to delete %s", hugefile);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java
index 208c491..4e1a734 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java
@@ -116,20 +116,9 @@ public class ITestS3ADeleteManyFiles extends S3AScaleTestBase {
 
   @Test
   public void testOpenCreate() throws IOException {
-    Path dir = new Path("/tests3a");
-    ContractTestUtils.createAndVerifyFile(fs, dir, 1024);
-    ContractTestUtils.createAndVerifyFile(fs, dir, 5 * 1024 * 1024);
-    ContractTestUtils.createAndVerifyFile(fs, dir, 20 * 1024 * 1024);
-
-
-    /*
-    Enable to test the multipart upload
-    try {
-      ContractTestUtils.createAndVerifyFile(fs, dir,
-          (long)6 * 1024 * 1024 * 1024);
-    } catch (IOException e) {
-      fail(e.getMessage());
-    }
-    */
+    final Path scaleTestDir = getTestPath();
+    final Path srcDir = new Path(scaleTestDir, "opencreate");
+    ContractTestUtils.createAndVerifyFile(fs, srcDir, 1024);
+    ContractTestUtils.createAndVerifyFile(fs, srcDir, 50 * 1024);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesArrayBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesArrayBlocks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesArrayBlocks.java
new file mode 100644
index 0000000..d6f15c8
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesArrayBlocks.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.scale;
+
+import org.apache.hadoop.fs.s3a.Constants;
+
+/**
+ * Use {@link Constants#FAST_UPLOAD_BUFFER_ARRAY} for buffering.
+ */
+public class ITestS3AHugeFilesArrayBlocks extends AbstractSTestS3AHugeFiles {
+
+  protected String getBlockOutputBufferName() {
+    return Constants.FAST_UPLOAD_BUFFER_ARRAY;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesByteBufferBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesByteBufferBlocks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesByteBufferBlocks.java
new file mode 100644
index 0000000..b1323c4
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesByteBufferBlocks.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.scale;
+
+import org.apache.hadoop.fs.s3a.Constants;
+
+import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BYTEBUFFER;
+
+/**
+ * Use {@link Constants#FAST_UPLOAD_BYTEBUFFER} for buffering.
+ */
+public class ITestS3AHugeFilesByteBufferBlocks
+    extends AbstractSTestS3AHugeFiles {
+
+  protected String getBlockOutputBufferName() {
+    return FAST_UPLOAD_BYTEBUFFER;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesClassicOutput.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesClassicOutput.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesClassicOutput.java
new file mode 100644
index 0000000..45eef24
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesClassicOutput.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.scale;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.Constants;
+
+/**
+ * Use classic output for writing things; tweaks the configuration to do
+ * this after it has been set up in the superclass.
+ * The generator test has been copied and re
+ */
+public class ITestS3AHugeFilesClassicOutput extends AbstractSTestS3AHugeFiles {
+
+  @Override
+  protected Configuration createConfiguration() {
+    final Configuration conf = super.createConfiguration();
+    conf.setBoolean(Constants.FAST_UPLOAD, false);
+    return conf;
+  }
+
+  protected String getBlockOutputBufferName() {
+    return "classic";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesDiskBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesDiskBlocks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesDiskBlocks.java
new file mode 100644
index 0000000..2be5769
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesDiskBlocks.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.scale;
+
+import org.apache.hadoop.fs.s3a.Constants;
+
+/**
+ * Use {@link Constants#FAST_UPLOAD_BUFFER_DISK} for buffering.
+ */
+public class ITestS3AHugeFilesDiskBlocks extends AbstractSTestS3AHugeFiles {
+
+  protected String getBlockOutputBufferName() {
+    return Constants.FAST_UPLOAD_BUFFER_DISK;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
index d861a16..af6d468 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
@@ -20,18 +20,18 @@ package org.apache.hadoop.fs.s3a.scale;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
-import org.apache.hadoop.fs.Path;
-
 import org.apache.hadoop.fs.s3a.S3AInputStream;
 import org.apache.hadoop.fs.s3a.S3AInstrumentation;
 import org.apache.hadoop.fs.s3a.S3ATestConstants;
-import org.apache.hadoop.fs.s3a.S3ATestUtils;
+import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 import org.junit.After;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.rules.TestName;
 import org.junit.rules.Timeout;
@@ -40,6 +40,8 @@ import org.slf4j.LoggerFactory;
 
 import java.io.InputStream;
 
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
+
 /**
  * Base class for scale tests; here is where the common scale configuration
  * keys are defined.
@@ -47,71 +49,18 @@ import java.io.InputStream;
 public class S3AScaleTestBase extends Assert implements S3ATestConstants {
 
   @Rule
-  public TestName methodName = new TestName();
+  public final TestName methodName = new TestName();
 
   @Rule
-  public Timeout testTimeout = new Timeout(30 * 60 * 1000);
+  public Timeout testTimeout = createTestTimeout();
 
-  @BeforeClass
-  public static void nameThread() {
+  @Before
+  public void nameThread() {
     Thread.currentThread().setName("JUnit");
   }
 
-  /**
-   * The number of operations to perform: {@value}.
-   */
-  public static final String KEY_OPERATION_COUNT =
-      SCALE_TEST + "operation.count";
-
-  /**
-   * The number of directory operations to perform: {@value}.
-   */
-  public static final String KEY_DIRECTORY_COUNT =
-      SCALE_TEST + "directory.count";
-
-  /**
-   * The readahead buffer: {@value}.
-   */
-  public static final String KEY_READ_BUFFER_SIZE =
-      S3A_SCALE_TEST + "read.buffer.size";
-
-  public static final int DEFAULT_READ_BUFFER_SIZE = 16384;
-
-  /**
-   * Key for a multi MB test file: {@value}.
-   */
-  public static final String KEY_CSVTEST_FILE =
-      S3A_SCALE_TEST + "csvfile";
-  /**
-   * Default path for the multi MB test file: {@value}.
-   */
-  public static final String DEFAULT_CSVTEST_FILE
-      = "s3a://landsat-pds/scene_list.gz";
-
-  /**
-   * Endpoint for the S3 CSV/scale tests. This defaults to
-   * being us-east.
-   */
-  public static final String KEY_CSVTEST_ENDPOINT =
-      S3A_SCALE_TEST + "csvfile.endpoint";
-
-  /**
-   * Endpoint for the S3 CSV/scale tests. This defaults to
-   * being us-east.
-   */
-  public static final String DEFAULT_CSVTEST_ENDPOINT =
-      "s3.amazonaws.com";
-
-  /**
-   * The default number of operations to perform: {@value}.
-   */
-  public static final long DEFAULT_OPERATION_COUNT = 2005;
-
-  /**
-   * Default number of directories to create when performing
-   * directory performance/scale tests.
-   */
-  public static final int DEFAULT_DIRECTORY_COUNT = 2;
+  public static final int _1KB = 1024;
+  public static final int _1MB = _1KB * _1KB;
 
   protected S3AFileSystem fs;
 
@@ -120,6 +69,8 @@ public class S3AScaleTestBase extends Assert implements S3ATestConstants {
 
   private Configuration conf;
 
+  private boolean enabled;
+
   /**
    * Configuration generator. May be overridden to inject
    * some custom options.
@@ -137,11 +88,33 @@ public class S3AScaleTestBase extends Assert implements S3ATestConstants {
     return conf;
   }
 
+  /**
+   * Setup. This triggers creation of the configuration.
+   */
   @Before
   public void setUp() throws Exception {
-    conf = createConfiguration();
+    demandCreateConfiguration();
     LOG.debug("Scale test operation count = {}", getOperationCount());
-    fs = S3ATestUtils.createTestFileSystem(conf);
+    // multipart purges are disabled on the scale tests
+    fs = createTestFileSystem(conf, false);
+    // check for the test being enabled
+    enabled = getTestPropertyBool(
+        getConf(),
+        KEY_SCALE_TESTS_ENABLED,
+        DEFAULT_SCALE_TESTS_ENABLED);
+    Assume.assumeTrue("Scale test disabled: to enable set property " +
+        KEY_SCALE_TESTS_ENABLED, enabled);
+  }
+
+  /**
+   * Create the configuration if it is not already set up.
+   * @return the configuration.
+   */
+  private synchronized Configuration demandCreateConfiguration() {
+    if (conf == null) {
+      conf = createConfiguration();
+    }
+    return conf;
   }
 
   @After
@@ -160,7 +133,27 @@ public class S3AScaleTestBase extends Assert implements S3ATestConstants {
   }
 
   /**
-   * Describe a test in the logs
+   * Create the timeout for tests. Some large tests may need a larger value.
+   * @return the test timeout to use
+   */
+  protected Timeout createTestTimeout() {
+    demandCreateConfiguration();
+    return new Timeout(
+        getTestTimeoutSeconds() * 1000);
+  }
+
+  /**
+   * Get the test timeout in seconds.
+   * @return the test timeout as set in system properties or the default.
+   */
+  protected static int getTestTimeoutSeconds() {
+    return getTestPropertyInt(null,
+        KEY_TEST_TIMEOUT,
+        DEFAULT_TEST_TIMEOUT);
+  }
+
+  /**
+   * Describe a test in the logs.
    * @param text text to print
    * @param args arguments to format in the printing
    */
@@ -189,4 +182,30 @@ public class S3AScaleTestBase extends Assert implements S3ATestConstants {
     }
   }
 
+  /**
+   * Get the gauge value of a statistic. Raises an assertion if
+   * there is no such gauge.
+   * @param statistic statistic to look up
+   * @return the value.
+   */
+  public long gaugeValue(Statistic statistic) {
+    S3AInstrumentation instrumentation = fs.getInstrumentation();
+    MutableGaugeLong gauge = instrumentation.lookupGauge(statistic.getSymbol());
+    assertNotNull("No gauge " + statistic
+        + " in " + instrumentation.dump("", " = ", "\n", true), gauge);
+    return gauge.value();
+  }
+
+  protected boolean isEnabled() {
+    return enabled;
+  }
+
+  /**
+   * Flag to indicate that this test is being used sequentially. This
+   * is used by some of the scale tests to validate test time expectations.
+   * @return true if the build indicates this test is being run in parallel.
+   */
+  protected boolean isParallelExecution() {
+    return Boolean.getBoolean(S3ATestConstants.KEY_PARALLEL_TEST_EXECUTION);
+  }
 }


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


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

Posted by sj...@apache.org.
HDFS-10986. DFSAdmin should log detailed error message if any. Contributed by MingLiang Liu


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

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

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


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

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


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


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

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


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

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

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


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


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


[41/50] [abbrv] hadoop git commit: HADOOP-13061. Refactor erasure coders. Contributed by Kai Sasaki

Posted by sj...@apache.org.
HADOOP-13061. Refactor erasure coders. Contributed by Kai Sasaki


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

Branch: refs/heads/HADOOP-13070
Commit: c023c748869063fb67d14ea996569c42578d1cea
Parents: bedfec0
Author: Kai Zheng <ka...@intel.com>
Authored: Tue Oct 18 12:02:53 2016 +0600
Committer: Kai Zheng <ka...@intel.com>
Committed: Tue Oct 18 12:02:53 2016 +0600

----------------------------------------------------------------------
 .../hadoop/fs/CommonConfigurationKeys.java      |  26 ---
 .../apache/hadoop/io/erasurecode/CodecUtil.java | 168 ++++++++++++++--
 .../io/erasurecode/ErasureCodeConstants.java    |   3 +-
 .../io/erasurecode/ErasureCodecOptions.java     |  37 ++++
 .../erasurecode/codec/AbstractErasureCodec.java |  53 -----
 .../io/erasurecode/codec/DummyErasureCodec.java |  45 +++++
 .../io/erasurecode/codec/ErasureCodec.java      |  76 +++++--
 .../io/erasurecode/codec/HHXORErasureCodec.java |  20 +-
 .../io/erasurecode/codec/RSErasureCodec.java    |  20 +-
 .../io/erasurecode/codec/XORErasureCodec.java   |  22 ++-
 .../io/erasurecode/codec/package-info.java      |  28 +++
 .../erasurecode/coder/AbstractErasureCoder.java |  64 ------
 .../coder/AbstractErasureCodingStep.java        |  61 ------
 .../coder/AbstractErasureDecoder.java           | 170 ----------------
 .../coder/AbstractErasureEncoder.java           |  62 ------
 .../coder/AbstractHHErasureCodingStep.java      |  49 -----
 .../erasurecode/coder/DummyErasureDecoder.java  |  46 +++++
 .../erasurecode/coder/DummyErasureEncoder.java  |  45 +++++
 .../io/erasurecode/coder/ErasureCoder.java      |  25 ++-
 .../io/erasurecode/coder/ErasureCodingStep.java |   8 +-
 .../io/erasurecode/coder/ErasureDecoder.java    | 198 +++++++++++++++++++
 .../erasurecode/coder/ErasureDecodingStep.java  |  21 +-
 .../io/erasurecode/coder/ErasureEncoder.java    |  91 +++++++++
 .../erasurecode/coder/ErasureEncodingStep.java  |  22 ++-
 .../erasurecode/coder/HHErasureCodingStep.java  |  68 +++++++
 .../erasurecode/coder/HHXORErasureDecoder.java  |  24 +--
 .../coder/HHXORErasureDecodingStep.java         |   2 +-
 .../erasurecode/coder/HHXORErasureEncoder.java  |  19 +-
 .../coder/HHXORErasureEncodingStep.java         |   2 +-
 .../io/erasurecode/coder/RSErasureDecoder.java  |  16 +-
 .../io/erasurecode/coder/RSErasureEncoder.java  |  20 +-
 .../io/erasurecode/coder/XORErasureDecoder.java |  15 +-
 .../io/erasurecode/coder/XORErasureEncoder.java |  16 +-
 .../io/erasurecode/coder/package-info.java      |  28 +++
 .../io/erasurecode/rawcoder/CoderUtil.java      |   2 +-
 .../conf/TestCommonConfigurationFields.java     |   5 +-
 .../erasurecode/TestCodecRawCoderMapping.java   |   3 +-
 .../codec/TestHHXORErasureCodec.java            |   6 +-
 .../erasurecode/coder/TestErasureCoderBase.java |  13 +-
 .../coder/TestHHXORErasureCoder.java            |   4 +-
 .../erasurecode/coder/TestRSErasureCoder.java   |   4 +-
 .../hadoop/hdfs/TestDFSStripedInputStream.java  |   3 +-
 .../hadoop/hdfs/TestDFSStripedOutputStream.java |   6 +-
 .../TestDFSStripedOutputStreamWithFailure.java  |   4 +-
 .../hadoop/hdfs/TestReconstructStripedFile.java |   4 +-
 45 files changed, 964 insertions(+), 660 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
index 2b530f0..fe522b3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
@@ -21,9 +21,6 @@ package org.apache.hadoop.fs;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.http.lib.StaticUserWebFilter;
-import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactory;
-import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactoryLegacy;
-import org.apache.hadoop.io.erasurecode.rawcoder.XORRawErasureCoderFactory;
 
 /** 
  * This class contains constants for configuration keys used
@@ -160,30 +157,7 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
   public static final boolean IO_COMPRESSION_CODEC_LZ4_USELZ4HC_DEFAULT =
       false;
 
-  /**
-   * Erasure Coding configuration family
-   */
 
-  /** Supported erasure codec classes */
-  public static final String IO_ERASURECODE_CODECS_KEY = "io.erasurecode.codecs";
-
-  /** Raw coder factory for the RS default codec. */
-  public static final String IO_ERASURECODE_CODEC_RS_DEFAULT_RAWCODER_KEY =
-      "io.erasurecode.codec.rs-default.rawcoder";
-  public static final String IO_ERASURECODE_CODEC_RS_DEFAULT_RAWCODER_DEFAULT =
-      RSRawErasureCoderFactory.class.getCanonicalName();
-
-  /** Raw coder factory for the RS legacy codec. */
-  public static final String IO_ERASURECODE_CODEC_RS_LEGACY_RAWCODER_KEY =
-      "io.erasurecode.codec.rs-legacy.rawcoder";
-  public static final String IO_ERASURECODE_CODEC_RS_LEGACY_RAWCODER_DEFAULT =
-      RSRawErasureCoderFactoryLegacy.class.getCanonicalName();
-
-  /** Raw coder factory for the XOR codec. */
-  public static final String IO_ERASURECODE_CODEC_XOR_RAWCODER_KEY =
-      "io.erasurecode.codec.xor.rawcoder";
-  public static final String IO_ERASURECODE_CODEC_XOR_RAWCODER_DEFAULT =
-      XORRawErasureCoderFactory.class.getCanonicalName();
 
   /**
    * Service Authorization

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java
index 9cd9561..977bacb 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java
@@ -20,20 +20,108 @@ package org.apache.hadoop.io.erasurecode;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.io.erasurecode.codec.ErasureCodec;
+import org.apache.hadoop.io.erasurecode.codec.HHXORErasureCodec;
+import org.apache.hadoop.io.erasurecode.codec.RSErasureCodec;
+import org.apache.hadoop.io.erasurecode.codec.XORErasureCodec;
+import org.apache.hadoop.io.erasurecode.coder.ErasureDecoder;
+import org.apache.hadoop.io.erasurecode.coder.ErasureEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactory;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactoryLegacy;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoderFactory;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.XORRawErasureCoderFactory;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
 
 /**
- * A codec & coder utility to help create raw coders conveniently.
+ * A codec & coder utility to help create coders conveniently.
+ *
+ * {@link CodecUtil} includes erasure coder configurations key and default
+ * values such as coder class name and erasure codec option values included
+ * by {@link ErasureCodecOptions}. {@link ErasureEncoder} and
+ * {@link ErasureDecoder} are created by createEncoder and createDecoder
+ * respectively.{@link RawErasureEncoder} and {@link RawErasureDecoder} are
+ * are created by createRawEncoder and createRawDecoder.
  */
 @InterfaceAudience.Private
 public final class CodecUtil {
 
+  /** Erasure coder XOR codec. */
+  public static final String IO_ERASURECODE_CODEC_XOR_KEY =
+      "io.erasurecode.codec.xor";
+  public static final String IO_ERASURECODE_CODEC_XOR =
+      XORErasureCodec.class.getCanonicalName();
+  /** Erasure coder Reed-Solomon codec. */
+  public static final String IO_ERASURECODE_CODEC_RS_DEFAULT_KEY =
+      "io.erasurecode.codec.rs";
+  public static final String IO_ERASURECODE_CODEC_RS_DEFAULT =
+      RSErasureCodec.class.getCanonicalName();
+  /** Erasure coder hitch hiker XOR codec. */
+  public static final String IO_ERASURECODE_CODEC_HHXOR_KEY =
+      "io.erasurecode.codec.hhxor";
+  public static final String IO_ERASURECODE_CODEC_HHXOR =
+      HHXORErasureCodec.class.getCanonicalName();
+
+  /** Supported erasure codec classes. */
+
+  /** Raw coder factory for the RS default codec. */
+  public static final String IO_ERASURECODE_CODEC_RS_DEFAULT_RAWCODER_KEY =
+      "io.erasurecode.codec.rs-default.rawcoder";
+  public static final String IO_ERASURECODE_CODEC_RS_DEFAULT_RAWCODER_DEFAULT =
+      RSRawErasureCoderFactory.class.getCanonicalName();
+
+  /** Raw coder factory for the RS legacy codec. */
+  public static final String IO_ERASURECODE_CODEC_RS_LEGACY_RAWCODER_KEY =
+      "io.erasurecode.codec.rs-legacy.rawcoder";
+  public static final String IO_ERASURECODE_CODEC_RS_LEGACY_RAWCODER_DEFAULT =
+      RSRawErasureCoderFactoryLegacy.class.getCanonicalName();
+
+  /** Raw coder factory for the XOR codec. */
+  public static final String IO_ERASURECODE_CODEC_XOR_RAWCODER_KEY =
+      "io.erasurecode.codec.xor.rawcoder";
+  public static final String IO_ERASURECODE_CODEC_XOR_RAWCODER_DEFAULT =
+      XORRawErasureCoderFactory.class.getCanonicalName();
+
   private CodecUtil() { }
 
   /**
+   * Create encoder corresponding to given codec.
+   * @param options Erasure codec options
+   * @return erasure encoder
+   */
+  public static ErasureEncoder createEncoder(Configuration conf,
+      ErasureCodecOptions options) {
+    Preconditions.checkNotNull(conf);
+    Preconditions.checkNotNull(options);
+
+    String codecKey = getCodecClassName(conf,
+        options.getSchema().getCodecName());
+
+    ErasureCodec codec = createCodec(conf, codecKey, options);
+    return codec.createEncoder();
+  }
+
+  /**
+   * Create decoder corresponding to given codec.
+   * @param options Erasure codec options
+   * @return erasure decoder
+   */
+  public static ErasureDecoder createDecoder(Configuration conf,
+      ErasureCodecOptions options) {
+    Preconditions.checkNotNull(conf);
+    Preconditions.checkNotNull(options);
+
+    String codecKey = getCodecClassName(conf,
+        options.getSchema().getCodecName());
+
+    ErasureCodec codec = createCodec(conf, codecKey, options);
+    return codec.createDecoder();
+  }
+
+  /**
    * Create RS raw encoder according to configuration.
    * @param conf configuration
    * @param coderOptions coder options that's used to create the coder
@@ -45,7 +133,7 @@ public final class CodecUtil {
     Preconditions.checkNotNull(conf);
     Preconditions.checkNotNull(codec);
 
-    String rawCoderFactoryKey = getFactNameFromCodec(conf, codec);
+    String rawCoderFactoryKey = getRawCoderFactNameFromCodec(conf, codec);
 
     RawErasureCoderFactory fact = createRawCoderFactory(conf,
         rawCoderFactoryKey);
@@ -65,7 +153,7 @@ public final class CodecUtil {
     Preconditions.checkNotNull(conf);
     Preconditions.checkNotNull(codec);
 
-    String rawCoderFactoryKey = getFactNameFromCodec(conf, codec);
+    String rawCoderFactoryKey = getRawCoderFactNameFromCodec(conf, codec);
 
     RawErasureCoderFactory fact = createRawCoderFactory(conf,
         rawCoderFactoryKey);
@@ -92,22 +180,21 @@ public final class CodecUtil {
     return fact;
   }
 
-  private static String getFactNameFromCodec(Configuration conf, String codec) {
+  private static String getRawCoderFactNameFromCodec(Configuration conf,
+                                                     String codec) {
     switch (codec) {
     case ErasureCodeConstants.RS_DEFAULT_CODEC_NAME:
       return conf.get(
-          CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_DEFAULT_RAWCODER_KEY,
-          CommonConfigurationKeys.
-              IO_ERASURECODE_CODEC_RS_DEFAULT_RAWCODER_DEFAULT);
+          IO_ERASURECODE_CODEC_RS_DEFAULT_RAWCODER_KEY,
+          IO_ERASURECODE_CODEC_RS_DEFAULT_RAWCODER_DEFAULT);
     case ErasureCodeConstants.RS_LEGACY_CODEC_NAME:
       return conf.get(
-          CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_LEGACY_RAWCODER_KEY,
-          CommonConfigurationKeys.
-              IO_ERASURECODE_CODEC_RS_LEGACY_RAWCODER_DEFAULT);
+          IO_ERASURECODE_CODEC_RS_LEGACY_RAWCODER_KEY,
+          IO_ERASURECODE_CODEC_RS_LEGACY_RAWCODER_DEFAULT);
     case ErasureCodeConstants.XOR_CODEC_NAME:
       return conf.get(
-          CommonConfigurationKeys.IO_ERASURECODE_CODEC_XOR_RAWCODER_KEY,
-          CommonConfigurationKeys.IO_ERASURECODE_CODEC_XOR_RAWCODER_DEFAULT);
+          IO_ERASURECODE_CODEC_XOR_RAWCODER_KEY,
+          IO_ERASURECODE_CODEC_XOR_RAWCODER_DEFAULT);
     default:
       // For custom codec, we throw exception if the factory is not configured
       String rawCoderKey = "io.erasurecode.codec." + codec + ".rawcoder";
@@ -119,4 +206,59 @@ public final class CodecUtil {
       return factName;
     }
   }
+
+  private static ErasureCodec createCodec(Configuration conf,
+      String codecClassName, ErasureCodecOptions options) {
+    ErasureCodec codec = null;
+    try {
+      Class<? extends ErasureCodec> codecClass =
+              conf.getClassByName(codecClassName)
+              .asSubclass(ErasureCodec.class);
+      Constructor<? extends ErasureCodec> constructor
+          = codecClass.getConstructor(Configuration.class,
+          ErasureCodecOptions.class);
+      codec = constructor.newInstance(conf, options);
+    } catch (ClassNotFoundException | InstantiationException |
+            IllegalAccessException | NoSuchMethodException |
+            InvocationTargetException e) {
+      throw new RuntimeException("Failed to create erasure codec", e);
+    }
+
+    if (codec == null) {
+      throw new RuntimeException("Failed to create erasure codec");
+    }
+
+    return codec;
+  }
+
+  private static String getCodecClassName(Configuration conf, String codec) {
+    switch (codec) {
+    case ErasureCodeConstants.RS_DEFAULT_CODEC_NAME:
+      return conf.get(
+          CodecUtil.IO_ERASURECODE_CODEC_RS_DEFAULT_KEY,
+          CodecUtil.IO_ERASURECODE_CODEC_RS_DEFAULT);
+    case ErasureCodeConstants.RS_LEGACY_CODEC_NAME:
+      //TODO:rs-legacy should be handled differently.
+      return conf.get(
+          CodecUtil.IO_ERASURECODE_CODEC_RS_DEFAULT_KEY,
+          CodecUtil.IO_ERASURECODE_CODEC_RS_DEFAULT);
+    case ErasureCodeConstants.XOR_CODEC_NAME:
+      return conf.get(
+          CodecUtil.IO_ERASURECODE_CODEC_XOR_KEY,
+          CodecUtil.IO_ERASURECODE_CODEC_XOR);
+    case ErasureCodeConstants.HHXOR_CODEC_NAME:
+      return conf.get(
+          CodecUtil.IO_ERASURECODE_CODEC_HHXOR_KEY,
+          CodecUtil.IO_ERASURECODE_CODEC_HHXOR);
+    default:
+      // For custom codec, we throw exception if the factory is not configured
+      String codecKey = "io.erasurecode.codec." + codec + ".coder";
+      String codecClass = conf.get(codecKey);
+      if (codecClass == null) {
+        throw new IllegalArgumentException("Codec not configured " +
+                "for custom codec " + codec);
+      }
+      return codecClass;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
index 1fb9488..8d6ff85 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
@@ -22,12 +22,13 @@ package org.apache.hadoop.io.erasurecode;
  */
 public final class ErasureCodeConstants {
 
-  private ErasureCodeConstants(){
+  private ErasureCodeConstants() {
   }
 
   public static final String RS_DEFAULT_CODEC_NAME = "rs-default";
   public static final String RS_LEGACY_CODEC_NAME = "rs-legacy";
   public static final String XOR_CODEC_NAME = "xor";
+  public static final String HHXOR_CODEC_NAME = "hhxor";
 
   public static final ECSchema RS_6_3_SCHEMA = new ECSchema(
       RS_DEFAULT_CODEC_NAME, 6, 3);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodecOptions.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodecOptions.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodecOptions.java
new file mode 100644
index 0000000..7ae0fc2
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodecOptions.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.erasurecode;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Erasure codec options.
+ */
+@InterfaceAudience.Private
+public class ErasureCodecOptions {
+  private ECSchema schema;
+
+  public ErasureCodecOptions(ECSchema schema) {
+    this.schema = schema;
+  }
+
+  public ECSchema getSchema() {
+    return schema;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java
deleted file mode 100644
index e2fb2cb..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java
+++ /dev/null
@@ -1,53 +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.io.erasurecode.codec;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.erasurecode.ECSchema;
-import org.apache.hadoop.io.erasurecode.grouper.BlockGrouper;
-
-/**
- * Abstract Erasure Codec that implements {@link ErasureCodec}.
- */
-@InterfaceAudience.Private
-public abstract class AbstractErasureCodec extends Configured
-    implements ErasureCodec {
-
-  private final ECSchema schema;
-
-  public AbstractErasureCodec(ECSchema schema) {
-    this.schema = schema;
-  }
-
-  public String getName() {
-    return schema.getCodecName();
-  }
-
-  public ECSchema getSchema() {
-    return schema;
-  }
-
-  @Override
-  public BlockGrouper createBlockGrouper() {
-    BlockGrouper blockGrouper = new BlockGrouper();
-    blockGrouper.setSchema(getSchema());
-
-    return blockGrouper;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/DummyErasureCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/DummyErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/DummyErasureCodec.java
new file mode 100644
index 0000000..3646f72
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/DummyErasureCodec.java
@@ -0,0 +1,45 @@
+/**
+ * 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.erasurecode.codec;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.erasurecode.ErasureCodecOptions;
+import org.apache.hadoop.io.erasurecode.coder.DummyErasureDecoder;
+import org.apache.hadoop.io.erasurecode.coder.DummyErasureEncoder;
+import org.apache.hadoop.io.erasurecode.coder.ErasureDecoder;
+import org.apache.hadoop.io.erasurecode.coder.ErasureEncoder;
+
+/**
+ * Dummy erasure coder does not real coding computing. This is used for only
+ * test or performance comparison with other erasure coders.
+ */
+public class DummyErasureCodec extends ErasureCodec {
+  public DummyErasureCodec(Configuration conf, ErasureCodecOptions options) {
+    super(conf, options);
+  }
+
+  @Override
+  public ErasureEncoder createEncoder() {
+    return new DummyErasureEncoder(getCoderOptions());
+  }
+
+  @Override
+  public ErasureDecoder createDecoder() {
+    return new DummyErasureDecoder(getCoderOptions());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java
index a2ea865..c75eaea 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java
@@ -18,34 +18,76 @@
 package org.apache.hadoop.io.erasurecode.codec;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.io.erasurecode.coder.ErasureCoder;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.erasurecode.CodecUtil;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.ErasureCodecOptions;
+import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
+import org.apache.hadoop.io.erasurecode.coder.ErasureDecoder;
+import org.apache.hadoop.io.erasurecode.coder.ErasureEncoder;
 import org.apache.hadoop.io.erasurecode.grouper.BlockGrouper;
 
 /**
- * Erasure Codec API that's to cover the essential specific aspects of a code.
- * Currently it cares only block grouper and erasure coder. In future we may
- * add more aspects here to make the behaviors customizable.
+ * Abstract Erasure Codec is defines the interface of each actual erasure
+ * codec classes.
  */
 @InterfaceAudience.Private
-public interface ErasureCodec extends Configurable {
+public abstract class ErasureCodec {
 
-  /**
-   * Create block grouper
-   * @return block grouper
-   */
-  public BlockGrouper createBlockGrouper();
+  private ECSchema schema;
+  private ErasureCodecOptions codecOptions;
+  private ErasureCoderOptions coderOptions;
+
+  public ErasureCodec(Configuration conf,
+                      ErasureCodecOptions options) {
+    this.schema = options.getSchema();
+    this.codecOptions = options;
+    boolean allowChangeInputs = false;
+    this.coderOptions = new ErasureCoderOptions(schema.getNumDataUnits(),
+        schema.getNumParityUnits(), allowChangeInputs, false);
+  }
+
+  public String getName() {
+    return schema.getCodecName();
+  }
+
+  public ECSchema getSchema() {
+    return schema;
+  }
 
   /**
-   * Create Erasure Encoder
-   * @return erasure encoder
+   * Get a {@link ErasureCodecOptions}.
+   * @return erasure codec options
    */
-  public ErasureCoder createEncoder();
+  public ErasureCodecOptions getCodecOptions() {
+    return codecOptions;
+  }
+
+  protected void setCodecOptions(ErasureCodecOptions options) {
+    this.codecOptions = options;
+    this.schema = options.getSchema();
+  }
 
   /**
-   * Create Erasure Decoder
-   * @return erasure decoder
+   * Get a {@link ErasureCoderOptions}.
+   * @return erasure coder options
    */
-  public ErasureCoder createDecoder();
+  public ErasureCoderOptions getCoderOptions() {
+    return coderOptions;
+  }
+
+  protected void setCoderOptions(ErasureCoderOptions options) {
+    this.coderOptions = options;
+  }
+
+  public abstract ErasureEncoder createEncoder();
+
+  public abstract ErasureDecoder createDecoder();
+
+  public BlockGrouper createBlockGrouper() {
+    BlockGrouper blockGrouper = new BlockGrouper();
+    blockGrouper.setSchema(getSchema());
 
+    return blockGrouper;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/HHXORErasureCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/HHXORErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/HHXORErasureCodec.java
index 3c8061d..42380f3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/HHXORErasureCodec.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/HHXORErasureCodec.java
@@ -18,8 +18,10 @@
 package org.apache.hadoop.io.erasurecode.codec;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.erasurecode.ECSchema;
-import org.apache.hadoop.io.erasurecode.coder.ErasureCoder;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.erasurecode.ErasureCodecOptions;
+import org.apache.hadoop.io.erasurecode.coder.ErasureDecoder;
+import org.apache.hadoop.io.erasurecode.coder.ErasureEncoder;
 import org.apache.hadoop.io.erasurecode.coder.HHXORErasureDecoder;
 import org.apache.hadoop.io.erasurecode.coder.HHXORErasureEncoder;
 
@@ -27,19 +29,19 @@ import org.apache.hadoop.io.erasurecode.coder.HHXORErasureEncoder;
  * A Hitchhiker-XOR erasure codec.
  */
 @InterfaceAudience.Private
-public class HHXORErasureCodec extends AbstractErasureCodec {
+public class HHXORErasureCodec extends ErasureCodec {
 
-  public HHXORErasureCodec(ECSchema schema) {
-    super(schema);
+  public HHXORErasureCodec(Configuration conf, ErasureCodecOptions options) {
+    super(conf, options);
   }
 
   @Override
-  public ErasureCoder createEncoder() {
-    return new HHXORErasureEncoder(getSchema());
+  public ErasureEncoder createEncoder() {
+    return new HHXORErasureEncoder(getCoderOptions());
   }
 
   @Override
-  public ErasureCoder createDecoder() {
-    return new HHXORErasureDecoder(getSchema());
+  public ErasureDecoder createDecoder() {
+    return new HHXORErasureDecoder(getCoderOptions());
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java
index 7a5786e..e57161c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java
@@ -18,8 +18,10 @@
 package org.apache.hadoop.io.erasurecode.codec;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.erasurecode.ECSchema;
-import org.apache.hadoop.io.erasurecode.coder.ErasureCoder;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.erasurecode.ErasureCodecOptions;
+import org.apache.hadoop.io.erasurecode.coder.ErasureDecoder;
+import org.apache.hadoop.io.erasurecode.coder.ErasureEncoder;
 import org.apache.hadoop.io.erasurecode.coder.RSErasureDecoder;
 import org.apache.hadoop.io.erasurecode.coder.RSErasureEncoder;
 
@@ -27,19 +29,19 @@ import org.apache.hadoop.io.erasurecode.coder.RSErasureEncoder;
  * A Reed-Solomon erasure codec.
  */
 @InterfaceAudience.Private
-public class RSErasureCodec extends AbstractErasureCodec {
+public class RSErasureCodec extends ErasureCodec {
 
-  public RSErasureCodec(ECSchema schema) {
-    super(schema);
+  public RSErasureCodec(Configuration conf, ErasureCodecOptions options) {
+    super(conf, options);
   }
 
   @Override
-  public ErasureCoder createEncoder() {
-    return new RSErasureEncoder(getSchema());
+  public ErasureEncoder createEncoder() {
+    return new RSErasureEncoder(getCoderOptions());
   }
 
   @Override
-  public ErasureCoder createDecoder() {
-    return new RSErasureDecoder(getSchema());
+  public ErasureDecoder createDecoder() {
+    return new RSErasureDecoder(getCoderOptions());
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java
index 9c38fc2..f9a0608 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java
@@ -18,8 +18,10 @@
 package org.apache.hadoop.io.erasurecode.codec;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.erasurecode.ECSchema;
-import org.apache.hadoop.io.erasurecode.coder.ErasureCoder;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.erasurecode.ErasureCodecOptions;
+import org.apache.hadoop.io.erasurecode.coder.ErasureDecoder;
+import org.apache.hadoop.io.erasurecode.coder.ErasureEncoder;
 import org.apache.hadoop.io.erasurecode.coder.XORErasureDecoder;
 import org.apache.hadoop.io.erasurecode.coder.XORErasureEncoder;
 
@@ -27,20 +29,20 @@ import org.apache.hadoop.io.erasurecode.coder.XORErasureEncoder;
  * A XOR erasure codec.
  */
 @InterfaceAudience.Private
-public class XORErasureCodec extends AbstractErasureCodec {
+public class XORErasureCodec extends ErasureCodec {
 
-  public XORErasureCodec(ECSchema schema) {
-    super(schema);
-    assert(schema.getNumParityUnits() == 1);
+  public XORErasureCodec(Configuration conf, ErasureCodecOptions options) {
+    super(conf, options);
+    assert(options.getSchema().getNumParityUnits() == 1);
   }
 
   @Override
-  public ErasureCoder createEncoder() {
-    return new XORErasureEncoder(getSchema());
+  public ErasureEncoder createEncoder() {
+    return new XORErasureEncoder(getCoderOptions());
   }
 
   @Override
-  public ErasureCoder createDecoder() {
-    return new XORErasureDecoder(getSchema());
+  public ErasureDecoder createDecoder() {
+    return new XORErasureDecoder(getCoderOptions());
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/package-info.java
new file mode 100644
index 0000000..ddfdd4b
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/package-info.java
@@ -0,0 +1,28 @@
+/**
+ * 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.
+ */
+
+/**
+ * Erasure codec framework.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.io.erasurecode.codec;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
deleted file mode 100644
index 1d1641f..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
+++ /dev/null
@@ -1,64 +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.io.erasurecode.coder;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.erasurecode.ECSchema;
-
-/**
- * A common class of basic facilities to be shared by encoder and decoder
- *
- * It implements the {@link ErasureCoder} interface.
- */
-@InterfaceAudience.Private
-public abstract class AbstractErasureCoder
-    extends Configured implements ErasureCoder {
-
-  private final int numDataUnits;
-  private final int numParityUnits;
-
-  public AbstractErasureCoder(int numDataUnits, int numParityUnits) {
-    this.numDataUnits = numDataUnits;
-    this.numParityUnits = numParityUnits;
-  }
-
-  public AbstractErasureCoder(ECSchema schema) {
-    this(schema.getNumDataUnits(), schema.getNumParityUnits());
-  }
-
-  @Override
-  public int getNumDataUnits() {
-    return numDataUnits;
-  }
-
-  @Override
-  public int getNumParityUnits() {
-    return numParityUnits;
-  }
-
-  @Override
-  public boolean preferDirectBuffer() {
-    return false;
-  }
-
-  @Override
-  public void release() {
-    // Nothing to do by default
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCodingStep.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCodingStep.java
deleted file mode 100644
index ccff92d..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCodingStep.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.io.erasurecode.coder;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.erasurecode.ECBlock;
-
-/**
- * Abstract class for common facilities shared by {@link ErasureEncodingStep}
- * and {@link ErasureDecodingStep}.
- *
- * It implements {@link ErasureEncodingStep}.
- */
-@InterfaceAudience.Private
-public abstract class AbstractErasureCodingStep implements ErasureCodingStep {
-
-  private ECBlock[] inputBlocks;
-  private ECBlock[] outputBlocks;
-
-  /**
-   * Constructor given input blocks and output blocks.
-   * @param inputBlocks
-   * @param outputBlocks
-   */
-  public AbstractErasureCodingStep(ECBlock[] inputBlocks,
-                                   ECBlock[] outputBlocks) {
-    this.inputBlocks = inputBlocks;
-    this.outputBlocks = outputBlocks;
-  }
-
-  @Override
-  public ECBlock[] getInputBlocks() {
-    return inputBlocks;
-  }
-
-  @Override
-  public ECBlock[] getOutputBlocks() {
-    return outputBlocks;
-  }
-
-  @Override
-  public void finish() {
-    // NOOP by default
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
deleted file mode 100644
index d976dd1..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
+++ /dev/null
@@ -1,170 +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.io.erasurecode.coder;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.erasurecode.ECBlock;
-import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-import org.apache.hadoop.io.erasurecode.ECSchema;
-
-/**
- * An abstract erasure decoder that's to be inherited by new decoders.
- *
- * It implements the {@link ErasureCoder} interface.
- */
-@InterfaceAudience.Private
-public abstract class AbstractErasureDecoder extends AbstractErasureCoder {
-
-  public AbstractErasureDecoder(int numDataUnits, int numParityUnits) {
-    super(numDataUnits, numParityUnits);
-  }
-
-  public AbstractErasureDecoder(ECSchema schema) {
-    super(schema);
-  }
-
-  @Override
-  public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup) {
-    // We may have more than this when considering complicate cases. HADOOP-11550
-    return prepareDecodingStep(blockGroup);
-  }
-
-  /**
-   * Perform decoding against a block blockGroup.
-   * @param blockGroup
-   * @return decoding step for caller to do the real work
-   */
-  protected abstract ErasureCodingStep prepareDecodingStep(
-      ECBlockGroup blockGroup);
-
-  /**
-   * We have all the data blocks and parity blocks as input blocks for
-   * recovering by default. It's codec specific
-   * @param blockGroup
-   * @return input blocks
-   */
-  protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) {
-    ECBlock[] inputBlocks = new ECBlock[getNumDataUnits() +
-            getNumParityUnits()];
-
-    System.arraycopy(blockGroup.getDataBlocks(), 0, inputBlocks,
-            0, getNumDataUnits());
-
-    System.arraycopy(blockGroup.getParityBlocks(), 0, inputBlocks,
-            getNumDataUnits(), getNumParityUnits());
-
-    return inputBlocks;
-  }
-
-  /**
-   * Which blocks were erased ?
-   * @param blockGroup
-   * @return output blocks to recover
-   */
-  protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
-    ECBlock[] outputBlocks = new ECBlock[getNumErasedBlocks(blockGroup)];
-
-    int idx = 0;
-
-    for (int i = 0; i < getNumDataUnits(); i++) {
-      if (blockGroup.getDataBlocks()[i].isErased()) {
-        outputBlocks[idx++] = blockGroup.getDataBlocks()[i];
-      }
-    }
-
-    for (int i = 0; i < getNumParityUnits(); i++) {
-      if (blockGroup.getParityBlocks()[i].isErased()) {
-        outputBlocks[idx++] = blockGroup.getParityBlocks()[i];
-      }
-    }
-
-    return outputBlocks;
-  }
-
-  /**
-   * Get the number of erased blocks in the block group.
-   * @param blockGroup
-   * @return number of erased blocks
-   */
-  protected int getNumErasedBlocks(ECBlockGroup blockGroup) {
-    int num = getNumErasedBlocks(blockGroup.getParityBlocks());
-    num += getNumErasedBlocks(blockGroup.getDataBlocks());
-    return num;
-  }
-
-  /**
-   * Find out how many blocks are erased.
-   * @param inputBlocks all the input blocks
-   * @return number of erased blocks
-   */
-  protected static int getNumErasedBlocks(ECBlock[] inputBlocks) {
-    int numErased = 0;
-    for (int i = 0; i < inputBlocks.length; i++) {
-      if (inputBlocks[i].isErased()) {
-        numErased ++;
-      }
-    }
-
-    return numErased;
-  }
-
-  /**
-   * Get indexes of erased blocks from inputBlocks
-   * @param inputBlocks
-   * @return indexes of erased blocks from inputBlocks
-   */
-  protected int[] getErasedIndexes(ECBlock[] inputBlocks) {
-    int numErased = getNumErasedBlocks(inputBlocks);
-    if (numErased == 0) {
-      return new int[0];
-    }
-
-    int[] erasedIndexes = new int[numErased];
-    int i = 0, j = 0;
-    for (; i < inputBlocks.length && j < erasedIndexes.length; i++) {
-      if (inputBlocks[i].isErased()) {
-        erasedIndexes[j++] = i;
-      }
-    }
-
-    return erasedIndexes;
-  }
-
-  /**
-   * Get erased input blocks from inputBlocks
-   * @param inputBlocks
-   * @return an array of erased blocks from inputBlocks
-   */
-  protected ECBlock[] getErasedBlocks(ECBlock[] inputBlocks) {
-    int numErased = getNumErasedBlocks(inputBlocks);
-    if (numErased == 0) {
-      return new ECBlock[0];
-    }
-
-    ECBlock[] erasedBlocks = new ECBlock[numErased];
-    int i = 0, j = 0;
-    for (; i < inputBlocks.length && j < erasedBlocks.length; i++) {
-      if (inputBlocks[i].isErased()) {
-        erasedBlocks[j++] = inputBlocks[i];
-      }
-    }
-
-    return erasedBlocks;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
deleted file mode 100644
index 5ec6b4e..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
+++ /dev/null
@@ -1,62 +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.io.erasurecode.coder;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.erasurecode.ECBlock;
-import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-import org.apache.hadoop.io.erasurecode.ECSchema;
-
-/**
- * An abstract erasure encoder that's to be inherited by new encoders.
- *
- * It implements the {@link ErasureCoder} interface.
- */
-@InterfaceAudience.Private
-public abstract class AbstractErasureEncoder extends AbstractErasureCoder {
-
-  public AbstractErasureEncoder(int numDataUnits, int numParityUnits) {
-    super(numDataUnits, numParityUnits);
-  }
-
-  public AbstractErasureEncoder(ECSchema schema) {
-    super(schema);
-  }
-
-  @Override
-  public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup) {
-    // We may have more than this when considering complicate cases. HADOOP-11550
-    return prepareEncodingStep(blockGroup);
-  }
-
-  /**
-   * Perform encoding against a block group.
-   * @param blockGroup
-   * @return encoding step for caller to do the real work
-   */
-  protected abstract ErasureCodingStep prepareEncodingStep(
-      ECBlockGroup blockGroup);
-
-  protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) {
-    return blockGroup.getDataBlocks();
-  }
-
-  protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
-    return blockGroup.getParityBlocks();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractHHErasureCodingStep.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractHHErasureCodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractHHErasureCodingStep.java
deleted file mode 100644
index e577c5d..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractHHErasureCodingStep.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.io.erasurecode.coder;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.erasurecode.ECBlock;
-
-/**
- * Abstract class for Hitchhiker common facilities shared by
- * {@link HHXORErasureEncodingStep}and {@link HHXORErasureDecodingStep}.
- *
- * It implements {@link AbstractErasureCodingStep}.
- */
-@InterfaceAudience.Private
-public abstract class AbstractHHErasureCodingStep
-        extends AbstractErasureCodingStep {
-
-  private static final int SUB_PACKET_SIZE = 2;
-
-  /**
-   * Constructor given input blocks and output blocks.
-   *
-   * @param inputBlocks
-   * @param outputBlocks
-   */
-  public AbstractHHErasureCodingStep(ECBlock[] inputBlocks,
-                                     ECBlock[] outputBlocks) {
-    super(inputBlocks, outputBlocks);
-  }
-
-  protected int getSubPacketSize() {
-    return SUB_PACKET_SIZE;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/DummyErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/DummyErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/DummyErasureDecoder.java
new file mode 100644
index 0000000..8a9434e
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/DummyErasureDecoder.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
+import org.apache.hadoop.io.erasurecode.rawcoder.DummyRawDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+
+/**
+ * Dummy erasure decoder does no real computation. Instead, it just returns
+ * zero bytes. This decoder can be used to isolate the performance issue to
+ * HDFS side logic instead of codec, and is intended for test only.
+ */
+public class DummyErasureDecoder extends ErasureDecoder {
+  public DummyErasureDecoder(ErasureCoderOptions options) {
+    super(options);
+  }
+
+  @Override
+  protected ErasureCodingStep prepareDecodingStep(ECBlockGroup blockGroup) {
+    RawErasureDecoder rawDecoder = new DummyRawDecoder(getOptions());
+
+    ECBlock[] inputBlocks = getInputBlocks(blockGroup);
+
+    return new ErasureDecodingStep(inputBlocks,
+        getErasedIndexes(inputBlocks),
+        getOutputBlocks(blockGroup), rawDecoder);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/DummyErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/DummyErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/DummyErasureEncoder.java
new file mode 100644
index 0000000..fc04f5e
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/DummyErasureEncoder.java
@@ -0,0 +1,45 @@
+/**
+ * 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.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
+import org.apache.hadoop.io.erasurecode.rawcoder.DummyRawEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
+
+/**
+ * Dummy erasure encoder does no real computation. Instead, it just returns
+ * zero bytes. This decoder can be used to isolate the performance issue to
+ * HDFS side logic instead of codec, and is intended for test only.
+ */
+public class DummyErasureEncoder extends ErasureEncoder {
+  public DummyErasureEncoder(ErasureCoderOptions options) {
+    super(options);
+  }
+
+  @Override
+  protected ErasureCodingStep prepareEncodingStep(ECBlockGroup blockGroup) {
+    RawErasureEncoder rawEncoder = new DummyRawEncoder(getOptions());
+
+    ECBlock[] inputBlocks = getInputBlocks(blockGroup);
+
+    return new ErasureEncodingStep(inputBlocks,
+        getOutputBlocks(blockGroup), rawEncoder);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
index 49cc8dd..b5ae1f1 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.io.erasurecode.coder;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
 
 /**
  * An erasure coder to perform encoding or decoding given a group. Generally it
@@ -39,18 +40,25 @@ import org.apache.hadoop.io.erasurecode.ECBlockGroup;
 public interface ErasureCoder extends Configurable {
 
   /**
-   * The number of data input units for the coding. A unit can be a byte,
-   * chunk or buffer or even a block.
+   * The number of data input units for the coding. A unit can be a byte, chunk
+   * or buffer or even a block.
    * @return count of data input units
    */
-  public int getNumDataUnits();
+  int getNumDataUnits();
 
   /**
    * The number of parity output units for the coding. A unit can be a byte,
    * chunk, buffer or even a block.
    * @return count of parity output units
    */
-  public int getNumParityUnits();
+  int getNumParityUnits();
+
+  /**
+   * The options of erasure coder. This option is passed to
+   * raw erasure coder as it is.
+   * @return erasure coder options
+   */
+  ErasureCoderOptions getOptions();
 
   /**
    * Calculate the encoding or decoding steps given a block blockGroup.
@@ -61,7 +69,7 @@ public interface ErasureCoder extends Configurable {
    * @param blockGroup the erasure coding block group containing all necessary
    *                   information for codec calculation
    */
-  public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup);
+  ErasureCodingStep calculateCoding(ECBlockGroup blockGroup);
 
   /**
    * Tell if direct or off-heap buffer is preferred or not. It's for callers to
@@ -70,10 +78,11 @@ public interface ErasureCoder extends Configurable {
    * @return true if direct buffer is preferred for performance consideration,
    * otherwise false.
    */
-  public boolean preferDirectBuffer();
+  boolean preferDirectBuffer();
 
   /**
-   * Release the resources if any. Good chance to invoke RawErasureCoder#release.
+   * Release the resources if any. Good chance to invoke
+   * RawErasureCoder#release.
    */
-  public void release();
+  void release();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java
index 991eb56..9dd0aed 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java
@@ -32,14 +32,14 @@ public interface ErasureCodingStep {
    * or parity blocks.
    * @return input blocks
    */
-  public ECBlock[] getInputBlocks();
+  ECBlock[] getInputBlocks();
 
   /**
    * Output blocks of writable buffers involved in this step, may be data
    * blocks or parity blocks.
    * @return output blocks
    */
-  public ECBlock[] getOutputBlocks();
+  ECBlock[] getOutputBlocks();
 
   /**
    * Perform encoding or decoding given the input chunks, and generated results
@@ -47,11 +47,11 @@ public interface ErasureCodingStep {
    * @param inputChunks
    * @param outputChunks
    */
-  public void performCoding(ECChunk[] inputChunks, ECChunk[] outputChunks);
+  void performCoding(ECChunk[] inputChunks, ECChunk[] outputChunks);
 
   /**
    * Notify erasure coder that all the chunks of input blocks are processed so
    * the coder can be able to update internal states, considering next step.
    */
-  public void finish();
+  void finish();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java
new file mode 100644
index 0000000..faf44d9
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java
@@ -0,0 +1,198 @@
+/**
+ * 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.erasurecode.coder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
+
+/**
+ * An abstract erasure decoder that's to be inherited by new decoders.
+ *
+ * It implements the {@link ErasureCoder} interface.
+ */
+@InterfaceAudience.Private
+public abstract class ErasureDecoder extends Configured
+    implements ErasureCoder {
+  private final int numDataUnits;
+  private final int numParityUnits;
+  private final ErasureCoderOptions options;
+
+  public ErasureDecoder(ErasureCoderOptions options) {
+    this.options = options;
+    this.numDataUnits = options.getNumDataUnits();
+    this.numParityUnits = options.getNumParityUnits();
+  }
+
+  @Override
+  public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup) {
+    // We may have more than this when considering complicate cases. HADOOP-11550
+    return prepareDecodingStep(blockGroup);
+  }
+
+  @Override
+  public int getNumDataUnits() {
+    return this.numDataUnits;
+  }
+
+  @Override
+  public int getNumParityUnits() {
+    return this.numParityUnits;
+  }
+
+  @Override
+  public ErasureCoderOptions getOptions() {
+    return options;
+  }
+
+  /**
+   * We have all the data blocks and parity blocks as input blocks for
+   * recovering by default. It's codec specific
+   * @param blockGroup
+   * @return input blocks
+   */
+  protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) {
+    ECBlock[] inputBlocks = new ECBlock[getNumDataUnits() +
+            getNumParityUnits()];
+
+    System.arraycopy(blockGroup.getDataBlocks(), 0, inputBlocks,
+            0, getNumDataUnits());
+
+    System.arraycopy(blockGroup.getParityBlocks(), 0, inputBlocks,
+            getNumDataUnits(), getNumParityUnits());
+
+    return inputBlocks;
+  }
+
+  /**
+   * Which blocks were erased ?
+   * @param blockGroup
+   * @return output blocks to recover
+   */
+  protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
+    ECBlock[] outputBlocks = new ECBlock[getNumErasedBlocks(blockGroup)];
+
+    int idx = 0;
+
+    for (int i = 0; i < getNumDataUnits(); i++) {
+      if (blockGroup.getDataBlocks()[i].isErased()) {
+        outputBlocks[idx++] = blockGroup.getDataBlocks()[i];
+      }
+    }
+
+    for (int i = 0; i < getNumParityUnits(); i++) {
+      if (blockGroup.getParityBlocks()[i].isErased()) {
+        outputBlocks[idx++] = blockGroup.getParityBlocks()[i];
+      }
+    }
+
+    return outputBlocks;
+  }
+
+  @Override
+  public boolean preferDirectBuffer() {
+    return false;
+  }
+
+  @Override
+  public void release() {
+    // Nothing to do by default
+  }
+
+  /**
+   * Perform decoding against a block blockGroup.
+   * @param blockGroup
+   * @return decoding step for caller to do the real work
+   */
+  protected abstract ErasureCodingStep prepareDecodingStep(
+      ECBlockGroup blockGroup);
+
+  /**
+   * Get the number of erased blocks in the block group.
+   * @param blockGroup
+   * @return number of erased blocks
+   */
+  protected int getNumErasedBlocks(ECBlockGroup blockGroup) {
+    int num = getNumErasedBlocks(blockGroup.getParityBlocks());
+    num += getNumErasedBlocks(blockGroup.getDataBlocks());
+    return num;
+  }
+
+  /**
+   * Find out how many blocks are erased.
+   * @param inputBlocks all the input blocks
+   * @return number of erased blocks
+   */
+  protected static int getNumErasedBlocks(ECBlock[] inputBlocks) {
+    int numErased = 0;
+    for (int i = 0; i < inputBlocks.length; i++) {
+      if (inputBlocks[i].isErased()) {
+        numErased ++;
+      }
+    }
+
+    return numErased;
+  }
+
+  /**
+   * Get indexes of erased blocks from inputBlocks
+   * @param inputBlocks
+   * @return indexes of erased blocks from inputBlocks
+   */
+  protected int[] getErasedIndexes(ECBlock[] inputBlocks) {
+    int numErased = getNumErasedBlocks(inputBlocks);
+    if (numErased == 0) {
+      return new int[0];
+    }
+
+    int[] erasedIndexes = new int[numErased];
+    int i = 0, j = 0;
+    for (; i < inputBlocks.length && j < erasedIndexes.length; i++) {
+      if (inputBlocks[i].isErased()) {
+        erasedIndexes[j++] = i;
+      }
+    }
+
+    return erasedIndexes;
+  }
+
+  /**
+   * Get erased input blocks from inputBlocks
+   * @param inputBlocks
+   * @return an array of erased blocks from inputBlocks
+   */
+  protected ECBlock[] getErasedBlocks(ECBlock[] inputBlocks) {
+    int numErased = getNumErasedBlocks(inputBlocks);
+    if (numErased == 0) {
+      return new ECBlock[0];
+    }
+
+    ECBlock[] erasedBlocks = new ECBlock[numErased];
+    int i = 0, j = 0;
+    for (; i < inputBlocks.length && j < erasedBlocks.length; i++) {
+      if (inputBlocks[i].isErased()) {
+        erasedBlocks[j++] = inputBlocks[i];
+      }
+    }
+
+    return erasedBlocks;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java
index b319cfa..ae396a2 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java
@@ -27,7 +27,9 @@ import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
  * a decoding step involved in the whole process of decoding a block group.
  */
 @InterfaceAudience.Private
-public class ErasureDecodingStep extends AbstractErasureCodingStep {
+public class ErasureDecodingStep implements ErasureCodingStep {
+  private ECBlock[] inputBlocks;
+  private ECBlock[] outputBlocks;
   private int[] erasedIndexes;
   private RawErasureDecoder rawDecoder;
 
@@ -41,7 +43,8 @@ public class ErasureDecodingStep extends AbstractErasureCodingStep {
   public ErasureDecodingStep(ECBlock[] inputBlocks, int[] erasedIndexes,
                              ECBlock[] outputBlocks,
                              RawErasureDecoder rawDecoder) {
-    super(inputBlocks, outputBlocks);
+    this.inputBlocks = inputBlocks;
+    this.outputBlocks = outputBlocks;
     this.erasedIndexes = erasedIndexes;
     this.rawDecoder = rawDecoder;
   }
@@ -51,4 +54,18 @@ public class ErasureDecodingStep extends AbstractErasureCodingStep {
     rawDecoder.decode(inputChunks, erasedIndexes, outputChunks);
   }
 
+  @Override
+  public ECBlock[] getInputBlocks() {
+    return inputBlocks;
+  }
+
+  @Override
+  public ECBlock[] getOutputBlocks() {
+    return outputBlocks;
+  }
+
+  @Override
+  public void finish() {
+    // TODO: Finalize decoder if necessary
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java
new file mode 100644
index 0000000..81666e9
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java
@@ -0,0 +1,91 @@
+/**
+ * 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.erasurecode.coder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
+
+/**
+ * An abstract erasure encoder that's to be inherited by new encoders.
+ *
+ * It implements the {@link ErasureCoder} interface.
+ */
+@InterfaceAudience.Private
+public abstract class ErasureEncoder extends Configured
+    implements ErasureCoder {
+
+  private final int numDataUnits;
+  private final int numParityUnits;
+  private final ErasureCoderOptions options;
+
+  public ErasureEncoder(ErasureCoderOptions options) {
+    this.options = options;
+    this.numDataUnits = options.getNumDataUnits();
+    this.numParityUnits = options.getNumParityUnits();
+  }
+
+  @Override
+  public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup) {
+    // We may have more than this when considering complicate cases. HADOOP-11550
+    return prepareEncodingStep(blockGroup);
+  }
+
+  @Override
+  public int getNumDataUnits() {
+    return numDataUnits;
+  }
+
+  @Override
+  public int getNumParityUnits() {
+    return numParityUnits;
+  }
+
+  @Override
+  public ErasureCoderOptions getOptions() {
+    return options;
+  }
+
+  protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) {
+    return blockGroup.getDataBlocks();
+  }
+
+  protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
+    return blockGroup.getParityBlocks();
+  }
+
+  @Override
+  public boolean preferDirectBuffer() {
+    return false;
+  }
+
+  @Override
+  public void release() {
+    // Nothing to do by default
+  }
+
+  /**
+   * Perform encoding against a block group.
+   * @param blockGroup
+   * @return encoding step for caller to do the real work
+   */
+  protected abstract ErasureCodingStep prepareEncodingStep(
+      ECBlockGroup blockGroup);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java
index 311de21..f0b2be8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java
@@ -27,8 +27,9 @@ import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
  * an encoding step involved in the whole process of encoding a block group.
  */
 @InterfaceAudience.Private
-public class ErasureEncodingStep extends AbstractErasureCodingStep {
-
+public class ErasureEncodingStep implements ErasureCodingStep {
+  private ECBlock[] inputBlocks;
+  private ECBlock[] outputBlocks;
   private RawErasureEncoder rawEncoder;
 
   /**
@@ -39,7 +40,8 @@ public class ErasureEncodingStep extends AbstractErasureCodingStep {
    */
   public ErasureEncodingStep(ECBlock[] inputBlocks, ECBlock[] outputBlocks,
                              RawErasureEncoder rawEncoder) {
-    super(inputBlocks, outputBlocks);
+    this.inputBlocks = inputBlocks;
+    this.outputBlocks = outputBlocks;
     this.rawEncoder = rawEncoder;
   }
 
@@ -48,4 +50,18 @@ public class ErasureEncodingStep extends AbstractErasureCodingStep {
     rawEncoder.encode(inputChunks, outputChunks);
   }
 
+  @Override
+  public ECBlock[] getInputBlocks() {
+    return inputBlocks;
+  }
+
+  @Override
+  public ECBlock[] getOutputBlocks() {
+    return outputBlocks;
+  }
+
+  @Override
+  public void finish() {
+    rawEncoder.release();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHErasureCodingStep.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHErasureCodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHErasureCodingStep.java
new file mode 100644
index 0000000..a0f5b72
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHErasureCodingStep.java
@@ -0,0 +1,68 @@
+/**
+ * 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.erasurecode.coder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.io.erasurecode.ECBlock;
+
+/**
+ * Abstract class for Hitchhiker common facilities shared by
+ * {@link HHXORErasureEncodingStep}and {@link HHXORErasureDecodingStep}.
+ *
+ * It implements {@link ErasureCodingStep}.
+ */
+@InterfaceAudience.Private
+public abstract class HHErasureCodingStep
+        implements ErasureCodingStep {
+
+  private ECBlock[] inputBlocks;
+  private ECBlock[] outputBlocks;
+
+  private static final int SUB_PACKET_SIZE = 2;
+
+  /**
+   * Constructor given input blocks and output blocks.
+   *
+   * @param inputBlocks
+   * @param outputBlocks
+   */
+  public HHErasureCodingStep(ECBlock[] inputBlocks,
+                                     ECBlock[] outputBlocks) {
+    this.inputBlocks = inputBlocks;
+    this.outputBlocks = outputBlocks;
+  }
+
+  protected int getSubPacketSize() {
+    return SUB_PACKET_SIZE;
+  }
+
+  @Override
+  public ECBlock[] getInputBlocks() {
+    return inputBlocks;
+  }
+
+  @Override
+  public ECBlock[] getOutputBlocks() {
+    return outputBlocks;
+  }
+
+  @Override
+  public void finish() {
+    // TODO: Finalize encoder/decoder if necessary
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureDecoder.java
index 94487d8..05e9384 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureDecoder.java
@@ -21,7 +21,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
@@ -39,16 +38,12 @@ import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
  * This is Hitchhiker-XOR erasure decoder that decodes a block group.
  */
 @InterfaceAudience.Private
-public class HHXORErasureDecoder extends AbstractErasureDecoder {
+public class HHXORErasureDecoder extends ErasureDecoder {
   private RawErasureDecoder rsRawDecoder;
   private RawErasureEncoder xorRawEncoder;
 
-  public HHXORErasureDecoder(int numDataUnits, int numParityUnits) {
-    super(numDataUnits, numParityUnits);
-  }
-
-  public HHXORErasureDecoder(ECSchema schema) {
-    super(schema);
+  public HHXORErasureDecoder(ErasureCoderOptions options) {
+    super(options);
   }
 
   @Override
@@ -71,25 +66,26 @@ public class HHXORErasureDecoder extends AbstractErasureDecoder {
 
   private RawErasureDecoder checkCreateRSRawDecoder() {
     if (rsRawDecoder == null) {
-      ErasureCoderOptions coderOptions = new ErasureCoderOptions(
-          getNumDataUnits(), getNumParityUnits());
       rsRawDecoder = CodecUtil.createRawDecoder(getConf(),
-              ErasureCodeConstants.RS_DEFAULT_CODEC_NAME, coderOptions);
+              ErasureCodeConstants.RS_DEFAULT_CODEC_NAME, getOptions());
     }
     return rsRawDecoder;
   }
 
   private RawErasureEncoder checkCreateXorRawEncoder() {
     if (xorRawEncoder == null) {
-      ErasureCoderOptions coderOptions = new ErasureCoderOptions(
-          getNumDataUnits(), getNumParityUnits());
       xorRawEncoder = CodecUtil.createRawEncoder(getConf(),
-          ErasureCodeConstants.XOR_CODEC_NAME, coderOptions);
+          ErasureCodeConstants.XOR_CODEC_NAME, getOptions());
     }
     return xorRawEncoder;
   }
 
   @Override
+  public boolean preferDirectBuffer() {
+    return false;
+  }
+
+  @Override
   public void release() {
     if (rsRawDecoder != null) {
       rsRawDecoder.release();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureDecodingStep.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureDecodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureDecodingStep.java
index 6c81836..98b8503 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureDecodingStep.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureDecodingStep.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
  * decoding a block group.
  */
 @InterfaceAudience.Private
-public class HHXORErasureDecodingStep extends AbstractHHErasureCodingStep {
+public class HHXORErasureDecodingStep extends HHErasureCodingStep {
   private int pbIndex;
   private int[] piggyBackIndex;
   private int[] piggyBackFullIndex;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureEncoder.java
index 219f25c..7a15a05 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureEncoder.java
@@ -21,7 +21,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
@@ -38,16 +37,12 @@ import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
  * This is Hitchhiker-XOR erasure encoder that encodes a block group.
  */
 @InterfaceAudience.Private
-public class HHXORErasureEncoder extends AbstractErasureEncoder {
+public class HHXORErasureEncoder extends ErasureEncoder {
   private RawErasureEncoder rsRawEncoder;
   private RawErasureEncoder xorRawEncoder;
 
-  public HHXORErasureEncoder(int numDataUnits, int numParityUnits) {
-    super(numDataUnits, numParityUnits);
-  }
-
-  public HHXORErasureEncoder(ECSchema schema) {
-    super(schema);
+  public HHXORErasureEncoder(ErasureCoderOptions options) {
+    super(options);
   }
 
   @Override
@@ -65,21 +60,17 @@ public class HHXORErasureEncoder extends AbstractErasureEncoder {
 
   private RawErasureEncoder checkCreateRSRawEncoder() {
     if (rsRawEncoder == null) {
-      ErasureCoderOptions coderOptions = new ErasureCoderOptions(
-          getNumDataUnits(), getNumParityUnits());
       rsRawEncoder = CodecUtil.createRawEncoder(getConf(),
-          ErasureCodeConstants.RS_DEFAULT_CODEC_NAME, coderOptions);
+          ErasureCodeConstants.RS_DEFAULT_CODEC_NAME, getOptions());
     }
     return rsRawEncoder;
   }
 
   private RawErasureEncoder checkCreateXorRawEncoder() {
     if (xorRawEncoder == null) {
-      ErasureCoderOptions erasureCoderOptions = new ErasureCoderOptions(
-          getNumDataUnits(), getNumParityUnits());
       xorRawEncoder = CodecUtil.createRawEncoder(getConf(),
           ErasureCodeConstants.XOR_CODEC_NAME,
-          erasureCoderOptions);
+          getOptions());
     }
     return xorRawEncoder;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureEncodingStep.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureEncodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureEncodingStep.java
index f83ee26..11b1bf1 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureEncodingStep.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureEncodingStep.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
  * encoding a block group.
  */
 @InterfaceAudience.Private
-public class HHXORErasureEncodingStep extends AbstractHHErasureCodingStep {
+public class HHXORErasureEncodingStep extends HHErasureCodingStep {
   private int[] piggyBackIndex;
   private RawErasureEncoder rsRawEncoder;
   private RawErasureEncoder xorRawEncoder;


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


[48/50] [abbrv] hadoop git commit: HADOOP-13560. S3ABlockOutputStream to support huge (many GB) file writes. Contributed by Steve Loughran

Posted by sj...@apache.org.
HADOOP-13560. S3ABlockOutputStream to support huge (many GB) file writes. Contributed by Steve Loughran


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

Branch: refs/heads/HADOOP-13070
Commit: 6c348c56918973fd988b110e79231324a8befe12
Parents: b733a6f
Author: Steve Loughran <st...@apache.org>
Authored: Tue Oct 18 19:33:38 2016 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Oct 18 21:16:02 2016 +0100

----------------------------------------------------------------------
 .../src/main/resources/core-default.xml         |  74 +-
 .../hadoop/fs/contract/ContractTestUtils.java   |  16 +-
 hadoop-tools/hadoop-aws/pom.xml                 |  58 +-
 .../s3a/BlockingThreadPoolExecutorService.java  | 168 +---
 .../org/apache/hadoop/fs/s3a/Constants.java     |  71 +-
 .../hadoop/fs/s3a/S3ABlockOutputStream.java     | 703 ++++++++++++++++
 .../org/apache/hadoop/fs/s3a/S3ADataBlocks.java | 821 +++++++++++++++++++
 .../hadoop/fs/s3a/S3AFastOutputStream.java      | 410 ---------
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java | 408 +++++++--
 .../hadoop/fs/s3a/S3AInstrumentation.java       | 248 +++++-
 .../apache/hadoop/fs/s3a/S3AOutputStream.java   |  57 +-
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java |  39 +
 .../fs/s3a/SemaphoredDelegatingExecutor.java    | 230 ++++++
 .../org/apache/hadoop/fs/s3a/Statistic.java     |  32 +-
 .../src/site/markdown/tools/hadoop-aws/index.md | 668 +++++++++++++--
 .../fs/contract/s3a/ITestS3AContractDistCp.java |  10 +-
 .../hadoop/fs/s3a/AbstractS3ATestBase.java      |   1 +
 .../ITestBlockingThreadPoolExecutorService.java |  48 +-
 .../hadoop/fs/s3a/ITestS3ABlockOutputArray.java |  90 ++
 .../fs/s3a/ITestS3ABlockOutputByteBuffer.java   |  30 +
 .../hadoop/fs/s3a/ITestS3ABlockOutputDisk.java  |  30 +
 .../fs/s3a/ITestS3ABlockingThreadPool.java      |   2 +
 .../hadoop/fs/s3a/ITestS3AConfiguration.java    |  29 +
 .../ITestS3AEncryptionBlockOutputStream.java    |  36 +
 .../s3a/ITestS3AEncryptionFastOutputStream.java |  35 -
 .../hadoop/fs/s3a/ITestS3AFastOutputStream.java |  74 --
 .../apache/hadoop/fs/s3a/ITestS3ATestUtils.java |  98 +++
 .../apache/hadoop/fs/s3a/S3ATestConstants.java  |  75 +-
 .../org/apache/hadoop/fs/s3a/S3ATestUtils.java  | 148 +++-
 .../apache/hadoop/fs/s3a/TestDataBlocks.java    | 124 +++
 .../ITestS3AFileContextStatistics.java          |   1 +
 .../fs/s3a/scale/AbstractSTestS3AHugeFiles.java | 412 ++++++++++
 .../fs/s3a/scale/ITestS3ADeleteManyFiles.java   |  19 +-
 .../s3a/scale/ITestS3AHugeFilesArrayBlocks.java |  31 +
 .../ITestS3AHugeFilesByteBufferBlocks.java      |  34 +
 .../scale/ITestS3AHugeFilesClassicOutput.java   |  41 +
 .../s3a/scale/ITestS3AHugeFilesDiskBlocks.java  |  31 +
 .../hadoop/fs/s3a/scale/S3AScaleTestBase.java   | 151 ++--
 38 files changed, 4647 insertions(+), 906 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 4882728..daa421c 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -994,8 +994,8 @@
 <property>
   <name>fs.s3a.threads.max</name>
   <value>10</value>
-  <description> Maximum number of concurrent active (part)uploads,
-    which each use a thread from the threadpool.</description>
+  <description>The total number of threads available in the filesystem for data
+    uploads *or any other queued filesystem operation*.</description>
 </property>
 
 <property>
@@ -1008,8 +1008,7 @@
 <property>
   <name>fs.s3a.max.total.tasks</name>
   <value>5</value>
-  <description>Number of (part)uploads allowed to the queue before
-    blocking additional uploads.</description>
+  <description>The number of operations which can be queued for execution</description>
 </property>
 
 <property>
@@ -1047,13 +1046,21 @@
   <name>fs.s3a.multipart.purge</name>
   <value>false</value>
   <description>True if you want to purge existing multipart uploads that may not have been
-     completed/aborted correctly</description>
+    completed/aborted correctly. The corresponding purge age is defined in
+    fs.s3a.multipart.purge.age.
+    If set, when the filesystem is instantiated then all outstanding uploads
+    older than the purge age will be terminated -across the entire bucket.
+    This will impact multipart uploads by other applications and users. so should
+    be used sparingly, with an age value chosen to stop failed uploads, without
+    breaking ongoing operations.
+  </description>
 </property>
 
 <property>
   <name>fs.s3a.multipart.purge.age</name>
   <value>86400</value>
-  <description>Minimum age in seconds of multipart uploads to purge</description>
+  <description>Minimum age in seconds of multipart uploads to purge.
+  </description>
 </property>
 
 <property>
@@ -1086,10 +1093,50 @@
 <property>
   <name>fs.s3a.fast.upload</name>
   <value>false</value>
-  <description>Upload directly from memory instead of buffering to
-    disk first. Memory usage and parallelism can be controlled as up to
-    fs.s3a.multipart.size memory is consumed for each (part)upload actively
-    uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description>
+  <description>
+    Use the incremental block-based fast upload mechanism with
+    the buffering mechanism set in fs.s3a.fast.upload.buffer.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.fast.upload.buffer</name>
+  <value>disk</value>
+  <description>
+    The buffering mechanism to use when using S3A fast upload
+    (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer.
+    This configuration option has no effect if fs.s3a.fast.upload is false.
+
+    "disk" will use the directories listed in fs.s3a.buffer.dir as
+    the location(s) to save data prior to being uploaded.
+
+    "array" uses arrays in the JVM heap
+
+    "bytebuffer" uses off-heap memory within the JVM.
+
+    Both "array" and "bytebuffer" will consume memory in a single stream up to the number
+    of blocks set by:
+
+        fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks.
+
+    If using either of these mechanisms, keep this value low
+
+    The total number of threads performing work across all threads is set by
+    fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued
+    work items.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.fast.upload.active.blocks</name>
+  <value>4</value>
+  <description>
+    Maximum Number of blocks a single output stream can have
+    active (uploading, or queued to the central FileSystem
+    instance's pool of queued operations.
+
+    This stops a single stream overloading the shared thread pool.
+  </description>
 </property>
 
 <property>
@@ -1101,13 +1148,6 @@
 </property>
 
 <property>
-  <name>fs.s3a.fast.buffer.size</name>
-  <value>1048576</value>
-  <description>Size of initial memory buffer in bytes allocated for an
-    upload. No effect if fs.s3a.fast.upload is false.</description>
-</property>
-
-<property>
   <name>fs.s3a.user.agent.prefix</name>
   <value></value>
   <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
index 03f47c1..16bfb9a 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
@@ -965,7 +965,7 @@ public class ContractTestUtils extends Assert {
    * @return the number of megabytes/second of the recorded operation
    */
   public static double bandwidthMBs(long bytes, long durationNS) {
-    return (bytes * 1000.0) / durationNS;
+    return bytes / (1024.0 * 1024) * 1.0e9 / durationNS;
   }
 
   /**
@@ -1415,6 +1415,14 @@ public class ContractTestUtils extends Assert {
       return endTime - startTime;
     }
 
+    /**
+     * Intermediate duration of the operation.
+     * @return how much time has passed since the start (in nanos).
+     */
+    public long elapsedTime() {
+      return now() - startTime;
+    }
+
     public double bandwidth(long bytes) {
       return bandwidthMBs(bytes, duration());
     }
@@ -1422,10 +1430,12 @@ public class ContractTestUtils extends Assert {
     /**
      * Bandwidth as bytes per second.
      * @param bytes bytes in
-     * @return the number of bytes per second this operation timed.
+     * @return the number of bytes per second this operation.
+     *         0 if duration == 0.
      */
     public double bandwidthBytes(long bytes) {
-      return (bytes * 1.0) / duration();
+      double duration = duration();
+      return duration > 0 ? bytes / duration : 0;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml
index 1c1bb02..1f9a6ff 100644
--- a/hadoop-tools/hadoop-aws/pom.xml
+++ b/hadoop-tools/hadoop-aws/pom.xml
@@ -35,6 +35,15 @@
     <file.encoding>UTF-8</file.encoding>
     <downloadSources>true</downloadSources>
     <hadoop.tmp.dir>${project.build.directory}/test</hadoop.tmp.dir>
+
+    <!-- are scale tests enabled ? -->
+    <fs.s3a.scale.test.enabled>unset</fs.s3a.scale.test.enabled>
+    <!-- Size in MB of huge files. -->
+    <fs.s3a.scale.test.huge.filesize>unset</fs.s3a.scale.test.huge.filesize>
+    <!-- Size in MB of the partion size in huge file uploads. -->
+    <fs.s3a.scale.test.huge.partitionsize>unset</fs.s3a.scale.test.huge.partitionsize>
+    <!-- Timeout in seconds for scale tests.-->
+    <fs.s3a.scale.test.timeout>3600</fs.s3a.scale.test.timeout>
   </properties>
 
   <profiles>
@@ -115,6 +124,11 @@
                 <!-- substitution.  Putting a prefix in front of it like -->
                 <!-- "fork-" makes it work. -->
                 <test.unique.fork.id>fork-${surefire.forkNumber}</test.unique.fork.id>
+                <!-- Propagate scale parameters -->
+                <fs.s3a.scale.test.enabled>${fs.s3a.scale.test.enabled}</fs.s3a.scale.test.enabled>
+                <fs.s3a.scale.test.huge.filesize>${fs.s3a.scale.test.huge.filesize}</fs.s3a.scale.test.huge.filesize>
+                <fs.s3a.scale.test.huge.huge.partitionsize>${fs.s3a.scale.test.huge.partitionsize}</fs.s3a.scale.test.huge.huge.partitionsize>
+                <fs.s3a.scale.test.timeout>${fs.s3a.scale.test.timeout}</fs.s3a.scale.test.timeout>
               </systemPropertyVariables>
             </configuration>
           </plugin>
@@ -132,7 +146,10 @@
                   <forkCount>${testsThreadCount}</forkCount>
                   <reuseForks>false</reuseForks>
                   <argLine>${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true</argLine>
+                  <forkedProcessTimeoutInSeconds>${fs.s3a.scale.test.timeout}</forkedProcessTimeoutInSeconds>
                   <systemPropertyVariables>
+                    <!-- Tell tests that they are being executed in parallel -->
+                    <test.parallel.execution>true</test.parallel.execution>
                     <test.build.data>${test.build.data}/${surefire.forkNumber}</test.build.data>
                     <test.build.dir>${test.build.dir}/${surefire.forkNumber}</test.build.dir>
                     <hadoop.tmp.dir>${hadoop.tmp.dir}/${surefire.forkNumber}</hadoop.tmp.dir>
@@ -142,6 +159,11 @@
                     <!-- substitution.  Putting a prefix in front of it like -->
                     <!-- "fork-" makes it work. -->
                     <test.unique.fork.id>fork-${surefire.forkNumber}</test.unique.fork.id>
+                    <!-- Propagate scale parameters -->
+                    <fs.s3a.scale.test.enabled>${fs.s3a.scale.test.enabled}</fs.s3a.scale.test.enabled>
+                    <fs.s3a.scale.test.huge.filesize>${fs.s3a.scale.test.huge.filesize}</fs.s3a.scale.test.huge.filesize>
+                    <fs.s3a.scale.test.huge.huge.partitionsize>${fs.s3a.scale.test.huge.partitionsize}</fs.s3a.scale.test.huge.huge.partitionsize>
+                    <fs.s3a.scale.test.timeout>${fs.s3a.scale.test.timeout}</fs.s3a.scale.test.timeout>
                   </systemPropertyVariables>
                   <!-- Some tests cannot run in parallel.  Tests that cover -->
                   <!-- access to the root directory must run in isolation -->
@@ -160,10 +182,11 @@
                   <excludes>
                     <exclude>**/ITestJets3tNativeS3FileSystemContract.java</exclude>
                     <exclude>**/ITestS3ABlockingThreadPool.java</exclude>
-                    <exclude>**/ITestS3AFastOutputStream.java</exclude>
                     <exclude>**/ITestS3AFileSystemContract.java</exclude>
                     <exclude>**/ITestS3AMiniYarnCluster.java</exclude>
                     <exclude>**/ITest*Root*.java</exclude>
+                    <exclude>**/ITestS3AFileContextStatistics.java</exclude>
+                    <include>**/ITestS3AHuge*.java</include>
                   </excludes>
                 </configuration>
               </execution>
@@ -174,6 +197,16 @@
                   <goal>verify</goal>
                 </goals>
                 <configuration>
+                  <forkedProcessTimeoutInSeconds>${fs.s3a.scale.test.timeout}</forkedProcessTimeoutInSeconds>
+                  <systemPropertyVariables>
+                    <!-- Tell tests that they are being executed sequentially -->
+                    <test.parallel.execution>false</test.parallel.execution>
+                    <!-- Propagate scale parameters -->
+                    <fs.s3a.scale.test.enabled>${fs.s3a.scale.test.enabled}</fs.s3a.scale.test.enabled>
+                    <fs.s3a.scale.test.huge.filesize>${fs.s3a.scale.test.huge.filesize}</fs.s3a.scale.test.huge.filesize>
+                    <fs.s3a.scale.test.huge.huge.partitionsize>${fs.s3a.scale.test.huge.partitionsize}</fs.s3a.scale.test.huge.huge.partitionsize>
+                    <fs.s3a.scale.test.timeout>${fs.s3a.scale.test.timeout}</fs.s3a.scale.test.timeout>
+                  </systemPropertyVariables>
                   <!-- Do a sequential run for tests that cannot handle -->
                   <!-- parallel execution. -->
                   <includes>
@@ -183,6 +216,8 @@
                     <include>**/ITestS3AFileSystemContract.java</include>
                     <include>**/ITestS3AMiniYarnCluster.java</include>
                     <include>**/ITest*Root*.java</include>
+                    <include>**/ITestS3AFileContextStatistics.java</include>
+                    <include>**/ITestS3AHuge*.java</include>
                   </includes>
                 </configuration>
               </execution>
@@ -210,7 +245,13 @@
                   <goal>verify</goal>
                 </goals>
                 <configuration>
-                  <forkedProcessTimeoutInSeconds>3600</forkedProcessTimeoutInSeconds>
+                  <systemPropertyVariables>
+                    <!-- Propagate scale parameters -->
+                    <fs.s3a.scale.test.enabled>${fs.s3a.scale.test.enabled}</fs.s3a.scale.test.enabled>
+                    <fs.s3a.scale.test.huge.filesize>${fs.s3a.scale.test.huge.filesize}</fs.s3a.scale.test.huge.filesize>
+                    <fs.s3a.scale.test.timeout>${fs.s3a.scale.test.timeout}</fs.s3a.scale.test.timeout>
+                  </systemPropertyVariables>
+                  <forkedProcessTimeoutInSeconds>${fs.s3a.scale.test.timeout}</forkedProcessTimeoutInSeconds>
                 </configuration>
               </execution>
             </executions>
@@ -218,6 +259,19 @@
         </plugins>
       </build>
     </profile>
+
+    <!-- Turn on scale tests-->
+    <profile>
+      <id>scale</id>
+      <activation>
+        <property>
+          <name>scale</name>
+        </property>
+      </activation>
+      <properties >
+        <fs.s3a.scale.test.enabled>true</fs.s3a.scale.test.enabled>
+      </properties>
+    </profile>
   </profiles>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BlockingThreadPoolExecutorService.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BlockingThreadPoolExecutorService.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BlockingThreadPoolExecutorService.java
index 597cce6..5ff96a5 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BlockingThreadPoolExecutorService.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BlockingThreadPoolExecutorService.java
@@ -18,30 +18,21 @@
 
 package org.apache.hadoop.fs.s3a;
 
-import java.util.Collection;
-import java.util.List;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.RejectedExecutionHandler;
-import java.util.concurrent.Semaphore;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.util.concurrent.ForwardingListeningExecutorService;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+
 /**
  * This ExecutorService blocks the submission of new tasks when its queue is
  * already full by using a semaphore. Task submissions require permits, task
@@ -50,17 +41,17 @@ import com.google.common.util.concurrent.MoreExecutors;
  * This is inspired by <a href="https://github.com/apache/incubator-s4/blob/master/subprojects/s4-comm/src/main/java/org/apache/s4/comm/staging/BlockingThreadPoolExecutorService.java">
  * this s4 threadpool</a>
  */
-public class BlockingThreadPoolExecutorService
-    extends ForwardingListeningExecutorService {
+@InterfaceAudience.Private
+final class BlockingThreadPoolExecutorService
+    extends SemaphoredDelegatingExecutor {
 
   private static Logger LOG = LoggerFactory
       .getLogger(BlockingThreadPoolExecutorService.class);
 
-  private Semaphore queueingPermits;
-  private ListeningExecutorService executorDelegatee;
-
   private static final AtomicInteger POOLNUMBER = new AtomicInteger(1);
 
+  private final ThreadPoolExecutor eventProcessingExecutor;
+
   /**
    * Returns a {@link java.util.concurrent.ThreadFactory} that names each
    * created thread uniquely,
@@ -69,7 +60,7 @@ public class BlockingThreadPoolExecutorService
    * @param prefix The prefix of every created Thread's name
    * @return a {@link java.util.concurrent.ThreadFactory} that names threads
    */
-  public static ThreadFactory getNamedThreadFactory(final String prefix) {
+  static ThreadFactory getNamedThreadFactory(final String prefix) {
     SecurityManager s = System.getSecurityManager();
     final ThreadGroup threadGroup = (s != null) ? s.getThreadGroup() :
         Thread.currentThread().getThreadGroup();
@@ -113,6 +104,12 @@ public class BlockingThreadPoolExecutorService
     };
   }
 
+  private BlockingThreadPoolExecutorService(int permitCount,
+      ThreadPoolExecutor eventProcessingExecutor) {
+    super(MoreExecutors.listeningDecorator(eventProcessingExecutor),
+        permitCount, false);
+    this.eventProcessingExecutor = eventProcessingExecutor;
+  }
 
   /**
    * A thread pool that that blocks clients submitting additional tasks if
@@ -125,10 +122,12 @@ public class BlockingThreadPoolExecutorService
    * @param unit time unit
    * @param prefixName prefix of name for threads
    */
-  public BlockingThreadPoolExecutorService(int activeTasks, int waitingTasks,
-      long keepAliveTime, TimeUnit unit, String prefixName) {
-    super();
-    queueingPermits = new Semaphore(waitingTasks + activeTasks, false);
+  public static BlockingThreadPoolExecutorService newInstance(
+      int activeTasks,
+      int waitingTasks,
+      long keepAliveTime, TimeUnit unit,
+      String prefixName) {
+
     /* Although we generally only expect up to waitingTasks tasks in the
     queue, we need to be able to buffer all tasks in case dequeueing is
     slower than enqueueing. */
@@ -147,126 +146,25 @@ public class BlockingThreadPoolExecutorService
               }
             });
     eventProcessingExecutor.allowCoreThreadTimeOut(true);
-    executorDelegatee =
-        MoreExecutors.listeningDecorator(eventProcessingExecutor);
-
-  }
-
-  @Override
-  protected ListeningExecutorService delegate() {
-    return executorDelegatee;
-  }
-
-  @Override
-  public <T> ListenableFuture<T> submit(Callable<T> task) {
-    try {
-      queueingPermits.acquire();
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      return Futures.immediateFailedCheckedFuture(e);
-    }
-    return super.submit(new CallableWithPermitRelease<T>(task));
-  }
-
-  @Override
-  public <T> ListenableFuture<T> submit(Runnable task, T result) {
-    try {
-      queueingPermits.acquire();
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      return Futures.immediateFailedCheckedFuture(e);
-    }
-    return super.submit(new RunnableWithPermitRelease(task), result);
-  }
-
-  @Override
-  public ListenableFuture<?> submit(Runnable task) {
-    try {
-      queueingPermits.acquire();
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      return Futures.immediateFailedCheckedFuture(e);
-    }
-    return super.submit(new RunnableWithPermitRelease(task));
-  }
-
-  @Override
-  public void execute(Runnable command) {
-    try {
-      queueingPermits.acquire();
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-    }
-    super.execute(new RunnableWithPermitRelease(command));
+    return new BlockingThreadPoolExecutorService(waitingTasks + activeTasks,
+        eventProcessingExecutor);
   }
 
   /**
-   * Releases a permit after the task is executed.
+   * Get the actual number of active threads.
+   * @return the active thread count
    */
-  class RunnableWithPermitRelease implements Runnable {
-
-    private Runnable delegatee;
-
-    public RunnableWithPermitRelease(Runnable delegatee) {
-      this.delegatee = delegatee;
-    }
-
-    @Override
-    public void run() {
-      try {
-        delegatee.run();
-      } finally {
-        queueingPermits.release();
-      }
-
-    }
-  }
-
-  /**
-   * Releases a permit after the task is completed.
-   */
-  class CallableWithPermitRelease<T> implements Callable<T> {
-
-    private Callable<T> delegatee;
-
-    public CallableWithPermitRelease(Callable<T> delegatee) {
-      this.delegatee = delegatee;
-    }
-
-    @Override
-    public T call() throws Exception {
-      try {
-        return delegatee.call();
-      } finally {
-        queueingPermits.release();
-      }
-    }
-
-  }
-
-  @Override
-  public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
-      throws InterruptedException {
-    throw new RuntimeException("Not implemented");
+  int getActiveCount() {
+    return eventProcessingExecutor.getActiveCount();
   }
 
   @Override
-  public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks,
-      long timeout, TimeUnit unit) throws InterruptedException {
-    throw new RuntimeException("Not implemented");
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "BlockingThreadPoolExecutorService{");
+    sb.append(super.toString());
+    sb.append(", activeCount=").append(getActiveCount());
+    sb.append('}');
+    return sb.toString();
   }
-
-  @Override
-  public <T> T invokeAny(Collection<? extends Callable<T>> tasks)
-      throws InterruptedException, ExecutionException {
-    throw new RuntimeException("Not implemented");
-  }
-
-  @Override
-  public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout,
-      TimeUnit unit)
-      throws InterruptedException, ExecutionException, TimeoutException {
-    throw new RuntimeException("Not implemented");
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
index 64fd8e5..65df0bf 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
@@ -35,6 +35,9 @@ public final class Constants {
   private Constants() {
   }
 
+  /** The minimum multipart size which S3 supports. */
+  public static final int MULTIPART_MIN_SIZE = 5 * 1024 * 1024;
+
   // s3 access key
   public static final String ACCESS_KEY = "fs.s3a.access.key";
 
@@ -124,14 +127,72 @@ public final class Constants {
   // comma separated list of directories
   public static final String BUFFER_DIR = "fs.s3a.buffer.dir";
 
-  // should we upload directly from memory rather than using a file buffer
+  // switch to the fast block-by-block upload mechanism
   public static final String FAST_UPLOAD = "fs.s3a.fast.upload";
   public static final boolean DEFAULT_FAST_UPLOAD = false;
 
   //initial size of memory buffer for a fast upload
+  @Deprecated
   public static final String FAST_BUFFER_SIZE = "fs.s3a.fast.buffer.size";
   public static final int DEFAULT_FAST_BUFFER_SIZE = 1048576; //1MB
 
+  /**
+   * What buffer to use.
+   * Default is {@link #FAST_UPLOAD_BUFFER_DISK}
+   * Value: {@value}
+   */
+  @InterfaceStability.Unstable
+  public static final String FAST_UPLOAD_BUFFER =
+      "fs.s3a.fast.upload.buffer";
+
+  /**
+   * Buffer blocks to disk: {@value}.
+   * Capacity is limited to available disk space.
+   */
+
+  @InterfaceStability.Unstable
+  public static final String FAST_UPLOAD_BUFFER_DISK = "disk";
+
+  /**
+   * Use an in-memory array. Fast but will run of heap rapidly: {@value}.
+   */
+  @InterfaceStability.Unstable
+  public static final String FAST_UPLOAD_BUFFER_ARRAY = "array";
+
+  /**
+   * Use a byte buffer. May be more memory efficient than the
+   * {@link #FAST_UPLOAD_BUFFER_ARRAY}: {@value}.
+   */
+  @InterfaceStability.Unstable
+  public static final String FAST_UPLOAD_BYTEBUFFER = "bytebuffer";
+
+  /**
+   * Default buffer option: {@value}.
+   */
+  @InterfaceStability.Unstable
+  public static final String DEFAULT_FAST_UPLOAD_BUFFER =
+      FAST_UPLOAD_BUFFER_DISK;
+
+  /**
+   * Maximum Number of blocks a single output stream can have
+   * active (uploading, or queued to the central FileSystem
+   * instance's pool of queued operations.
+   * This stops a single stream overloading the shared thread pool.
+   * {@value}
+   * <p>
+   * Default is {@link #DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS}
+   */
+  @InterfaceStability.Unstable
+  public static final String FAST_UPLOAD_ACTIVE_BLOCKS =
+      "fs.s3a.fast.upload.active.blocks";
+
+  /**
+   * Limit of queued block upload operations before writes
+   * block. Value: {@value}
+   */
+  @InterfaceStability.Unstable
+  public static final int DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS = 4;
+
   // Private | PublicRead | PublicReadWrite | AuthenticatedRead |
   // LogDeliveryWrite | BucketOwnerRead | BucketOwnerFullControl
   public static final String CANNED_ACL = "fs.s3a.acl.default";
@@ -145,7 +206,7 @@ public final class Constants {
   // purge any multipart uploads older than this number of seconds
   public static final String PURGE_EXISTING_MULTIPART_AGE =
       "fs.s3a.multipart.purge.age";
-  public static final long DEFAULT_PURGE_EXISTING_MULTIPART_AGE = 14400;
+  public static final long DEFAULT_PURGE_EXISTING_MULTIPART_AGE = 86400;
 
   // s3 server-side encryption
   public static final String SERVER_SIDE_ENCRYPTION_ALGORITHM =
@@ -215,4 +276,10 @@ public final class Constants {
   public static final Class<? extends S3ClientFactory>
       DEFAULT_S3_CLIENT_FACTORY_IMPL =
           S3ClientFactory.DefaultS3ClientFactory.class;
+
+  /**
+   * Maximum number of partitions in a multipart upload: {@value}.
+   */
+  @InterfaceAudience.Private
+  public static final int MAX_MULTIPART_COUNT = 10000;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
new file mode 100644
index 0000000..b66a23f
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
@@ -0,0 +1,703 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import com.amazonaws.AmazonClientException;
+import com.amazonaws.event.ProgressEvent;
+import com.amazonaws.event.ProgressEventType;
+import com.amazonaws.event.ProgressListener;
+import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
+import com.amazonaws.services.s3.model.PartETag;
+import com.amazonaws.services.s3.model.PutObjectRequest;
+import com.amazonaws.services.s3.model.PutObjectResult;
+import com.amazonaws.services.s3.model.UploadPartRequest;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.util.Progressable;
+
+import static org.apache.hadoop.fs.s3a.S3AUtils.*;
+import static org.apache.hadoop.fs.s3a.Statistic.*;
+
+/**
+ * Upload files/parts directly via different buffering mechanisms:
+ * including memory and disk.
+ *
+ * If the stream is closed and no update has started, then the upload
+ * is instead done as a single PUT operation.
+ *
+ * Unstable: statistics and error handling might evolve.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+class S3ABlockOutputStream extends OutputStream {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(S3ABlockOutputStream.class);
+
+  /** Owner FileSystem. */
+  private final S3AFileSystem fs;
+
+  /** Object being uploaded. */
+  private final String key;
+
+  /** Size of all blocks. */
+  private final int blockSize;
+
+  /** Callback for progress. */
+  private final ProgressListener progressListener;
+  private final ListeningExecutorService executorService;
+
+  /**
+   * Retry policy for multipart commits; not all AWS SDK versions retry that.
+   */
+  private final RetryPolicy retryPolicy =
+      RetryPolicies.retryUpToMaximumCountWithProportionalSleep(
+          5,
+          2000,
+          TimeUnit.MILLISECONDS);
+  /**
+   * Factory for blocks.
+   */
+  private final S3ADataBlocks.BlockFactory blockFactory;
+
+  /** Preallocated byte buffer for writing single characters. */
+  private final byte[] singleCharWrite = new byte[1];
+
+  /** Multipart upload details; null means none started. */
+  private MultiPartUpload multiPartUpload;
+
+  /** Closed flag. */
+  private final AtomicBoolean closed = new AtomicBoolean(false);
+
+  /** Current data block. Null means none currently active */
+  private S3ADataBlocks.DataBlock activeBlock;
+
+  /** Count of blocks uploaded. */
+  private long blockCount = 0;
+
+  /** Statistics to build up. */
+  private final S3AInstrumentation.OutputStreamStatistics statistics;
+
+  /**
+   * Write operation helper; encapsulation of the filesystem operations.
+   */
+  private final S3AFileSystem.WriteOperationHelper writeOperationHelper;
+
+  /**
+   * An S3A output stream which uploads partitions in a separate pool of
+   * threads; different {@link S3ADataBlocks.BlockFactory}
+   * instances can control where data is buffered.
+   *
+   * @param fs S3AFilesystem
+   * @param key S3 object to work on.
+   * @param executorService the executor service to use to schedule work
+   * @param progress report progress in order to prevent timeouts. If
+   * this object implements {@code ProgressListener} then it will be
+   * directly wired up to the AWS client, so receive detailed progress
+   * information.
+   * @param blockSize size of a single block.
+   * @param blockFactory factory for creating stream destinations
+   * @param statistics stats for this stream
+   * @param writeOperationHelper state of the write operation.
+   * @throws IOException on any problem
+   */
+  S3ABlockOutputStream(S3AFileSystem fs,
+      String key,
+      ExecutorService executorService,
+      Progressable progress,
+      long blockSize,
+      S3ADataBlocks.BlockFactory blockFactory,
+      S3AInstrumentation.OutputStreamStatistics statistics,
+      S3AFileSystem.WriteOperationHelper writeOperationHelper)
+      throws IOException {
+    this.fs = fs;
+    this.key = key;
+    this.blockFactory = blockFactory;
+    this.blockSize = (int) blockSize;
+    this.statistics = statistics;
+    this.writeOperationHelper = writeOperationHelper;
+    Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE,
+        "Block size is too small: %d", blockSize);
+    this.executorService = MoreExecutors.listeningDecorator(executorService);
+    this.multiPartUpload = null;
+    this.progressListener = (progress instanceof ProgressListener) ?
+        (ProgressListener) progress
+        : new ProgressableListener(progress);
+    // create that first block. This guarantees that an open + close sequence
+    // writes a 0-byte entry.
+    createBlockIfNeeded();
+    LOG.debug("Initialized S3ABlockOutputStream for {}" +
+        " output to {}", writeOperationHelper, activeBlock);
+  }
+
+  /**
+   * Demand create a destination block.
+   * @return the active block; null if there isn't one.
+   * @throws IOException on any failure to create
+   */
+  private synchronized S3ADataBlocks.DataBlock createBlockIfNeeded()
+      throws IOException {
+    if (activeBlock == null) {
+      blockCount++;
+      if (blockCount>= Constants.MAX_MULTIPART_COUNT) {
+        LOG.error("Number of partitions in stream exceeds limit for S3: " +
+             + Constants.MAX_MULTIPART_COUNT +  " write may fail.");
+      }
+      activeBlock = blockFactory.create(this.blockSize);
+    }
+    return activeBlock;
+  }
+
+  /**
+   * Synchronized accessor to the active block.
+   * @return the active block; null if there isn't one.
+   */
+  private synchronized S3ADataBlocks.DataBlock getActiveBlock() {
+    return activeBlock;
+  }
+
+  /**
+   * Predicate to query whether or not there is an active block.
+   * @return true if there is an active block.
+   */
+  private synchronized boolean hasActiveBlock() {
+    return activeBlock != null;
+  }
+
+  /**
+   * Clear the active block.
+   */
+  private void clearActiveBlock() {
+    LOG.debug("Clearing active block");
+    synchronized (this) {
+      activeBlock = null;
+    }
+  }
+
+  /**
+   * Check for the filesystem being open.
+   * @throws IOException if the filesystem is closed.
+   */
+  void checkOpen() throws IOException {
+    if (closed.get()) {
+      throw new IOException("Filesystem " + writeOperationHelper + " closed");
+    }
+  }
+
+  /**
+   * The flush operation does not trigger an upload; that awaits
+   * the next block being full. What it does do is call {@code flush() }
+   * on the current block, leaving it to choose how to react.
+   * @throws IOException Any IO problem.
+   */
+  @Override
+  public synchronized void flush() throws IOException {
+    checkOpen();
+    S3ADataBlocks.DataBlock dataBlock = getActiveBlock();
+    if (dataBlock != null) {
+      dataBlock.flush();
+    }
+  }
+
+  /**
+   * Writes a byte to the destination. If this causes the buffer to reach
+   * its limit, the actual upload is submitted to the threadpool.
+   * @param b the int of which the lowest byte is written
+   * @throws IOException on any problem
+   */
+  @Override
+  public synchronized void write(int b) throws IOException {
+    singleCharWrite[0] = (byte)b;
+    write(singleCharWrite, 0, 1);
+  }
+
+  /**
+   * Writes a range of bytes from to the memory buffer. If this causes the
+   * buffer to reach its limit, the actual upload is submitted to the
+   * threadpool and the remainder of the array is written to memory
+   * (recursively).
+   * @param source byte array containing
+   * @param offset offset in array where to start
+   * @param len number of bytes to be written
+   * @throws IOException on any problem
+   */
+  @Override
+  public synchronized void write(byte[] source, int offset, int len)
+      throws IOException {
+
+    S3ADataBlocks.validateWriteArgs(source, offset, len);
+    checkOpen();
+    if (len == 0) {
+      return;
+    }
+    S3ADataBlocks.DataBlock block = createBlockIfNeeded();
+    int written = block.write(source, offset, len);
+    int remainingCapacity = block.remainingCapacity();
+    if (written < len) {
+      // not everything was written \u2014the block has run out
+      // of capacity
+      // Trigger an upload then process the remainder.
+      LOG.debug("writing more data than block has capacity -triggering upload");
+      uploadCurrentBlock();
+      // tail recursion is mildly expensive, but given buffer sizes must be MB.
+      // it's unlikely to recurse very deeply.
+      this.write(source, offset + written, len - written);
+    } else {
+      if (remainingCapacity == 0) {
+        // the whole buffer is done, trigger an upload
+        uploadCurrentBlock();
+      }
+    }
+  }
+
+  /**
+   * Start an asynchronous upload of the current block.
+   * @throws IOException Problems opening the destination for upload
+   * or initializing the upload.
+   */
+  private synchronized void uploadCurrentBlock() throws IOException {
+    Preconditions.checkState(hasActiveBlock(), "No active block");
+    LOG.debug("Writing block # {}", blockCount);
+    if (multiPartUpload == null) {
+      LOG.debug("Initiating Multipart upload");
+      multiPartUpload = new MultiPartUpload();
+    }
+    try {
+      multiPartUpload.uploadBlockAsync(getActiveBlock());
+    } finally {
+      // set the block to null, so the next write will create a new block.
+      clearActiveBlock();
+    }
+  }
+
+  /**
+   * Close the stream.
+   *
+   * This will not return until the upload is complete
+   * or the attempt to perform the upload has failed.
+   * Exceptions raised in this method are indicative that the write has
+   * failed and data is at risk of being lost.
+   * @throws IOException on any failure.
+   */
+  @Override
+  public void close() throws IOException {
+    if (closed.getAndSet(true)) {
+      // already closed
+      LOG.debug("Ignoring close() as stream is already closed");
+      return;
+    }
+    S3ADataBlocks.DataBlock block = getActiveBlock();
+    boolean hasBlock = hasActiveBlock();
+    LOG.debug("{}: Closing block #{}: current block= {}",
+        this,
+        blockCount,
+        hasBlock ? block : "(none)");
+    try {
+      if (multiPartUpload == null) {
+        if (hasBlock) {
+          // no uploads of data have taken place, put the single block up.
+          // This must happen even if there is no data, so that 0 byte files
+          // are created.
+          putObject();
+        }
+      } else {
+        // there has already been at least one block scheduled for upload;
+        // put up the current then wait
+        if (hasBlock && block.hasData()) {
+          //send last part
+          uploadCurrentBlock();
+        }
+        // wait for the partial uploads to finish
+        final List<PartETag> partETags =
+            multiPartUpload.waitForAllPartUploads();
+        // then complete the operation
+        multiPartUpload.complete(partETags);
+      }
+      LOG.debug("Upload complete for {}", writeOperationHelper);
+    } catch (IOException ioe) {
+      writeOperationHelper.writeFailed(ioe);
+      throw ioe;
+    } finally {
+      LOG.debug("Closing block and factory");
+      IOUtils.closeStream(block);
+      IOUtils.closeStream(blockFactory);
+      LOG.debug("Statistics: {}", statistics);
+      IOUtils.closeStream(statistics);
+      clearActiveBlock();
+    }
+    // All end of write operations, including deleting fake parent directories
+    writeOperationHelper.writeSuccessful();
+  }
+
+  /**
+   * Upload the current block as a single PUT request; if the buffer
+   * is empty a 0-byte PUT will be invoked, as it is needed to create an
+   * entry at the far end.
+   * @throws IOException any problem.
+   */
+  private void putObject() throws IOException {
+    LOG.debug("Executing regular upload for {}", writeOperationHelper);
+
+    final S3ADataBlocks.DataBlock block = getActiveBlock();
+    int size = block.dataSize();
+    final PutObjectRequest putObjectRequest =
+        writeOperationHelper.newPutRequest(
+            block.startUpload(),
+            size);
+    long transferQueueTime = now();
+    BlockUploadProgress callback =
+        new BlockUploadProgress(
+            block, progressListener, transferQueueTime);
+    putObjectRequest.setGeneralProgressListener(callback);
+    statistics.blockUploadQueued(size);
+    ListenableFuture<PutObjectResult> putObjectResult =
+        executorService.submit(new Callable<PutObjectResult>() {
+          @Override
+          public PutObjectResult call() throws Exception {
+            PutObjectResult result = fs.putObjectDirect(putObjectRequest);
+            block.close();
+            return result;
+          }
+        });
+    clearActiveBlock();
+    //wait for completion
+    try {
+      putObjectResult.get();
+    } catch (InterruptedException ie) {
+      LOG.warn("Interrupted object upload", ie);
+      Thread.currentThread().interrupt();
+    } catch (ExecutionException ee) {
+      throw extractException("regular upload", key, ee);
+    }
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "S3ABlockOutputStream{");
+    sb.append(writeOperationHelper.toString());
+    sb.append(", blockSize=").append(blockSize);
+    // unsynced access; risks consistency in exchange for no risk of deadlock.
+    S3ADataBlocks.DataBlock block = activeBlock;
+    if (block != null) {
+      sb.append(", activeBlock=").append(block);
+    }
+    sb.append('}');
+    return sb.toString();
+  }
+
+  private void incrementWriteOperations() {
+    fs.incrementWriteOperations();
+  }
+
+  /**
+   * Current time in milliseconds.
+   * @return time
+   */
+  private long now() {
+    return System.currentTimeMillis();
+  }
+
+  /**
+   * Multiple partition upload.
+   */
+  private class MultiPartUpload {
+    private final String uploadId;
+    private final List<ListenableFuture<PartETag>> partETagsFutures;
+
+    public MultiPartUpload() throws IOException {
+      this.uploadId = writeOperationHelper.initiateMultiPartUpload();
+      this.partETagsFutures = new ArrayList<>(2);
+      LOG.debug("Initiated multi-part upload for {} with " +
+          "id '{}'", writeOperationHelper, uploadId);
+    }
+
+    /**
+     * Upload a block of data.
+     * This will take the block
+     * @param block block to upload
+     * @throws IOException upload failure
+     */
+    private void uploadBlockAsync(final S3ADataBlocks.DataBlock block)
+        throws IOException {
+      LOG.debug("Queueing upload of {}", block);
+      final int size = block.dataSize();
+      final InputStream uploadStream = block.startUpload();
+      final int currentPartNumber = partETagsFutures.size() + 1;
+      final UploadPartRequest request =
+          writeOperationHelper.newUploadPartRequest(
+              uploadId,
+              uploadStream,
+              currentPartNumber,
+              size);
+      long transferQueueTime = now();
+      BlockUploadProgress callback =
+          new BlockUploadProgress(
+              block, progressListener, transferQueueTime);
+      request.setGeneralProgressListener(callback);
+      statistics.blockUploadQueued(block.dataSize());
+      ListenableFuture<PartETag> partETagFuture =
+          executorService.submit(new Callable<PartETag>() {
+            @Override
+            public PartETag call() throws Exception {
+              // this is the queued upload operation
+              LOG.debug("Uploading part {} for id '{}'", currentPartNumber,
+                  uploadId);
+              // do the upload
+              PartETag partETag = fs.uploadPart(request).getPartETag();
+              LOG.debug("Completed upload of {}", block);
+              LOG.debug("Stream statistics of {}", statistics);
+
+              // close the block
+              block.close();
+              return partETag;
+            }
+          });
+      partETagsFutures.add(partETagFuture);
+    }
+
+    /**
+     * Block awaiting all outstanding uploads to complete.
+     * @return list of results
+     * @throws IOException IO Problems
+     */
+    private List<PartETag> waitForAllPartUploads() throws IOException {
+      LOG.debug("Waiting for {} uploads to complete", partETagsFutures.size());
+      try {
+        return Futures.allAsList(partETagsFutures).get();
+      } catch (InterruptedException ie) {
+        LOG.warn("Interrupted partUpload", ie);
+        Thread.currentThread().interrupt();
+        return null;
+      } catch (ExecutionException ee) {
+        //there is no way of recovering so abort
+        //cancel all partUploads
+        LOG.debug("While waiting for upload completion", ee);
+        LOG.debug("Cancelling futures");
+        for (ListenableFuture<PartETag> future : partETagsFutures) {
+          future.cancel(true);
+        }
+        //abort multipartupload
+        this.abort();
+        throw extractException("Multi-part upload with id '" + uploadId
+                + "' to " + key, key, ee);
+      }
+    }
+
+    /**
+     * This completes a multipart upload.
+     * Sometimes it fails; here retries are handled to avoid losing all data
+     * on a transient failure.
+     * @param partETags list of partial uploads
+     * @throws IOException on any problem
+     */
+    private CompleteMultipartUploadResult complete(List<PartETag> partETags)
+        throws IOException {
+      int retryCount = 0;
+      AmazonClientException lastException;
+      String operation =
+          String.format("Completing multi-part upload for key '%s'," +
+                  " id '%s' with %s partitions ",
+              key, uploadId, partETags.size());
+      do {
+        try {
+          LOG.debug(operation);
+          return writeOperationHelper.completeMultipartUpload(
+                  uploadId,
+                  partETags);
+        } catch (AmazonClientException e) {
+          lastException = e;
+          statistics.exceptionInMultipartComplete();
+        }
+      } while (shouldRetry(operation, lastException, retryCount++));
+      // this point is only reached if the operation failed more than
+      // the allowed retry count
+      throw translateException(operation, key, lastException);
+    }
+
+    /**
+     * Abort a multi-part upload. Retries are attempted on failures.
+     * IOExceptions are caught; this is expected to be run as a cleanup process.
+     */
+    public void abort() {
+      int retryCount = 0;
+      AmazonClientException lastException;
+      fs.incrementStatistic(OBJECT_MULTIPART_UPLOAD_ABORTED);
+      String operation =
+          String.format("Aborting multi-part upload for '%s', id '%s",
+              writeOperationHelper, uploadId);
+      do {
+        try {
+          LOG.debug(operation);
+          writeOperationHelper.abortMultipartUpload(uploadId);
+          return;
+        } catch (AmazonClientException e) {
+          lastException = e;
+          statistics.exceptionInMultipartAbort();
+        }
+      } while (shouldRetry(operation, lastException, retryCount++));
+      // this point is only reached if the operation failed more than
+      // the allowed retry count
+      LOG.warn("Unable to abort multipart upload, you may need to purge  " +
+          "uploaded parts", lastException);
+    }
+
+    /**
+     * Predicate to determine whether a failed operation should
+     * be attempted again.
+     * If a retry is advised, the exception is automatically logged and
+     * the filesystem statistic {@link Statistic#IGNORED_ERRORS} incremented.
+     * The method then sleeps for the sleep time suggested by the sleep policy;
+     * if the sleep is interrupted then {@code Thread.interrupted()} is set
+     * to indicate the thread was interrupted; then false is returned.
+     *
+     * @param operation operation for log message
+     * @param e exception raised.
+     * @param retryCount  number of retries already attempted
+     * @return true if another attempt should be made
+     */
+    private boolean shouldRetry(String operation,
+        AmazonClientException e,
+        int retryCount) {
+      try {
+        RetryPolicy.RetryAction retryAction =
+            retryPolicy.shouldRetry(e, retryCount, 0, true);
+        boolean retry = retryAction == RetryPolicy.RetryAction.RETRY;
+        if (retry) {
+          fs.incrementStatistic(IGNORED_ERRORS);
+          LOG.info("Retrying {} after exception ", operation, e);
+          Thread.sleep(retryAction.delayMillis);
+        }
+        return retry;
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        return false;
+      } catch (Exception ignored) {
+        return false;
+      }
+    }
+
+  }
+
+  /**
+   * The upload progress listener registered for events returned
+   * during the upload of a single block.
+   * It updates statistics and handles the end of the upload.
+   * Transfer failures are logged at WARN.
+   */
+  private final class BlockUploadProgress implements ProgressListener {
+    private final S3ADataBlocks.DataBlock block;
+    private final ProgressListener nextListener;
+    private final long transferQueueTime;
+    private long transferStartTime;
+
+    /**
+     * Track the progress of a single block upload.
+     * @param block block to monitor
+     * @param nextListener optional next progress listener
+     * @param transferQueueTime time the block was transferred
+     * into the queue
+     */
+    private BlockUploadProgress(S3ADataBlocks.DataBlock block,
+        ProgressListener nextListener,
+        long transferQueueTime) {
+      this.block = block;
+      this.transferQueueTime = transferQueueTime;
+      this.nextListener = nextListener;
+    }
+
+    @Override
+    public void progressChanged(ProgressEvent progressEvent) {
+      ProgressEventType eventType = progressEvent.getEventType();
+      long bytesTransferred = progressEvent.getBytesTransferred();
+
+      int size = block.dataSize();
+      switch (eventType) {
+
+      case REQUEST_BYTE_TRANSFER_EVENT:
+        // bytes uploaded
+        statistics.bytesTransferred(bytesTransferred);
+        break;
+
+      case TRANSFER_PART_STARTED_EVENT:
+        transferStartTime = now();
+        statistics.blockUploadStarted(transferStartTime - transferQueueTime,
+            size);
+        incrementWriteOperations();
+        break;
+
+      case TRANSFER_PART_COMPLETED_EVENT:
+        statistics.blockUploadCompleted(now() - transferStartTime, size);
+        break;
+
+      case TRANSFER_PART_FAILED_EVENT:
+        statistics.blockUploadFailed(now() - transferStartTime, size);
+        LOG.warn("Transfer failure of block {}", block);
+        break;
+
+      default:
+        // nothing
+      }
+
+      if (nextListener != null) {
+        nextListener.progressChanged(progressEvent);
+      }
+    }
+  }
+
+  /**
+   * Bridge from AWS {@code ProgressListener} to Hadoop {@link Progressable}.
+   */
+  private static class ProgressableListener implements ProgressListener {
+    private final Progressable progress;
+
+    public ProgressableListener(Progressable progress) {
+      this.progress = progress;
+    }
+
+    public void progressChanged(ProgressEvent progressEvent) {
+      if (progress != null) {
+        progress.progress();
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java
new file mode 100644
index 0000000..0fe2af7
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java
@@ -0,0 +1,821 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.io.BufferedOutputStream;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.Closeable;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.util.DirectBufferPool;
+
+import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*;
+
+/**
+ * Set of classes to support output streaming into blocks which are then
+ * uploaded as partitions.
+ */
+final class S3ADataBlocks {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(S3ADataBlocks.class);
+
+  private S3ADataBlocks() {
+  }
+
+  /**
+   * Validate args to a write command. These are the same validation checks
+   * expected for any implementation of {@code OutputStream.write()}.
+   * @param b byte array containing data
+   * @param off offset in array where to start
+   * @param len number of bytes to be written
+   * @throws NullPointerException for a null buffer
+   * @throws IndexOutOfBoundsException if indices are out of range
+   */
+  static void validateWriteArgs(byte[] b, int off, int len)
+      throws IOException {
+    Preconditions.checkNotNull(b);
+    if ((off < 0) || (off > b.length) || (len < 0) ||
+        ((off + len) > b.length) || ((off + len) < 0)) {
+      throw new IndexOutOfBoundsException(
+          "write (b[" + b.length + "], " + off + ", " + len + ')');
+    }
+  }
+
+  /**
+   * Create a factory.
+   * @param owner factory owner
+   * @param name factory name -the option from {@link Constants}.
+   * @return the factory, ready to be initialized.
+   * @throws IllegalArgumentException if the name is unknown.
+   */
+  static BlockFactory createFactory(S3AFileSystem owner,
+      String name) {
+    switch (name) {
+    case Constants.FAST_UPLOAD_BUFFER_ARRAY:
+      return new ArrayBlockFactory(owner);
+    case Constants.FAST_UPLOAD_BUFFER_DISK:
+      return new DiskBlockFactory(owner);
+    case Constants.FAST_UPLOAD_BYTEBUFFER:
+      return new ByteBufferBlockFactory(owner);
+    default:
+      throw new IllegalArgumentException("Unsupported block buffer" +
+          " \"" + name + '"');
+    }
+  }
+
+  /**
+   * Base class for block factories.
+   */
+  static abstract class BlockFactory implements Closeable {
+
+    private final S3AFileSystem owner;
+
+    protected BlockFactory(S3AFileSystem owner) {
+      this.owner = owner;
+    }
+
+
+    /**
+     * Create a block.
+     * @param limit limit of the block.
+     * @return a new block.
+     */
+    abstract DataBlock create(int limit) throws IOException;
+
+    /**
+     * Implement any close/cleanup operation.
+     * Base class is a no-op
+     * @throws IOException -ideally, it shouldn't.
+     */
+    @Override
+    public void close() throws IOException {
+    }
+
+    /**
+     * Owner.
+     */
+    protected S3AFileSystem getOwner() {
+      return owner;
+    }
+  }
+
+  /**
+   * This represents a block being uploaded.
+   */
+  static abstract class DataBlock implements Closeable {
+
+    enum DestState {Writing, Upload, Closed}
+
+    private volatile DestState state = Writing;
+
+    /**
+     * Atomically enter a state, verifying current state.
+     * @param current current state. null means "no check"
+     * @param next next state
+     * @throws IllegalStateException if the current state is not as expected
+     */
+    protected synchronized final void enterState(DestState current,
+        DestState next)
+        throws IllegalStateException {
+      verifyState(current);
+      LOG.debug("{}: entering state {}", this, next);
+      state = next;
+    }
+
+    /**
+     * Verify that the block is in the declared state.
+     * @param expected expected state.
+     * @throws IllegalStateException if the DataBlock is in the wrong state
+     */
+    protected final void verifyState(DestState expected)
+        throws IllegalStateException {
+      if (expected != null && state != expected) {
+        throw new IllegalStateException("Expected stream state " + expected
+            + " -but actual state is " + state + " in " + this);
+      }
+    }
+
+    /**
+     * Current state.
+     * @return the current state.
+     */
+    final DestState getState() {
+      return state;
+    }
+
+    /**
+     * Return the current data size.
+     * @return the size of the data
+     */
+    abstract int dataSize();
+
+    /**
+     * Predicate to verify that the block has the capacity to write
+     * the given set of bytes.
+     * @param bytes number of bytes desired to be written.
+     * @return true if there is enough space.
+     */
+    abstract boolean hasCapacity(long bytes);
+
+    /**
+     * Predicate to check if there is data in the block.
+     * @return true if there is
+     */
+    boolean hasData() {
+      return dataSize() > 0;
+    }
+
+    /**
+     * The remaining capacity in the block before it is full.
+     * @return the number of bytes remaining.
+     */
+    abstract int remainingCapacity();
+
+    /**
+     * Write a series of bytes from the buffer, from the offset.
+     * Returns the number of bytes written.
+     * Only valid in the state {@code Writing}.
+     * Base class verifies the state but does no writing.
+     * @param buffer buffer
+     * @param offset offset
+     * @param length length of write
+     * @return number of bytes written
+     * @throws IOException trouble
+     */
+    int write(byte[] buffer, int offset, int length) throws IOException {
+      verifyState(Writing);
+      Preconditions.checkArgument(buffer != null, "Null buffer");
+      Preconditions.checkArgument(length >= 0, "length is negative");
+      Preconditions.checkArgument(offset >= 0, "offset is negative");
+      Preconditions.checkArgument(
+          !(buffer.length - offset < length),
+          "buffer shorter than amount of data to write");
+      return 0;
+    }
+
+    /**
+     * Flush the output.
+     * Only valid in the state {@code Writing}.
+     * In the base class, this is a no-op
+     * @throws IOException any IO problem.
+     */
+    void flush() throws IOException {
+      verifyState(Writing);
+    }
+
+    /**
+     * Switch to the upload state and return a stream for uploading.
+     * Base class calls {@link #enterState(DestState, DestState)} to
+     * manage the state machine.
+     * @return the stream
+     * @throws IOException trouble
+     */
+    InputStream startUpload() throws IOException {
+      LOG.debug("Start datablock upload");
+      enterState(Writing, Upload);
+      return null;
+    }
+
+    /**
+     * Enter the closed state.
+     * @return true if the class was in any other state, implying that
+     * the subclass should do its close operations
+     */
+    protected synchronized boolean enterClosedState() {
+      if (!state.equals(Closed)) {
+        enterState(null, Closed);
+        return true;
+      } else {
+        return false;
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (enterClosedState()) {
+        LOG.debug("Closed {}", this);
+        innerClose();
+      }
+    }
+
+    /**
+     * Inner close logic for subclasses to implement.
+     */
+    protected void innerClose() throws IOException {
+
+    }
+
+  }
+
+  // ====================================================================
+
+  /**
+   * Use byte arrays on the heap for storage.
+   */
+  static class ArrayBlockFactory extends BlockFactory {
+
+    ArrayBlockFactory(S3AFileSystem owner) {
+      super(owner);
+    }
+
+    @Override
+    DataBlock create(int limit) throws IOException {
+      return new ByteArrayBlock(limit);
+    }
+
+  }
+
+  /**
+   * Stream to memory via a {@code ByteArrayOutputStream}.
+   *
+   * This was taken from {@code S3AFastOutputStream} and has the
+   * same problem which surfaced there: it can consume a lot of heap space
+   * proportional to the mismatch between writes to the stream and
+   * the JVM-wide upload bandwidth to the S3 endpoint.
+   * The memory consumption can be limited by tuning the filesystem settings
+   * to restrict the number of queued/active uploads.
+   */
+
+  static class ByteArrayBlock extends DataBlock {
+    private ByteArrayOutputStream buffer;
+    private final int limit;
+    // cache data size so that it is consistent after the buffer is reset.
+    private Integer dataSize;
+
+    ByteArrayBlock(int limit) {
+      this.limit = limit;
+      buffer = new ByteArrayOutputStream();
+    }
+
+    /**
+     * Get the amount of data; if there is no buffer then the size is 0.
+     * @return the amount of data available to upload.
+     */
+    @Override
+    int dataSize() {
+      return dataSize != null ? dataSize : buffer.size();
+    }
+
+    @Override
+    InputStream startUpload() throws IOException {
+      super.startUpload();
+      dataSize = buffer.size();
+      ByteArrayInputStream bufferData = new ByteArrayInputStream(
+          buffer.toByteArray());
+      buffer = null;
+      return bufferData;
+    }
+
+    @Override
+    boolean hasCapacity(long bytes) {
+      return dataSize() + bytes <= limit;
+    }
+
+    @Override
+    int remainingCapacity() {
+      return limit - dataSize();
+    }
+
+    @Override
+    int write(byte[] b, int offset, int len) throws IOException {
+      super.write(b, offset, len);
+      int written = Math.min(remainingCapacity(), len);
+      buffer.write(b, offset, written);
+      return written;
+    }
+
+    @Override
+    protected void innerClose() {
+      buffer = null;
+    }
+
+    @Override
+    public String toString() {
+      return "ByteArrayBlock{" +
+          "state=" + getState() +
+          ", limit=" + limit +
+          ", dataSize=" + dataSize +
+          '}';
+    }
+  }
+
+  // ====================================================================
+
+  /**
+   * Stream via Direct ByteBuffers; these are allocated off heap
+   * via {@link DirectBufferPool}.
+   * This is actually the most complex of all the block factories,
+   * due to the need to explicitly recycle buffers; in comparison, the
+   * {@link DiskBlock} buffer delegates the work of deleting files to
+   * the {@link DiskBlock.FileDeletingInputStream}. Here the
+   * input stream {@link ByteBufferInputStream} has a similar task, along
+   * with the foundational work of streaming data from a byte array.
+   */
+
+  static class ByteBufferBlockFactory extends BlockFactory {
+
+    private final DirectBufferPool bufferPool = new DirectBufferPool();
+    private final AtomicInteger buffersOutstanding = new AtomicInteger(0);
+
+    ByteBufferBlockFactory(S3AFileSystem owner) {
+      super(owner);
+    }
+
+    @Override
+    ByteBufferBlock create(int limit) throws IOException {
+      return new ByteBufferBlock(limit);
+    }
+
+    private ByteBuffer requestBuffer(int limit) {
+      LOG.debug("Requesting buffer of size {}", limit);
+      buffersOutstanding.incrementAndGet();
+      return bufferPool.getBuffer(limit);
+    }
+
+    private void releaseBuffer(ByteBuffer buffer) {
+      LOG.debug("Releasing buffer");
+      bufferPool.returnBuffer(buffer);
+      buffersOutstanding.decrementAndGet();
+    }
+
+    /**
+     * Get count of outstanding buffers.
+     * @return the current buffer count
+     */
+    public int getOutstandingBufferCount() {
+      return buffersOutstanding.get();
+    }
+
+    @Override
+    public String toString() {
+      return "ByteBufferBlockFactory{"
+          + "buffersOutstanding=" + buffersOutstanding +
+          '}';
+    }
+
+    /**
+     * A DataBlock which requests a buffer from pool on creation; returns
+     * it when the output stream is closed.
+     */
+    class ByteBufferBlock extends DataBlock {
+      private ByteBuffer buffer;
+      private final int bufferSize;
+      // cache data size so that it is consistent after the buffer is reset.
+      private Integer dataSize;
+
+      /**
+       * Instantiate. This will request a ByteBuffer of the desired size.
+       * @param bufferSize buffer size
+       */
+      ByteBufferBlock(int bufferSize) {
+        this.bufferSize = bufferSize;
+        buffer = requestBuffer(bufferSize);
+      }
+
+      /**
+       * Get the amount of data; if there is no buffer then the size is 0.
+       * @return the amount of data available to upload.
+       */
+      @Override
+      int dataSize() {
+        return dataSize != null ? dataSize : bufferCapacityUsed();
+      }
+
+      @Override
+      ByteBufferInputStream startUpload() throws IOException {
+        super.startUpload();
+        dataSize = bufferCapacityUsed();
+        // set the buffer up from reading from the beginning
+        buffer.limit(buffer.position());
+        buffer.position(0);
+        return new ByteBufferInputStream(dataSize, buffer);
+      }
+
+      @Override
+      public boolean hasCapacity(long bytes) {
+        return bytes <= remainingCapacity();
+      }
+
+      @Override
+      public int remainingCapacity() {
+        return buffer != null ? buffer.remaining() : 0;
+      }
+
+      private int bufferCapacityUsed() {
+        return buffer.capacity() - buffer.remaining();
+      }
+
+      @Override
+      int write(byte[] b, int offset, int len) throws IOException {
+        super.write(b, offset, len);
+        int written = Math.min(remainingCapacity(), len);
+        buffer.put(b, offset, written);
+        return written;
+      }
+
+      @Override
+      protected void innerClose() {
+        buffer = null;
+      }
+
+      @Override
+      public String toString() {
+        return "ByteBufferBlock{"
+            + "state=" + getState() +
+            ", dataSize=" + dataSize() +
+            ", limit=" + bufferSize +
+            ", remainingCapacity=" + remainingCapacity() +
+            '}';
+      }
+
+    }
+
+    /**
+     * Provide an input stream from a byte buffer; supporting
+     * {@link #mark(int)}, which is required to enable replay of failed
+     * PUT attempts.
+     * This input stream returns the buffer to the pool afterwards.
+     */
+    class ByteBufferInputStream extends InputStream {
+
+      private final int size;
+      private ByteBuffer byteBuffer;
+
+      ByteBufferInputStream(int size, ByteBuffer byteBuffer) {
+        LOG.debug("Creating ByteBufferInputStream of size {}", size);
+        this.size = size;
+        this.byteBuffer = byteBuffer;
+      }
+
+      /**
+       * Return the buffer to the pool after the stream is closed.
+       */
+      @Override
+      public synchronized void close() {
+        if (byteBuffer != null) {
+          LOG.debug("releasing buffer");
+          releaseBuffer(byteBuffer);
+          byteBuffer = null;
+        }
+      }
+
+      /**
+       * Verify that the stream is open.
+       * @throws IOException if the stream is closed
+       */
+      private void verifyOpen() throws IOException {
+        if (byteBuffer == null) {
+          throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+        }
+      }
+
+      public synchronized int read() throws IOException {
+        if (available() > 0) {
+          return byteBuffer.get() & 0xFF;
+        } else {
+          return -1;
+        }
+      }
+
+      @Override
+      public synchronized long skip(long offset) throws IOException {
+        verifyOpen();
+        long newPos = position() + offset;
+        if (newPos < 0) {
+          throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK);
+        }
+        if (newPos > size) {
+          throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
+        }
+        byteBuffer.position((int) newPos);
+        return newPos;
+      }
+
+      @Override
+      public synchronized int available() {
+        Preconditions.checkState(byteBuffer != null,
+            FSExceptionMessages.STREAM_IS_CLOSED);
+        return byteBuffer.remaining();
+      }
+
+      /**
+       * Get the current buffer position.
+       * @return the buffer position
+       */
+      public synchronized int position() {
+        return byteBuffer.position();
+      }
+
+      /**
+       * Check if there is data left.
+       * @return true if there is data remaining in the buffer.
+       */
+      public synchronized boolean hasRemaining() {
+        return byteBuffer.hasRemaining();
+      }
+
+      @Override
+      public synchronized void mark(int readlimit) {
+        LOG.debug("mark at {}", position());
+        byteBuffer.mark();
+      }
+
+      @Override
+      public synchronized void reset() throws IOException {
+        LOG.debug("reset");
+        byteBuffer.reset();
+      }
+
+      @Override
+      public boolean markSupported() {
+        return true;
+      }
+
+      /**
+       * Read in data.
+       * @param buffer destination buffer
+       * @param offset offset within the buffer
+       * @param length length of bytes to read
+       * @throws EOFException if the position is negative
+       * @throws IndexOutOfBoundsException if there isn't space for the
+       * amount of data requested.
+       * @throws IllegalArgumentException other arguments are invalid.
+       */
+      @SuppressWarnings("NullableProblems")
+      public synchronized int read(byte[] buffer, int offset, int length)
+          throws IOException {
+        Preconditions.checkArgument(length >= 0, "length is negative");
+        Preconditions.checkArgument(buffer != null, "Null buffer");
+        if (buffer.length - offset < length) {
+          throw new IndexOutOfBoundsException(
+              FSExceptionMessages.TOO_MANY_BYTES_FOR_DEST_BUFFER
+                  + ": request length =" + length
+                  + ", with offset =" + offset
+                  + "; buffer capacity =" + (buffer.length - offset));
+        }
+        verifyOpen();
+        if (!hasRemaining()) {
+          return -1;
+        }
+
+        int toRead = Math.min(length, available());
+        byteBuffer.get(buffer, offset, toRead);
+        return toRead;
+      }
+
+      @Override
+      public String toString() {
+        final StringBuilder sb = new StringBuilder(
+            "ByteBufferInputStream{");
+        sb.append("size=").append(size);
+        ByteBuffer buffer = this.byteBuffer;
+        if (buffer != null) {
+          sb.append(", available=").append(buffer.remaining());
+        }
+        sb.append('}');
+        return sb.toString();
+      }
+    }
+  }
+
+  // ====================================================================
+
+  /**
+   * Buffer blocks to disk.
+   */
+  static class DiskBlockFactory extends BlockFactory {
+
+    DiskBlockFactory(S3AFileSystem owner) {
+      super(owner);
+    }
+
+    /**
+     * Create a temp file and a block which writes to it.
+     * @param limit limit of the block.
+     * @return the new block
+     * @throws IOException IO problems
+     */
+    @Override
+    DataBlock create(int limit) throws IOException {
+      File destFile = getOwner()
+          .createTmpFileForWrite("s3ablock", limit, getOwner().getConf());
+      return new DiskBlock(destFile, limit);
+    }
+  }
+
+  /**
+   * Stream to a file.
+   * This will stop at the limit; the caller is expected to create a new block
+   */
+  static class DiskBlock extends DataBlock {
+
+    private int bytesWritten;
+    private final File bufferFile;
+    private final int limit;
+    private BufferedOutputStream out;
+    private InputStream uploadStream;
+
+    DiskBlock(File bufferFile, int limit)
+        throws FileNotFoundException {
+      this.limit = limit;
+      this.bufferFile = bufferFile;
+      out = new BufferedOutputStream(new FileOutputStream(bufferFile));
+    }
+
+    @Override
+    int dataSize() {
+      return bytesWritten;
+    }
+
+    @Override
+    boolean hasCapacity(long bytes) {
+      return dataSize() + bytes <= limit;
+    }
+
+    @Override
+    int remainingCapacity() {
+      return limit - bytesWritten;
+    }
+
+    @Override
+    int write(byte[] b, int offset, int len) throws IOException {
+      super.write(b, offset, len);
+      int written = Math.min(remainingCapacity(), len);
+      out.write(b, offset, written);
+      bytesWritten += written;
+      return written;
+    }
+
+    @Override
+    InputStream startUpload() throws IOException {
+      super.startUpload();
+      try {
+        out.flush();
+      } finally {
+        out.close();
+        out = null;
+      }
+      uploadStream = new FileInputStream(bufferFile);
+      return new FileDeletingInputStream(uploadStream);
+    }
+
+    /**
+     * The close operation will delete the destination file if it still
+     * exists.
+     * @throws IOException IO problems
+     */
+    @Override
+    protected void innerClose() throws IOException {
+      final DestState state = getState();
+      LOG.debug("Closing {}", this);
+      switch (state) {
+      case Writing:
+        if (bufferFile.exists()) {
+          // file was not uploaded
+          LOG.debug("Deleting buffer file as upload did not start");
+          boolean deleted = bufferFile.delete();
+          if (!deleted && bufferFile.exists()) {
+            LOG.warn("Failed to delete buffer file {}", bufferFile);
+          }
+        }
+        break;
+
+      case Upload:
+        LOG.debug("Buffer file {} exists \u2014close upload stream", bufferFile);
+        break;
+
+      case Closed:
+        // no-op
+        break;
+
+      default:
+        // this state can never be reached, but checkstyle complains, so
+        // it is here.
+      }
+    }
+
+    /**
+     * Flush operation will flush to disk.
+     * @throws IOException IOE raised on FileOutputStream
+     */
+    @Override
+    void flush() throws IOException {
+      super.flush();
+      out.flush();
+    }
+
+    @Override
+    public String toString() {
+      String sb = "FileBlock{"
+          + "destFile=" + bufferFile +
+          ", state=" + getState() +
+          ", dataSize=" + dataSize() +
+          ", limit=" + limit +
+          '}';
+      return sb;
+    }
+
+    /**
+     * An input stream which deletes the buffer file when closed.
+     */
+    private final class FileDeletingInputStream extends FilterInputStream {
+      private final AtomicBoolean closed = new AtomicBoolean(false);
+
+      FileDeletingInputStream(InputStream source) {
+        super(source);
+      }
+
+      /**
+       * Delete the input file when closed.
+       * @throws IOException IO problem
+       */
+      @Override
+      public void close() throws IOException {
+        try {
+          super.close();
+        } finally {
+          if (!closed.getAndSet(true)) {
+            if (!bufferFile.delete()) {
+              LOG.warn("delete({}) returned false",
+                  bufferFile.getAbsoluteFile());
+            }
+          }
+        }
+      }
+    }
+  }
+
+}


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


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

Posted by sj...@apache.org.
HDFS-11008. Change unit test for testing parsing "-source" parameter in Balancer CLI. Contributed by Mingliang Liu


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

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

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


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


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


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

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


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

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

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


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


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


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

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


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

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

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


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


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


[49/50] [abbrv] hadoop git commit: HDFS-10906. Add unit tests for Trash with HDFS encryption zones. Contributed by Hanisha Koneru.

Posted by sj...@apache.org.
HDFS-10906. Add unit tests for Trash with HDFS encryption zones. Contributed by Hanisha Koneru.


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

Branch: refs/heads/HADOOP-13070
Commit: c62ae7107f025091652e79db3edfca5c4dc84e4a
Parents: 6c348c5
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Mon Oct 17 15:25:24 2016 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Tue Oct 18 14:05:43 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  32 +-
 .../hdfs/TestTrashWithEncryptionZones.java      | 188 ++++++++
 .../TestTrashWithSecureEncryptionZones.java     | 443 +++++++++++++++++++
 3 files changed, 662 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c62ae710/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 963aaa6..7f26b03 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -27,6 +27,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
 
 import java.io.BufferedOutputStream;
@@ -114,7 +115,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
@@ -167,6 +167,7 @@ import org.apache.hadoop.util.VersionInfo;
 import org.apache.log4j.Level;
 import org.junit.Assume;
 import org.mockito.internal.util.reflection.Whitebox;
+import org.apache.hadoop.util.ToolRunner;
 
 import com.google.common.annotations.VisibleForTesting;
 import static org.apache.hadoop.hdfs.StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
@@ -2054,4 +2055,33 @@ public class DFSTestUtil {
       }
     }
   }
+
+  public static void verifyDelete(FsShell shell, FileSystem fs, Path path,
+      boolean shouldExistInTrash) throws Exception {
+    Path trashPath = Path.mergePaths(shell.getCurrentTrashDir(path), path);
+
+    verifyDelete(shell, fs, path, trashPath, shouldExistInTrash);
+  }
+
+  public static void verifyDelete(FsShell shell, FileSystem fs, Path path,
+      Path trashPath, boolean shouldExistInTrash) throws Exception {
+    assertTrue(path + " file does not exist", fs.exists(path));
+
+    // Verify that trashPath has a path component named ".Trash"
+    Path checkTrash = trashPath;
+    while (!checkTrash.isRoot() && !checkTrash.getName().equals(".Trash")) {
+      checkTrash = checkTrash.getParent();
+    }
+    assertEquals("No .Trash component found in trash path " + trashPath,
+        ".Trash", checkTrash.getName());
+
+    String[] argv = new String[]{"-rm", "-r", path.toString()};
+    int res = ToolRunner.run(shell, argv);
+    assertEquals("rm failed", 0, res);
+    if (shouldExistInTrash) {
+      assertTrue("File not in trash : " + trashPath, fs.exists(trashPath));
+    } else {
+      assertFalse("File in trash : " + trashPath, fs.exists(trashPath));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c62ae710/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestTrashWithEncryptionZones.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestTrashWithEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestTrashWithEncryptionZones.java
new file mode 100644
index 0000000..2a8d493
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestTrashWithEncryptionZones.java
@@ -0,0 +1,188 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.FileSystemTestWrapper;
+import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.client.CreateEncryptionZoneFlag;
+import org.apache.hadoop.hdfs.client.HdfsAdmin;
+import org.apache.hadoop.hdfs.server.namenode.EncryptionZoneManager;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.log4j.Level;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.security.PrivilegedExceptionAction;
+import java.util.EnumSet;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * This class tests Trash functionality in Encryption Zones.
+ */
+public class TestTrashWithEncryptionZones {
+  private Configuration conf;
+  private FileSystemTestHelper fsHelper;
+
+  private MiniDFSCluster cluster;
+  private HdfsAdmin dfsAdmin;
+  private DistributedFileSystem fs;
+  private File testRootDir;
+  private static final String TEST_KEY = "test_key";
+
+  private FileSystemTestWrapper fsWrapper;
+  private static Configuration clientConf;
+  private static FsShell shell;
+
+  private static AtomicInteger zoneCounter = new AtomicInteger(1);
+  private static AtomicInteger fileCounter = new AtomicInteger(1);
+  private static final int LEN = 8192;
+
+  private static final EnumSet< CreateEncryptionZoneFlag > NO_TRASH =
+      EnumSet.of(CreateEncryptionZoneFlag.NO_TRASH);
+  private static final EnumSet<CreateEncryptionZoneFlag> PROVISION_TRASH =
+      EnumSet.of(CreateEncryptionZoneFlag.PROVISION_TRASH);
+
+  private String getKeyProviderURI() {
+    return JavaKeyStoreProvider.SCHEME_NAME + "://file" +
+        new Path(testRootDir.toString(), "test.jks").toUri();
+  }
+
+  @Before
+  public void setup() throws Exception {
+    conf = new HdfsConfiguration();
+    fsHelper = new FileSystemTestHelper();
+    // Set up java key store
+    String testRoot = fsHelper.getTestRootDir();
+    testRootDir = new File(testRoot).getAbsoluteFile();
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        getKeyProviderURI());
+    conf.setBoolean(DFSConfigKeys
+        .DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
+    // Lower the batch size for testing
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES,
+        2);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    org.apache.log4j.Logger
+        .getLogger(EncryptionZoneManager.class).setLevel(Level.TRACE);
+    fs = cluster.getFileSystem();
+    fsWrapper = new FileSystemTestWrapper(fs);
+    dfsAdmin = new HdfsAdmin(cluster.getURI(), conf);
+    setProvider();
+    // Create a test key
+    DFSTestUtil.createKey(TEST_KEY, cluster, conf);
+
+    clientConf = new Configuration(conf);
+    clientConf.setLong(FS_TRASH_INTERVAL_KEY, 1);
+    shell = new FsShell(clientConf);
+  }
+
+  protected void setProvider() {
+    // Need to set the client's KeyProvider to the NN's for JKS,
+    // else the updates do not get flushed properly
+    fs.getClient().setKeyProvider(cluster.getNameNode().getNamesystem()
+        .getProvider());
+  }
+
+  @After
+  public void teardown() {
+    if (cluster != null) {
+      cluster.shutdown();
+      cluster = null;
+    }
+  }
+
+  @Test
+  public void testDeleteWithinEncryptionZone() throws Exception {
+    final Path zone = new Path("/zones");
+    fs.mkdirs(zone);
+    final Path zone1 = new Path("/zones/zone" + zoneCounter.getAndIncrement());
+    fs.mkdirs(zone1);
+    dfsAdmin.createEncryptionZone(zone1, TEST_KEY, PROVISION_TRASH);
+
+    final Path encFile1 = new Path(zone1, "encFile" + fileCounter
+        .getAndIncrement());
+    DFSTestUtil.createFile(fs, encFile1, LEN, (short) 1, 0xFEED);
+
+    //Verify file deletion
+    DFSTestUtil.verifyDelete(shell, fs, encFile1, true);
+
+    //Verify directory deletion
+    DFSTestUtil.verifyDelete(shell, fs, zone1, true);
+  }
+
+  @Test
+  public void testDeleteEZWithMultipleUsers() throws Exception {
+    final Path zone = new Path("/zones");
+    fs.mkdirs(zone);
+    final Path zone1 = new Path("/zones/zone" + zoneCounter.getAndIncrement());
+    fs.mkdirs(zone1);
+    dfsAdmin.createEncryptionZone(zone1, TEST_KEY, NO_TRASH);
+
+    fsWrapper.setPermission(zone1,
+        new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
+
+    final Path encFile1 = new Path(zone1, "encFile" + fileCounter
+        .getAndIncrement());
+    DFSTestUtil.createFile(fs, encFile1, LEN, (short) 1, 0xFEED);
+
+    // create a non-privileged user
+    final UserGroupInformation user = UserGroupInformation
+        .createUserForTesting("user", new String[]{"mygroup"});
+
+    final Path encFile2 = new Path(zone1, "encFile" + fileCounter
+        .getAndIncrement());
+
+    user.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        // create a file /zones/zone1/encFile2 in EZ
+        // this file is owned by user:mygroup
+        FileSystem fs2 = FileSystem.get(cluster.getConfiguration(0));
+        DFSTestUtil.createFile(fs2, encFile2, LEN, (short) 1, 0xFEED);
+
+        // Delete /zones/zone1/encFile2, which moves the file to
+        // /zones/zone1/.Trash/user/Current/zones/zone1/encFile2
+        DFSTestUtil.verifyDelete(shell, fs, encFile2, true);
+
+        // Delete /zones/zone1 should not succeed as current user is not admin
+        String[] argv = new String[]{"-rm", "-r", zone1.toString()};
+        int res = ToolRunner.run(shell, argv);
+        assertEquals("Non-admin could delete an encryption zone with multiple" +
+            " users : " + zone1, 1, res);
+        return null;
+      }
+    });
+
+    shell = new FsShell(clientConf);
+    DFSTestUtil.verifyDelete(shell, fs, zone1, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c62ae710/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestTrashWithSecureEncryptionZones.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestTrashWithSecureEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestTrashWithSecureEncryptionZones.java
new file mode 100644
index 0000000..314adfb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestTrashWithSecureEncryptionZones.java
@@ -0,0 +1,443 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.apache.hadoop.fs.CommonConfigurationKeys
+    .IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic
+    .FS_TRASH_INTERVAL_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic
+    .KMS_CLIENT_ENC_KEY_CACHE_LOW_WATERMARK;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.
+    KMS_CLIENT_ENC_KEY_CACHE_SIZE;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.
+    DFS_DATA_TRANSFER_PROTECTION_KEY;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
+import org.apache.hadoop.crypto.key.kms.server.KMSConfiguration;
+import org.apache.hadoop.crypto.key.kms.server.MiniKMS;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.CreateEncryptionZoneFlag;
+import org.apache.hadoop.hdfs.client.HdfsAdmin;
+import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runners.MethodSorters;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.Writer;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * This class tests Trash functionality in Encryption Zones with Kerberos
+ * enabled.
+ */
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class TestTrashWithSecureEncryptionZones {
+  private static HdfsConfiguration baseConf;
+  private static File baseDir;
+  private static final EnumSet<CreateEncryptionZoneFlag> PROVISION_TRASH =
+      EnumSet.of(CreateEncryptionZoneFlag.PROVISION_TRASH);
+
+  private static final String HDFS_USER_NAME = "hdfs";
+  private static final String SPNEGO_USER_NAME = "HTTP";
+  private static final String OOZIE_USER_NAME = "oozie";
+  private static final String OOZIE_PROXIED_USER_NAME = "oozie_user";
+
+  private static String hdfsPrincipal;
+  private static String spnegoPrincipal;
+  private static String keytab;
+
+  // MiniKDC
+  private static MiniKdc kdc;
+
+  // MiniKMS
+  private static MiniKMS miniKMS;
+  private static final String TEST_KEY = "test_key";
+  private static final Path CURRENT = new Path("Current");
+
+  // MiniDFS
+  private static MiniDFSCluster cluster;
+  private static HdfsConfiguration conf;
+  private static FileSystem fs;
+  private static HdfsAdmin dfsAdmin;
+  private static Configuration clientConf;
+  private static FsShell shell;
+
+  private static AtomicInteger zoneCounter = new AtomicInteger(1);
+  private static AtomicInteger fileCounter = new AtomicInteger(1);
+  private static final int LEN = 8192;
+
+  public static File getTestDir() throws Exception {
+    File file = new File("dummy");
+    file = file.getAbsoluteFile();
+    file = file.getParentFile();
+    file = new File(file, "target");
+    file = new File(file, UUID.randomUUID().toString());
+    if (!file.mkdirs()) {
+      throw new RuntimeException("Could not create test directory: " + file);
+    }
+    return file;
+  }
+
+  @BeforeClass
+  public static void init() throws Exception {
+    baseDir = getTestDir();
+    FileUtil.fullyDelete(baseDir);
+    assertTrue(baseDir.mkdirs());
+
+    Properties kdcConf = MiniKdc.createConf();
+    kdc = new MiniKdc(kdcConf, baseDir);
+    kdc.start();
+
+    baseConf = new HdfsConfiguration();
+    SecurityUtil.setAuthenticationMethod(UserGroupInformation
+        .AuthenticationMethod.KERBEROS, baseConf);
+    UserGroupInformation.setConfiguration(baseConf);
+    assertTrue("Expected configuration to enable security",
+        UserGroupInformation.isSecurityEnabled());
+
+    File keytabFile = new File(baseDir, "test.keytab");
+    keytab = keytabFile.getAbsolutePath();
+    // Windows will not reverse name lookup "127.0.0.1" to "localhost".
+    String krbInstance = Path.WINDOWS ? "127.0.0.1" : "localhost";
+
+    kdc.createPrincipal(keytabFile,
+        HDFS_USER_NAME + "/" + krbInstance,
+        SPNEGO_USER_NAME + "/" + krbInstance,
+        OOZIE_USER_NAME + "/" + krbInstance,
+        OOZIE_PROXIED_USER_NAME + "/" + krbInstance);
+
+    hdfsPrincipal = HDFS_USER_NAME + "/" + krbInstance + "@" + kdc.getRealm();
+    spnegoPrincipal = SPNEGO_USER_NAME + "/" + krbInstance + "@" + kdc
+        .getRealm();
+
+    baseConf.set(DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY, hdfsPrincipal);
+    baseConf.set(DFS_NAMENODE_KEYTAB_FILE_KEY, keytab);
+    baseConf.set(DFS_DATANODE_KERBEROS_PRINCIPAL_KEY, hdfsPrincipal);
+    baseConf.set(DFS_DATANODE_KEYTAB_FILE_KEY, keytab);
+    baseConf.set(DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY,
+        spnegoPrincipal);
+    baseConf.setBoolean(DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
+    baseConf.set(DFS_DATA_TRANSFER_PROTECTION_KEY, "authentication");
+    baseConf.set(DFS_HTTP_POLICY_KEY, HttpConfig.Policy.HTTPS_ONLY.name());
+    baseConf.set(DFS_NAMENODE_HTTPS_ADDRESS_KEY, "localhost:0");
+    baseConf.set(DFS_DATANODE_HTTPS_ADDRESS_KEY, "localhost:0");
+    baseConf.set(DFS_JOURNALNODE_HTTPS_ADDRESS_KEY, "localhost:0");
+    baseConf.setInt(IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_KEY, 10);
+
+    // Set a small (2=4*0.5) KMSClient EDEK cache size to trigger
+    // on demand refill upon the 3rd file creation
+    baseConf.set(KMS_CLIENT_ENC_KEY_CACHE_SIZE, "4");
+    baseConf.set(KMS_CLIENT_ENC_KEY_CACHE_LOW_WATERMARK, "0.5");
+
+    String keystoresDir = baseDir.getAbsolutePath();
+    String sslConfDir = KeyStoreTestUtil.getClasspathDir(
+        TestSecureEncryptionZoneWithKMS.class);
+    KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, baseConf, false);
+    baseConf.set(DFS_CLIENT_HTTPS_KEYSTORE_RESOURCE_KEY,
+        KeyStoreTestUtil.getClientSSLConfigFileName());
+    baseConf.set(DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY,
+        KeyStoreTestUtil.getServerSSLConfigFileName());
+
+    File kmsFile = new File(baseDir, "kms-site.xml");
+    if (kmsFile.exists()) {
+      FileUtil.fullyDelete(kmsFile);
+    }
+
+    Configuration kmsConf = new Configuration(true);
+    kmsConf.set(
+        KMSConfiguration.KEY_PROVIDER_URI,
+        "jceks://file@" + new Path(baseDir.toString(), "kms.keystore")
+            .toUri());
+    kmsConf.set("hadoop.kms.authentication.type", "kerberos");
+    kmsConf.set("hadoop.kms.authentication.kerberos.keytab", keytab);
+    kmsConf.set("hadoop.kms.authentication.kerberos.principal",
+        "HTTP/localhost");
+    kmsConf.set("hadoop.kms.authentication.kerberos.name.rules", "DEFAULT");
+    kmsConf.set("hadoop.kms.acl.GENERATE_EEK", "hdfs");
+
+    Writer writer = new FileWriter(kmsFile);
+    kmsConf.writeXml(writer);
+    writer.close();
+
+    // Start MiniKMS
+    MiniKMS.Builder miniKMSBuilder = new MiniKMS.Builder();
+    miniKMS = miniKMSBuilder.setKmsConfDir(baseDir).build();
+    miniKMS.start();
+
+    baseConf.set(CommonConfigurationKeysPublic
+        .HADOOP_SECURITY_KEY_PROVIDER_PATH, getKeyProviderURI());
+    baseConf.setBoolean(DFSConfigKeys
+        .DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
+
+    conf = new HdfsConfiguration(baseConf);
+    cluster = new MiniDFSCluster.Builder(conf)
+        .build();
+    cluster.waitActive();
+
+    fs = cluster.getFileSystem();
+    dfsAdmin = new HdfsAdmin(cluster.getURI(), conf);
+
+    // Wait cluster to be active
+    cluster.waitActive();
+
+    // Create a test key
+    DFSTestUtil.createKey(TEST_KEY, cluster, conf);
+    clientConf = new Configuration(conf);
+    clientConf.setLong(FS_TRASH_INTERVAL_KEY, 1);
+    shell = new FsShell(clientConf);
+    System.setProperty("user.name", HDFS_USER_NAME);
+  }
+
+  @AfterClass
+  public static void destroy() {
+    IOUtils.cleanup(null, fs);
+    if (cluster != null) {
+      cluster.shutdown();
+      cluster = null;
+    }
+
+    if (kdc != null) {
+      kdc.stop();
+    }
+    if (miniKMS != null) {
+      miniKMS.stop();
+    }
+    FileUtil.fullyDelete(baseDir);
+  }
+
+  private static String getKeyProviderURI() {
+    return KMSClientProvider.SCHEME_NAME + "://" +
+        miniKMS.getKMSUrl().toExternalForm().replace("://", "@");
+  }
+
+  @Test
+  public void testTrashCheckpoint() throws Exception {
+    final Path zone1 = new Path("/zone" + zoneCounter.getAndIncrement());
+    fs.mkdirs(zone1);
+    final Path zone2 = new Path(zone1 + "/zone" +
+        zoneCounter.getAndIncrement());
+    fs.mkdirs(zone2);
+    dfsAdmin.createEncryptionZone(zone2, TEST_KEY, PROVISION_TRASH);
+
+    final Path encFile1 = new Path(zone2, "encFile" + fileCounter
+        .getAndIncrement());
+    DFSTestUtil.createFile(fs, encFile1, LEN, (short) 1, 0xFEED);
+
+    //Verify Trash checkpoint within Encryption Zone
+    Path trashDir = new Path(zone2, fs.TRASH_PREFIX + "/" + HDFS_USER_NAME +
+        "/" + CURRENT);
+    String trashPath = trashDir.toString() + encFile1.toString();
+    Path deletedFile = verifyTrashLocationWithShellDelete(encFile1);
+    assertEquals("Deleted file not at the expected trash location: " +
+        trashPath, trashPath, deletedFile.toUri().getPath());
+
+    //Verify Trash checkpoint outside the encryption zone when the whole
+    // encryption zone is deleted and moved
+    trashPath = fs.getHomeDirectory().toUri().getPath() + "/" + fs
+        .TRASH_PREFIX + "/" + CURRENT + zone2;
+    Path deletedDir = verifyTrashLocationWithShellDelete(zone2);
+    assertEquals("Deleted zone not at the expected trash location: " +
+        trashPath, trashPath, deletedDir.toUri().getPath());
+  }
+
+  @Test
+  public void testTrashExpunge() throws Exception {
+    final Path zone1 = new Path("/zone" + zoneCounter.getAndIncrement());
+    fs.mkdirs(zone1);
+    final Path zone2 = new Path("/zone" + zoneCounter.getAndIncrement());
+    fs.mkdirs(zone2);
+    dfsAdmin.createEncryptionZone(zone1, TEST_KEY, PROVISION_TRASH);
+
+    final Path file1 = new Path(zone1, "encFile" + fileCounter
+        .getAndIncrement());
+    final Path file2 = new Path(zone2, "file" + fileCounter.getAndIncrement());
+    DFSTestUtil.createFile(fs, file1, LEN, (short) 1, 0xFEED);
+    DFSTestUtil.createFile(fs, file2, LEN, (short) 1, 0xFEED);
+
+    //Verify Trash expunge within the encryption zone
+    List<Path> trashPaths = Lists.newArrayList();
+    trashPaths.add(verifyTrashLocationWithShellDelete(file1));
+    trashPaths.add(verifyTrashLocationWithShellDelete(file2));
+    verifyTrashExpunge(trashPaths);
+
+    //Verify Trash expunge when the whole encryption zone has been deleted
+    final Path file3 = new Path(zone1, "encFile" + fileCounter
+        .getAndIncrement());
+    DFSTestUtil.createFile(fs, file3, LEN, (short) 1, 0XFEED);
+    Path trashPath = verifyTrashLocationWithShellDelete(file3);
+    //Delete encryption zone
+    DFSTestUtil.verifyDelete(shell, fs, zone1, true);
+    verifyTrashExpunge(Lists.newArrayList(trashPath));
+
+  }
+
+  @Test
+  public void testDeleteWithSkipTrash() throws Exception {
+    final Path zone1 = new Path("/zone" + zoneCounter.getAndIncrement());
+    fs.mkdirs(zone1);
+
+    final Path encFile1 = new Path(zone1, "encFile" + fileCounter
+        .getAndIncrement());
+    final Path encFile2 = new Path(zone1, "encFile" + fileCounter
+        .getAndIncrement());
+    DFSTestUtil.createFile(fs, encFile1, LEN, (short) 1, 0xFEED);
+    DFSTestUtil.createFile(fs, encFile2, LEN, (short) 1, 0xFEED);
+
+    //Verify file deletion with skipTrash
+    verifyDeleteWithSkipTrash(encFile1);
+
+    //Verify file deletion without skipTrash
+    DFSTestUtil.verifyDelete(shell, fs, encFile2, true);
+  }
+
+  @Test
+  public void testDeleteEmptyDirectory() throws Exception {
+    final Path zone1 = new Path("/zone" + zoneCounter.getAndIncrement());
+    final Path zone2 = new Path("/zone" + zoneCounter.getAndIncrement());
+    fs.mkdirs(zone1);
+    fs.mkdirs(zone2);
+
+    final Path trashDir1 = new Path(shell.getCurrentTrashDir(zone1) + "/" +
+        zone1);
+    final Path trashDir2 = new Path(shell.getCurrentTrashDir(zone1) + "/" +
+        zone2);
+
+    //Delete empty directory with -r option
+    String[] argv1 = new String[]{"-rm", "-r", zone1.toString()};
+    int res = ToolRunner.run(shell, argv1);
+    assertEquals("rm failed", 0, res);
+    assertTrue("Empty directory not deleted even with -r : " + trashDir1, fs
+        .exists(trashDir1));
+
+    //Delete empty directory without -r option
+    String[] argv2 = new String[]{"-rm", zone2.toString()};
+    res = ToolRunner.run(shell, argv2);
+    assertEquals("rm on empty directory did not fail", 1, res);
+    assertTrue("Empty directory deleted without -r : " + trashDir2, !fs.exists(
+        trashDir2));
+  }
+
+  @Test
+  public void testDeleteFromTrashWithinEZ() throws Exception {
+    final Path zone1 = new Path("/zone" + zoneCounter.getAndIncrement());
+    fs.mkdirs(zone1);
+    dfsAdmin.createEncryptionZone(zone1, TEST_KEY, PROVISION_TRASH);
+
+    final Path encFile1 = new Path(zone1, "encFile" + fileCounter
+        .getAndIncrement());
+    DFSTestUtil.createFile(fs, encFile1, LEN, (short) 1, 0xFEED);
+
+    final Path trashFile = new Path(shell.getCurrentTrashDir(encFile1) + "/" +
+        encFile1);
+
+    String[] argv = new String[]{"-rm", "-r", encFile1.toString()};
+    int res = ToolRunner.run(shell, argv);
+    assertEquals("rm failed", 0, res);
+
+    String[] argvDeleteTrash = new String[]{"-rm", "-r", trashFile.toString()};
+    int resDeleteTrash = ToolRunner.run(shell, argvDeleteTrash);
+    assertEquals("rm failed", 0, resDeleteTrash);
+    assertFalse("File deleted from Trash : " + trashFile, fs.exists(trashFile));
+  }
+
+  @Test
+  public void testTrashRetentionAfterNamenodeRestart() throws Exception {
+    final Path zone1 = new Path("/zone" + zoneCounter.getAndIncrement());
+    fs.mkdirs(zone1);
+    dfsAdmin.createEncryptionZone(zone1, TEST_KEY, PROVISION_TRASH);
+
+    final Path encFile1 = new Path(zone1, "encFile" + fileCounter
+        .getAndIncrement());
+    DFSTestUtil.createFile(fs, encFile1, LEN, (short) 1, 0xFEED);
+
+    final Path trashFile = new Path(shell.getCurrentTrashDir(encFile1) + "/" +
+        encFile1);
+    String[] argv = new String[]{"-rm", "-r", encFile1.toString()};
+    int res = ToolRunner.run(shell, argv);
+    assertEquals("rm failed", 0, res);
+
+    assertTrue("File not in trash : " + trashFile, fs.exists(trashFile));
+    cluster.restartNameNode(0);
+    cluster.waitActive();
+    fs = cluster.getFileSystem();
+
+    assertTrue("On Namenode restart, file deleted from trash : " +
+        trashFile, fs.exists(trashFile));
+  }
+
+  private Path verifyTrashLocationWithShellDelete(Path path)
+      throws Exception {
+
+    final Path trashFile = new Path(shell.getCurrentTrashDir(path) + "/" +
+        path);
+    File deletedFile = new File(String.valueOf(trashFile));
+    assertFalse("File already present in Trash before delete", deletedFile
+        .exists());
+
+    DFSTestUtil.verifyDelete(shell, fs, path, trashFile, true);
+    return trashFile;
+  }
+
+  private void verifyTrashExpunge(List<Path> trashFiles) throws Exception {
+    String[] argv = new String[]{"-expunge"};
+    int res = ToolRunner.run(shell, argv);
+    assertEquals("expunge failed", 0, res);
+
+    for (Path trashFile : trashFiles) {
+      assertFalse("File exists in trash after expunge : " + trashFile, fs
+          .exists(trashFile));
+    }
+  }
+
+  private void verifyDeleteWithSkipTrash(Path path) throws Exception {
+    assertTrue(path + " file does not exist", fs.exists(path));
+
+    final Path trashFile = new Path(shell.getCurrentTrashDir(path) + "/" +
+        path);
+
+    String[] argv = new String[]{"-rm", "-r", "-skipTrash", path.toString()};
+    int res = ToolRunner.run(shell, argv);
+    assertEquals("rm failed", 0, res);
+    assertFalse("File in trash even with -skipTrash", fs.exists(trashFile));
+  }
+}


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


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

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

This closes #142


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

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

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


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


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


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

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


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

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

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


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


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


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

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


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

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

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


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

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


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


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

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


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

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

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


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


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


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

Posted by sj...@apache.org.
YARN-5699. Retrospect yarn entity fields which are publishing in events info fields. Contributed by Rohith Sharma K S.


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

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

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


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

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

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

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

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

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

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

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


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


[42/50] [abbrv] hadoop git commit: HDFS-10920. TestStorageMover#testNoSpaceDisk is failing intermittently. Contributed by Rakesh R

Posted by sj...@apache.org.
HDFS-10920. TestStorageMover#testNoSpaceDisk is failing intermittently. 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/d26a1bb9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d26a1bb9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d26a1bb9

Branch: refs/heads/HADOOP-13070
Commit: d26a1bb9d60f50763887d66399579bac7ca81982
Parents: c023c74
Author: Kai Zheng <ka...@intel.com>
Authored: Tue Oct 18 14:51:08 2016 +0600
Committer: Kai Zheng <ka...@intel.com>
Committed: Tue Oct 18 14:51:08 2016 +0600

----------------------------------------------------------------------
 .../hadoop/hdfs/server/mover/TestStorageMover.java | 17 ++++++++++-------
 1 file changed, 10 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d26a1bb9/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 92a70a0..1b5bd81 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
@@ -613,8 +613,10 @@ public class TestStorageMover {
   }
 
   private void waitForAllReplicas(int expectedReplicaNum, Path file,
-      DistributedFileSystem dfs) throws Exception {
-    for (int i = 0; i < 5; i++) {
+      DistributedFileSystem dfs, int retryCount) throws Exception {
+    LOG.info("Waiting for replicas count " + expectedReplicaNum
+        + ", file name: " + file);
+    for (int i = 0; i < retryCount; i++) {
       LocatedBlocks lbs = dfs.getClient().getLocatedBlocks(file.toString(), 0,
           BLOCK_SIZE);
       LocatedBlock lb = lbs.get(0);
@@ -664,7 +666,7 @@ public class TestStorageMover {
       for (int i = 0; i < 2; i++) {
         final Path p = new Path(pathPolicyMap.hot, "file" + i);
         DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, replication, 0L);
-        waitForAllReplicas(replication, p, test.dfs);
+        waitForAllReplicas(replication, p, test.dfs, 10);
       }
 
       // set all the DISK volume to full
@@ -679,16 +681,17 @@ public class TestStorageMover {
       final Replication r = test.getReplication(file0);
       final short newReplication = (short) 5;
       test.dfs.setReplication(file0, newReplication);
-      Thread.sleep(10000);
+      waitForAllReplicas(newReplication, file0, test.dfs, 10);
       test.verifyReplication(file0, r.disk, newReplication - r.disk);
 
       // test creating a cold file and then increase replication
       final Path p = new Path(pathPolicyMap.cold, "foo");
       DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, replication, 0L);
+      waitForAllReplicas(replication, p, test.dfs, 10);
       test.verifyReplication(p, 0, replication);
 
       test.dfs.setReplication(p, newReplication);
-      Thread.sleep(10000);
+      waitForAllReplicas(newReplication, p, test.dfs, 10);
       test.verifyReplication(p, 0, newReplication);
 
       //test move a hot file to warm
@@ -722,7 +725,7 @@ public class TestStorageMover {
       for (int i = 0; i < 2; i++) {
         final Path p = new Path(pathPolicyMap.cold, "file" + i);
         DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, replication, 0L);
-        waitForAllReplicas(replication, p, test.dfs);
+        waitForAllReplicas(replication, p, test.dfs, 10);
       }
 
       // set all the ARCHIVE volume to full
@@ -739,7 +742,7 @@ public class TestStorageMover {
 
         final short newReplication = (short) 5;
         test.dfs.setReplication(file0, newReplication);
-        Thread.sleep(10000);
+        waitForAllReplicas(r.archive, file0, test.dfs, 10);
 
         test.verifyReplication(file0, 0, r.archive);
       }


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


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

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


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

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

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


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

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


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


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

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


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

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

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


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

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

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

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

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

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

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

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


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


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

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


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

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

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


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


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


[36/50] [abbrv] hadoop git commit: HDFS-9390. Block management for maintenance states.

Posted by sj...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMaintenanceState.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMaintenanceState.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMaintenanceState.java
index 63617ad..c125f45 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMaintenanceState.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMaintenanceState.java
@@ -18,13 +18,19 @@
 package org.apache.hadoop.hdfs;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
@@ -32,6 +38,8 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.util.Time;
 import org.junit.Test;
@@ -40,13 +48,23 @@ import org.junit.Test;
  * This class tests node maintenance.
  */
 public class TestMaintenanceState extends AdminStatesBaseTest {
-  public static final Log LOG = LogFactory.getLog(TestMaintenanceState.class);
-  static private final long EXPIRATION_IN_MS = 500;
+  public static final Logger LOG =
+      LoggerFactory.getLogger(TestMaintenanceState.class);
+  static private final long EXPIRATION_IN_MS = 50;
+  private int minMaintenanceR =
+      DFSConfigKeys.DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_DEFAULT;
 
   public TestMaintenanceState() {
     setUseCombinedHostFileManager();
   }
 
+  void setMinMaintenanceR(int minMaintenanceR) {
+    this.minMaintenanceR = minMaintenanceR;
+    getConf().setInt(
+        DFSConfigKeys.DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_KEY,
+        minMaintenanceR);
+  }
+
   /**
    * Verify a node can transition from AdminStates.ENTERING_MAINTENANCE to
    * AdminStates.NORMAL.
@@ -55,21 +73,25 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
   public void testTakeNodeOutOfEnteringMaintenance() throws Exception {
     LOG.info("Starting testTakeNodeOutOfEnteringMaintenance");
     final int replicas = 1;
-    final int numNamenodes = 1;
-    final int numDatanodes = 1;
-    final Path file1 = new Path("/testTakeNodeOutOfEnteringMaintenance.dat");
+    final Path file = new Path("/testTakeNodeOutOfEnteringMaintenance.dat");
 
-    startCluster(numNamenodes, numDatanodes);
+    startCluster(1, 1);
 
-    FileSystem fileSys = getCluster().getFileSystem(0);
-    writeFile(fileSys, file1, replicas, 1);
+    final FileSystem fileSys = getCluster().getFileSystem(0);
+    final FSNamesystem ns = getCluster().getNamesystem(0);
+    writeFile(fileSys, file, replicas, 1);
 
-    DatanodeInfo nodeOutofService = takeNodeOutofService(0,
+    final DatanodeInfo nodeOutofService = takeNodeOutofService(0,
         null, Long.MAX_VALUE, null, AdminStates.ENTERING_MAINTENANCE);
 
+    // When node is in ENTERING_MAINTENANCE state, it can still serve read
+    // requests
+    assertNull(checkWithRetry(ns, fileSys, file, replicas, null,
+        nodeOutofService));
+
     putNodeInService(0, nodeOutofService.getDatanodeUuid());
 
-    cleanupFile(fileSys, file1);
+    cleanupFile(fileSys, file);
   }
 
   /**
@@ -80,23 +102,21 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
   public void testEnteringMaintenanceExpiration() throws Exception {
     LOG.info("Starting testEnteringMaintenanceExpiration");
     final int replicas = 1;
-    final int numNamenodes = 1;
-    final int numDatanodes = 1;
-    final Path file1 = new Path("/testTakeNodeOutOfEnteringMaintenance.dat");
+    final Path file = new Path("/testEnteringMaintenanceExpiration.dat");
 
-    startCluster(numNamenodes, numDatanodes);
+    startCluster(1, 1);
 
-    FileSystem fileSys = getCluster().getFileSystem(0);
-    writeFile(fileSys, file1, replicas, 1);
+    final FileSystem fileSys = getCluster().getFileSystem(0);
+    writeFile(fileSys, file, replicas, 1);
 
-    // expires in 500 milliseconds
-    DatanodeInfo nodeOutofService = takeNodeOutofService(0, null,
-        Time.monotonicNow() + EXPIRATION_IN_MS, null,
-        AdminStates.ENTERING_MAINTENANCE);
+    final DatanodeInfo nodeOutofService = takeNodeOutofService(0, null,
+        Long.MAX_VALUE, null, AdminStates.ENTERING_MAINTENANCE);
 
-    waitNodeState(nodeOutofService, AdminStates.NORMAL);
+    // Adjust the expiration.
+    takeNodeOutofService(0, nodeOutofService.getDatanodeUuid(),
+        Time.monotonicNow() + EXPIRATION_IN_MS, null, AdminStates.NORMAL);
 
-    cleanupFile(fileSys, file1);
+    cleanupFile(fileSys, file);
   }
 
   /**
@@ -106,20 +126,18 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
   public void testInvalidExpiration() throws Exception {
     LOG.info("Starting testInvalidExpiration");
     final int replicas = 1;
-    final int numNamenodes = 1;
-    final int numDatanodes = 1;
-    final Path file1 = new Path("/testTakeNodeOutOfEnteringMaintenance.dat");
+    final Path file = new Path("/testInvalidExpiration.dat");
 
-    startCluster(numNamenodes, numDatanodes);
+    startCluster(1, 1);
 
-    FileSystem fileSys = getCluster().getFileSystem(0);
-    writeFile(fileSys, file1, replicas, 1);
+    final FileSystem fileSys = getCluster().getFileSystem(0);
+    writeFile(fileSys, file, replicas, 1);
 
     // expiration has to be greater than Time.monotonicNow().
     takeNodeOutofService(0, null, Time.monotonicNow(), null,
         AdminStates.NORMAL);
 
-    cleanupFile(fileSys, file1);
+    cleanupFile(fileSys, file);
   }
 
   /**
@@ -129,18 +147,17 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
   @Test(timeout = 360000)
   public void testPutDeadNodeToMaintenance() throws Exception {
     LOG.info("Starting testPutDeadNodeToMaintenance");
-    final int numNamenodes = 1;
-    final int numDatanodes = 1;
     final int replicas = 1;
-    final Path file1 = new Path("/testPutDeadNodeToMaintenance.dat");
+    final Path file = new Path("/testPutDeadNodeToMaintenance.dat");
 
-    startCluster(numNamenodes, numDatanodes);
+    startCluster(1, 1);
 
-    FileSystem fileSys = getCluster().getFileSystem(0);
-    FSNamesystem ns = getCluster().getNamesystem(0);
-    writeFile(fileSys, file1, replicas, 1);
+    final FileSystem fileSys = getCluster().getFileSystem(0);
+    final FSNamesystem ns = getCluster().getNamesystem(0);
+    writeFile(fileSys, file, replicas, 1);
 
-    MiniDFSCluster.DataNodeProperties dnProp = getCluster().stopDataNode(0);
+    final MiniDFSCluster.DataNodeProperties dnProp =
+        getCluster().stopDataNode(0);
     DFSTestUtil.waitForDatanodeState(
         getCluster(), dnProp.datanode.getDatanodeUuid(), false, 20000);
 
@@ -153,7 +170,7 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
     assertEquals(deadInMaintenance + 1, ns.getNumInMaintenanceDeadDataNodes());
     assertEquals(liveInMaintenance, ns.getNumInMaintenanceLiveDataNodes());
 
-    cleanupFile(fileSys, file1);
+    cleanupFile(fileSys, file);
   }
 
   /**
@@ -164,16 +181,14 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
   @Test(timeout = 360000)
   public void testPutDeadNodeToMaintenanceWithExpiration() throws Exception {
     LOG.info("Starting testPutDeadNodeToMaintenanceWithExpiration");
-    final int numNamenodes = 1;
-    final int numDatanodes = 1;
-    final int replicas = 1;
-    final Path file1 = new Path("/testPutDeadNodeToMaintenance.dat");
+    final Path file =
+        new Path("/testPutDeadNodeToMaintenanceWithExpiration.dat");
 
-    startCluster(numNamenodes, numDatanodes);
+    startCluster(1, 1);
 
     FileSystem fileSys = getCluster().getFileSystem(0);
     FSNamesystem ns = getCluster().getNamesystem(0);
-    writeFile(fileSys, file1, replicas, 1);
+    writeFile(fileSys, file, 1, 1);
 
     MiniDFSCluster.DataNodeProperties dnProp = getCluster().stopDataNode(0);
     DFSTestUtil.waitForDatanodeState(
@@ -184,16 +199,17 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
 
     DatanodeInfo nodeOutofService = takeNodeOutofService(0,
         dnProp.datanode.getDatanodeUuid(),
-        Time.monotonicNow() + EXPIRATION_IN_MS, null,
-        AdminStates.IN_MAINTENANCE);
+        Long.MAX_VALUE, null, AdminStates.IN_MAINTENANCE);
 
-    waitNodeState(nodeOutofService, AdminStates.NORMAL);
+    // Adjust the expiration.
+    takeNodeOutofService(0, nodeOutofService.getDatanodeUuid(),
+        Time.monotonicNow() + EXPIRATION_IN_MS, null, AdminStates.NORMAL);
 
     // no change
     assertEquals(deadInMaintenance, ns.getNumInMaintenanceDeadDataNodes());
     assertEquals(liveInMaintenance, ns.getNumInMaintenanceLiveDataNodes());
 
-    cleanupFile(fileSys, file1);
+    cleanupFile(fileSys, file);
   }
 
   /**
@@ -202,15 +218,12 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
   @Test(timeout = 360000)
   public void testTransitionFromDecommissioned() throws IOException {
     LOG.info("Starting testTransitionFromDecommissioned");
-    final int numNamenodes = 1;
-    final int numDatanodes = 4;
-    final int replicas = 3;
-    final Path file1 = new Path("/testTransitionFromDecommissioned.dat");
+    final Path file = new Path("/testTransitionFromDecommissioned.dat");
 
-    startCluster(numNamenodes, numDatanodes);
+    startCluster(1, 4);
 
-    FileSystem fileSys = getCluster().getFileSystem(0);
-    writeFile(fileSys, file1, replicas, 1);
+    final FileSystem fileSys = getCluster().getFileSystem(0);
+    writeFile(fileSys, file, 3, 1);
 
     DatanodeInfo nodeOutofService = takeNodeOutofService(0, null, 0, null,
         AdminStates.DECOMMISSIONED);
@@ -218,7 +231,7 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
     takeNodeOutofService(0, nodeOutofService.getDatanodeUuid(), Long.MAX_VALUE,
         null, AdminStates.IN_MAINTENANCE);
 
-    cleanupFile(fileSys, file1);
+    cleanupFile(fileSys, file);
   }
 
   /**
@@ -228,34 +241,33 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
   @Test(timeout = 360000)
   public void testTransitionFromDecommissionedAndExpired() throws IOException {
     LOG.info("Starting testTransitionFromDecommissionedAndExpired");
-    final int numNamenodes = 1;
-    final int numDatanodes = 4;
-    final int replicas = 3;
-    final Path file1 = new Path("/testTransitionFromDecommissioned.dat");
+    final Path file =
+        new Path("/testTransitionFromDecommissionedAndExpired.dat");
 
-    startCluster(numNamenodes, numDatanodes);
+    startCluster(1, 4);
 
-    FileSystem fileSys = getCluster().getFileSystem(0);
-    writeFile(fileSys, file1, replicas, 1);
+    final FileSystem fileSys = getCluster().getFileSystem(0);
+    writeFile(fileSys, file, 3, 1);
 
-    DatanodeInfo nodeOutofService = takeNodeOutofService(0, null, 0, null,
-        AdminStates.DECOMMISSIONED);
+    final DatanodeInfo nodeOutofService = takeNodeOutofService(0, null, 0,
+        null, AdminStates.DECOMMISSIONED);
 
     takeNodeOutofService(0, nodeOutofService.getDatanodeUuid(),
-        Time.monotonicNow() + EXPIRATION_IN_MS, null,
-        AdminStates.IN_MAINTENANCE);
+        Long.MAX_VALUE, null, AdminStates.IN_MAINTENANCE);
 
-    waitNodeState(nodeOutofService, AdminStates.NORMAL);
+    // Adjust the expiration.
+    takeNodeOutofService(0, nodeOutofService.getDatanodeUuid(),
+        Time.monotonicNow() + EXPIRATION_IN_MS, null, AdminStates.NORMAL);
 
-    cleanupFile(fileSys, file1);
+    cleanupFile(fileSys, file);
   }
 
   /**
    * When a node is put to maintenance, it first transitions to
    * AdminStates.ENTERING_MAINTENANCE. It makes sure all blocks have minimal
    * replication before it can be transitioned to AdminStates.IN_MAINTENANCE.
-   * If node becomes dead when it is in AdminStates.ENTERING_MAINTENANCE, admin
-   * state should stay in AdminStates.ENTERING_MAINTENANCE state.
+   * If node becomes dead when it is in AdminStates.ENTERING_MAINTENANCE, it
+   * should stay in AdminStates.ENTERING_MAINTENANCE state.
    */
   @Test(timeout = 360000)
   public void testNodeDeadWhenInEnteringMaintenance() throws Exception {
@@ -263,16 +275,16 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
     final int numNamenodes = 1;
     final int numDatanodes = 1;
     final int replicas = 1;
-    final Path file1 = new Path("/testNodeDeadWhenInEnteringMaintenance.dat");
+    final Path file = new Path("/testNodeDeadWhenInEnteringMaintenance.dat");
 
     startCluster(numNamenodes, numDatanodes);
 
-    FileSystem fileSys = getCluster().getFileSystem(0);
-    FSNamesystem ns = getCluster().getNamesystem(0);
-    writeFile(fileSys, file1, replicas, 1);
+    final FileSystem fileSys = getCluster().getFileSystem(0);
+    final FSNamesystem ns = getCluster().getNamesystem(0);
+    writeFile(fileSys, file, replicas, 1);
 
     DatanodeInfo nodeOutofService = takeNodeOutofService(0,
-        getFirstBlockFirstReplicaUuid(fileSys, file1), Long.MAX_VALUE, null,
+        getFirstBlockFirstReplicaUuid(fileSys, file), Long.MAX_VALUE, null,
         AdminStates.ENTERING_MAINTENANCE);
     assertEquals(1, ns.getNumEnteringMaintenanceDataNodes());
 
@@ -281,30 +293,627 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
     DFSTestUtil.waitForDatanodeState(
         getCluster(), nodeOutofService.getDatanodeUuid(), false, 20000);
     DFSClient client = getDfsClient(0);
-    assertEquals("maintenance node shouldn't be alive", numDatanodes - 1,
+    assertEquals("maintenance node shouldn't be live", numDatanodes - 1,
         client.datanodeReport(DatanodeReportType.LIVE).length);
+    assertEquals(1, ns.getNumEnteringMaintenanceDataNodes());
 
     getCluster().restartDataNode(dnProp, true);
     getCluster().waitActive();
     waitNodeState(nodeOutofService, AdminStates.ENTERING_MAINTENANCE);
     assertEquals(1, ns.getNumEnteringMaintenanceDataNodes());
+    assertEquals("maintenance node should be live", numDatanodes,
+        client.datanodeReport(DatanodeReportType.LIVE).length);
+
+    cleanupFile(fileSys, file);
+  }
+
+  /**
+   * When a node is put to maintenance, it first transitions to
+   * AdminStates.ENTERING_MAINTENANCE. It makes sure all blocks have
+   * been properly replicated before it can be transitioned to
+   * AdminStates.IN_MAINTENANCE. The expected replication count takes
+   * DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_KEY and
+   * its file's replication factor into account.
+   */
+  @Test(timeout = 360000)
+  public void testExpectedReplications() throws IOException {
+    LOG.info("Starting testExpectedReplications");
+    testExpectedReplication(1);
+    testExpectedReplication(2);
+    testExpectedReplication(3);
+    testExpectedReplication(4);
+  }
+
+  private void testExpectedReplication(int replicationFactor)
+      throws IOException {
+    testExpectedReplication(replicationFactor,
+        Math.max(replicationFactor - 1, this.minMaintenanceR));
+  }
+
+  private void testExpectedReplication(int replicationFactor,
+      int expectedReplicasInRead) throws IOException {
+    startCluster(1, 5);
+
+    final Path file = new Path("/testExpectedReplication.dat");
+
+    final FileSystem fileSys = getCluster().getFileSystem(0);
+    final FSNamesystem ns = getCluster().getNamesystem(0);
+
+    writeFile(fileSys, file, replicationFactor, 1);
+
+    DatanodeInfo nodeOutofService = takeNodeOutofService(0,
+        getFirstBlockFirstReplicaUuid(fileSys, file), Long.MAX_VALUE,
+        null, AdminStates.IN_MAINTENANCE);
+
+    // The block should be replicated to another datanode to meet
+    // expected replication count.
+    assertNull(checkWithRetry(ns, fileSys, file, expectedReplicasInRead,
+        nodeOutofService));
+
+    cleanupFile(fileSys, file);
+  }
+
+  /**
+   * Verify a node can transition directly to AdminStates.IN_MAINTENANCE when
+   * DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_KEY is set to zero.
+   */
+  @Test(timeout = 360000)
+  public void testZeroMinMaintenanceReplication() throws Exception {
+    LOG.info("Starting testZeroMinMaintenanceReplication");
+    setMinMaintenanceR(0);
+    startCluster(1, 1);
+
+    final Path file = new Path("/testZeroMinMaintenanceReplication.dat");
+    final int replicas = 1;
+
+    FileSystem fileSys = getCluster().getFileSystem(0);
+    writeFile(fileSys, file, replicas, 1);
+
+    takeNodeOutofService(0, null, Long.MAX_VALUE, null,
+        AdminStates.IN_MAINTENANCE);
+
+    cleanupFile(fileSys, file);
+  }
+
+  /**
+   * Verify a node can transition directly to AdminStates.IN_MAINTENANCE when
+   * DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_KEY is set to zero. Then later
+   * transition to NORMAL after maintenance expiration.
+   */
+  @Test(timeout = 360000)
+  public void testZeroMinMaintenanceReplicationWithExpiration()
+      throws Exception {
+    LOG.info("Starting testZeroMinMaintenanceReplicationWithExpiration");
+    setMinMaintenanceR(0);
+    startCluster(1, 1);
+
+    final Path file =
+        new Path("/testZeroMinMaintenanceReplicationWithExpiration.dat");
+
+    FileSystem fileSys = getCluster().getFileSystem(0);
+    writeFile(fileSys, file, 1, 1);
+
+    DatanodeInfo nodeOutofService = takeNodeOutofService(0, null,
+        Long.MAX_VALUE, null, AdminStates.IN_MAINTENANCE);
+
+    // Adjust the expiration.
+    takeNodeOutofService(0, nodeOutofService.getDatanodeUuid(),
+        Time.monotonicNow() + EXPIRATION_IN_MS, null, AdminStates.NORMAL);
+
+    cleanupFile(fileSys, file);
+  }
+
+  /**
+   * Transition from IN_MAINTENANCE to DECOMMISSIONED.
+   */
+  @Test(timeout = 360000)
+  public void testTransitionToDecommission() throws IOException {
+    LOG.info("Starting testTransitionToDecommission");
+    final int numNamenodes = 1;
+    final int numDatanodes = 4;
+    startCluster(numNamenodes, numDatanodes);
+
+    final Path file = new Path("testTransitionToDecommission.dat");
+    final int replicas = 3;
+
+    FileSystem fileSys = getCluster().getFileSystem(0);
+    FSNamesystem ns = getCluster().getNamesystem(0);
+
+    writeFile(fileSys, file, replicas, 1);
+
+    DatanodeInfo nodeOutofService = takeNodeOutofService(0,
+        getFirstBlockFirstReplicaUuid(fileSys, file), Long.MAX_VALUE, null,
+        AdminStates.IN_MAINTENANCE);
+
+    DFSClient client = getDfsClient(0);
+    assertEquals("All datanodes must be alive", numDatanodes,
+        client.datanodeReport(DatanodeReportType.LIVE).length);
+
+    // test 1, verify the replica in IN_MAINTENANCE state isn't in LocatedBlock
+    assertNull(checkWithRetry(ns, fileSys, file, replicas - 1,
+        nodeOutofService));
+
+    takeNodeOutofService(0, nodeOutofService.getDatanodeUuid(), 0, null,
+        AdminStates.DECOMMISSIONED);
+
+    // test 2 after decommission has completed, the replication count is
+    // replicas + 1 which includes the decommissioned node.
+    assertNull(checkWithRetry(ns, fileSys, file, replicas + 1, null));
+
+    // test 3, put the node in service, replication count should restore.
+    putNodeInService(0, nodeOutofService.getDatanodeUuid());
+    assertNull(checkWithRetry(ns, fileSys, file, replicas, null));
+
+    cleanupFile(fileSys, file);
+  }
+
+  /**
+   * Transition from decommissioning state to maintenance state.
+   */
+  @Test(timeout = 360000)
+  public void testTransitionFromDecommissioning() throws IOException {
+    LOG.info("Starting testTransitionFromDecommissioning");
+    startCluster(1, 3);
+
+    final Path file = new Path("/testTransitionFromDecommissioning.dat");
+    final int replicas = 3;
+
+    final FileSystem fileSys = getCluster().getFileSystem(0);
+    final FSNamesystem ns = getCluster().getNamesystem(0);
+
+    writeFile(fileSys, file, replicas);
+
+    final DatanodeInfo nodeOutofService = takeNodeOutofService(0, null, 0,
+        null, AdminStates.DECOMMISSION_INPROGRESS);
+
+    takeNodeOutofService(0, nodeOutofService.getDatanodeUuid(), Long.MAX_VALUE,
+        null, AdminStates.IN_MAINTENANCE);
+
+    assertNull(checkWithRetry(ns, fileSys, file, replicas - 1,
+        nodeOutofService));
+
+    cleanupFile(fileSys, file);
+  }
+
+
+  /**
+   * First put a node in maintenance, then put a different node
+   * in decommission. Make sure decommission process take
+   * maintenance replica into account.
+   */
+  @Test(timeout = 360000)
+  public void testDecommissionDifferentNodeAfterMaintenances()
+      throws Exception {
+    testDecommissionDifferentNodeAfterMaintenance(2);
+    testDecommissionDifferentNodeAfterMaintenance(3);
+    testDecommissionDifferentNodeAfterMaintenance(4);
+  }
+
+  private void testDecommissionDifferentNodeAfterMaintenance(int repl)
+      throws Exception {
+    startCluster(1, 5);
+
+    final Path file =
+        new Path("/testDecommissionDifferentNodeAfterMaintenance.dat");
+
+    final FileSystem fileSys = getCluster().getFileSystem(0);
+    final FSNamesystem ns = getCluster().getNamesystem(0);
+
+    writeFile(fileSys, file, repl, 1);
+    final DatanodeInfo[] nodes = getFirstBlockReplicasDatanodeInfos(fileSys,
+        file);
+    String maintenanceDNUuid = nodes[0].getDatanodeUuid();
+    String decommissionDNUuid = nodes[1].getDatanodeUuid();
+    DatanodeInfo maintenanceDN = takeNodeOutofService(0, maintenanceDNUuid,
+        Long.MAX_VALUE, null, null, AdminStates.IN_MAINTENANCE);
 
-    cleanupFile(fileSys, file1);
+    Map<DatanodeInfo, Long> maintenanceNodes = new HashMap<>();
+    maintenanceNodes.put(nodes[0], Long.MAX_VALUE);
+    takeNodeOutofService(0, decommissionDNUuid, 0, null, maintenanceNodes,
+        AdminStates.DECOMMISSIONED);
+    // Out of the replicas returned, one is the decommissioned node.
+    assertNull(checkWithRetry(ns, fileSys, file, repl, maintenanceDN));
+
+    putNodeInService(0, maintenanceDN);
+    assertNull(checkWithRetry(ns, fileSys, file, repl + 1, null));
+
+    cleanupFile(fileSys, file);
+  }
+
+
+  @Test(timeout = 360000)
+  public void testChangeReplicationFactors() throws IOException {
+    // Prior to any change, there is 1 maintenance node and 2 live nodes.
+
+    // Replication factor is adjusted from 3 to 4.
+    // After the change, given 1 maintenance + 2 live is less than the
+    // newFactor, one live nodes will be added.
+    testChangeReplicationFactor(3, 4, 3);
+
+    // Replication factor is adjusted from 3 to 2.
+    // After the change, given 2 live nodes is the same as the newFactor,
+    // no live nodes will be invalidated.
+    testChangeReplicationFactor(3, 2, 2);
+
+    // Replication factor is adjusted from 3 to 1.
+    // After the change, given 2 live nodes is greater than the newFactor,
+    // one live nodes will be invalidated.
+    testChangeReplicationFactor(3, 1, 1);
+  }
+
+  /**
+   * After the change of replication factor, # of live replicas <=
+   * the new replication factor.
+   */
+  private void testChangeReplicationFactor(int oldFactor, int newFactor,
+      int expectedLiveReplicas) throws IOException {
+    LOG.info("Starting testChangeReplicationFactor {} {} {}",
+        oldFactor, newFactor, expectedLiveReplicas);
+    startCluster(1, 5);
+
+    final Path file = new Path("/testChangeReplicationFactor.dat");
+
+    final FileSystem fileSys = getCluster().getFileSystem(0);
+    final FSNamesystem ns = getCluster().getNamesystem(0);
+
+    writeFile(fileSys, file, oldFactor, 1);
+
+    final DatanodeInfo nodeOutofService = takeNodeOutofService(0,
+        getFirstBlockFirstReplicaUuid(fileSys, file), Long.MAX_VALUE, null,
+        AdminStates.IN_MAINTENANCE);
+
+    // Verify that the nodeOutofService remains in blocksMap and
+    // # of live replicas For read operation is expected.
+    assertNull(checkWithRetry(ns, fileSys, file, oldFactor - 1,
+        nodeOutofService));
+
+    final DFSClient client = getDfsClient(0);
+    client.setReplication(file.toString(), (short)newFactor);
+
+    // Verify that the nodeOutofService remains in blocksMap and
+    // # of live replicas for read operation.
+    assertNull(checkWithRetry(ns, fileSys, file, expectedLiveReplicas,
+        nodeOutofService));
+
+    putNodeInService(0, nodeOutofService.getDatanodeUuid());
+    assertNull(checkWithRetry(ns, fileSys, file, newFactor, null));
+
+    cleanupFile(fileSys, file);
+  }
+
+
+  /**
+   * Verify the following scenario.
+   * a. Put a live node to maintenance => 1 maintenance, 2 live.
+   * b. The maintenance node becomes dead => block map still has 1 maintenance,
+   *    2 live.
+   * c. Take the node out of maintenance => NN should schedule the replication
+   *    and end up with 3 live.
+   */
+  @Test(timeout = 360000)
+  public void testTakeDeadNodeOutOfMaintenance() throws Exception {
+    LOG.info("Starting testTakeDeadNodeOutOfMaintenance");
+    final int numNamenodes = 1;
+    final int numDatanodes = 4;
+    startCluster(numNamenodes, numDatanodes);
+
+    final Path file = new Path("/testTakeDeadNodeOutOfMaintenance.dat");
+    final int replicas = 3;
+
+    final FileSystem fileSys = getCluster().getFileSystem(0);
+    final FSNamesystem ns = getCluster().getNamesystem(0);
+    writeFile(fileSys, file, replicas, 1);
+
+    final DatanodeInfo nodeOutofService = takeNodeOutofService(0,
+        getFirstBlockFirstReplicaUuid(fileSys, file), Long.MAX_VALUE, null,
+        AdminStates.IN_MAINTENANCE);
+
+    assertNull(checkWithRetry(ns, fileSys, file, replicas - 1,
+        nodeOutofService));
+
+    final DFSClient client = getDfsClient(0);
+    assertEquals("All datanodes must be alive", numDatanodes,
+        client.datanodeReport(DatanodeReportType.LIVE).length);
+
+    getCluster().stopDataNode(nodeOutofService.getXferAddr());
+    DFSTestUtil.waitForDatanodeState(
+        getCluster(), nodeOutofService.getDatanodeUuid(), false, 20000);
+    assertEquals("maintenance node shouldn't be alive", numDatanodes - 1,
+        client.datanodeReport(DatanodeReportType.LIVE).length);
+
+    // Dead maintenance node's blocks should remain in block map.
+    assertNull(checkWithRetry(ns, fileSys, file, replicas - 1,
+        nodeOutofService));
+
+    // When dead maintenance mode is transitioned to out of maintenance mode,
+    // its blocks should be removed from block map.
+    // This will then trigger replication to restore the live replicas back
+    // to replication factor.
+    putNodeInService(0, nodeOutofService.getDatanodeUuid());
+    assertNull(checkWithRetry(ns, fileSys, file, replicas, nodeOutofService,
+        null));
+
+    cleanupFile(fileSys, file);
   }
 
-  static protected String getFirstBlockFirstReplicaUuid(FileSystem fileSys,
+
+  /**
+   * Verify the following scenario.
+   * a. Put a live node to maintenance => 1 maintenance, 2 live.
+   * b. The maintenance node becomes dead => block map still has 1 maintenance,
+   *    2 live.
+   * c. Restart nn => block map only has 2 live => restore the 3 live.
+   * d. Restart the maintenance dn => 1 maintenance, 3 live.
+   * e. Take the node out of maintenance => over replication => 3 live.
+   */
+  @Test(timeout = 360000)
+  public void testWithNNAndDNRestart() throws Exception {
+    LOG.info("Starting testWithNNAndDNRestart");
+    final int numNamenodes = 1;
+    final int numDatanodes = 4;
+    startCluster(numNamenodes, numDatanodes);
+
+    final Path file = new Path("/testWithNNAndDNRestart.dat");
+    final int replicas = 3;
+
+    final FileSystem fileSys = getCluster().getFileSystem(0);
+    FSNamesystem ns = getCluster().getNamesystem(0);
+    writeFile(fileSys, file, replicas, 1);
+
+    DatanodeInfo nodeOutofService = takeNodeOutofService(0,
+        getFirstBlockFirstReplicaUuid(fileSys, file), Long.MAX_VALUE, null,
+        AdminStates.IN_MAINTENANCE);
+
+    assertNull(checkWithRetry(ns, fileSys, file, replicas - 1,
+        nodeOutofService));
+
+    DFSClient client = getDfsClient(0);
+    assertEquals("All datanodes must be alive", numDatanodes,
+        client.datanodeReport(DatanodeReportType.LIVE).length);
+
+    MiniDFSCluster.DataNodeProperties dnProp =
+        getCluster().stopDataNode(nodeOutofService.getXferAddr());
+    DFSTestUtil.waitForDatanodeState(
+        getCluster(), nodeOutofService.getDatanodeUuid(), false, 20000);
+    assertEquals("maintenance node shouldn't be alive", numDatanodes - 1,
+        client.datanodeReport(DatanodeReportType.LIVE).length);
+
+    // Dead maintenance node's blocks should remain in block map.
+    assertNull(checkWithRetry(ns, fileSys, file, replicas - 1,
+        nodeOutofService));
+
+    // restart nn, nn will restore 3 live replicas given it doesn't
+    // know the maintenance node has the replica.
+    getCluster().restartNameNode(0);
+    ns = getCluster().getNamesystem(0);
+    assertNull(checkWithRetry(ns, fileSys, file, replicas, null));
+
+    // restart dn, nn has 1 maintenance replica and 3 live replicas.
+    getCluster().restartDataNode(dnProp, true);
+    getCluster().waitActive();
+    assertNull(checkWithRetry(ns, fileSys, file, replicas, nodeOutofService));
+
+    // Put the node in service, a redundant replica should be removed.
+    putNodeInService(0, nodeOutofService.getDatanodeUuid());
+    assertNull(checkWithRetry(ns, fileSys, file, replicas, null));
+
+    cleanupFile(fileSys, file);
+  }
+
+
+  /**
+   * Machine under maintenance state won't be chosen for new block allocation.
+   */
+  @Test(timeout = 3600000)
+  public void testWriteAfterMaintenance() throws IOException {
+    LOG.info("Starting testWriteAfterMaintenance");
+    startCluster(1, 3);
+
+    final Path file = new Path("/testWriteAfterMaintenance.dat");
+    int replicas = 3;
+
+    final FileSystem fileSys = getCluster().getFileSystem(0);
+    FSNamesystem ns = getCluster().getNamesystem(0);
+
+    final DatanodeInfo nodeOutofService = takeNodeOutofService(0, null,
+        Long.MAX_VALUE, null, AdminStates.IN_MAINTENANCE);
+
+    writeFile(fileSys, file, replicas, 2);
+
+    // Verify nodeOutofService wasn't chosen for write operation.
+    assertNull(checkWithRetry(ns, fileSys, file, replicas - 1,
+        nodeOutofService, null));
+
+    // Put the node back to service, live replicas should be restored.
+    putNodeInService(0, nodeOutofService.getDatanodeUuid());
+    assertNull(checkWithRetry(ns, fileSys, file, replicas, null));
+
+    cleanupFile(fileSys, file);
+  }
+
+  /**
+   * A node has blocks under construction when it is put to maintenance.
+   * Given there are minReplication replicas somewhere else,
+   * it can be transitioned to AdminStates.IN_MAINTENANCE.
+   */
+  @Test(timeout = 360000)
+  public void testEnterMaintenanceWhenFileOpen() throws Exception {
+    LOG.info("Starting testEnterMaintenanceWhenFileOpen");
+    startCluster(1, 3);
+
+    final Path file = new Path("/testEnterMaintenanceWhenFileOpen.dat");
+
+    final FileSystem fileSys = getCluster().getFileSystem(0);
+    writeIncompleteFile(fileSys, file, (short)3, (short)2);
+
+    takeNodeOutofService(0, null, Long.MAX_VALUE, null,
+        AdminStates.IN_MAINTENANCE);
+
+    cleanupFile(fileSys, file);
+  }
+
+  /**
+   * Machine under maintenance state won't be chosen for invalidation.
+   */
+  @Test(timeout = 360000)
+  public void testInvalidation() throws IOException {
+    LOG.info("Starting testInvalidation");
+    int numNamenodes = 1;
+    int numDatanodes = 3;
+    startCluster(numNamenodes, numDatanodes);
+
+    Path file = new Path("/testInvalidation.dat");
+    int replicas = 3;
+
+    FileSystem fileSys = getCluster().getFileSystem(0);
+    FSNamesystem ns = getCluster().getNamesystem(0);
+
+    writeFile(fileSys, file, replicas);
+
+    DatanodeInfo nodeOutofService = takeNodeOutofService(0, null,
+        Long.MAX_VALUE, null, AdminStates.IN_MAINTENANCE);
+
+    DFSClient client = getDfsClient(0);
+    client.setReplication(file.toString(), (short) 1);
+
+    // Verify the nodeOutofService remains in blocksMap.
+    assertNull(checkWithRetry(ns, fileSys, file, 1, nodeOutofService));
+
+    // Restart NN and verify the nodeOutofService remains in blocksMap.
+    getCluster().restartNameNode(0);
+    ns = getCluster().getNamesystem(0);
+    assertNull(checkWithRetry(ns, fileSys, file, 1, nodeOutofService));
+
+    cleanupFile(fileSys, file);
+  }
+
+  static String getFirstBlockFirstReplicaUuid(FileSystem fileSys,
       Path name) throws IOException {
+    DatanodeInfo[] nodes = getFirstBlockReplicasDatanodeInfos(fileSys, name);
+    if (nodes != null && nodes.length != 0) {
+      return nodes[0].getDatanodeUuid();
+    } else {
+      return null;
+    }
+  }
+
+  /*
+  * Verify that the number of replicas are as expected for each block in
+  * the given file.
+  *
+  * @return - null if no failure found, else an error message string.
+  */
+  static String checkFile(FSNamesystem ns, FileSystem fileSys,
+      Path name, int repl, DatanodeInfo expectedExcludedNode,
+      DatanodeInfo expectedMaintenanceNode) throws IOException {
     // need a raw stream
     assertTrue("Not HDFS:"+fileSys.getUri(),
         fileSys instanceof DistributedFileSystem);
     HdfsDataInputStream dis = (HdfsDataInputStream)fileSys.open(name);
+    BlockManager bm = ns.getBlockManager();
     Collection<LocatedBlock> dinfo = dis.getAllBlocks();
+    String output;
     for (LocatedBlock blk : dinfo) { // for each block
       DatanodeInfo[] nodes = blk.getLocations();
-      if (nodes.length > 0) {
-        return nodes[0].getDatanodeUuid();
+      for (int j = 0; j < nodes.length; j++) { // for each replica
+        if (expectedExcludedNode != null &&
+            nodes[j].equals(expectedExcludedNode)) {
+          //excluded node must not be in LocatedBlock.
+          output = "For block " + blk.getBlock() + " replica on " +
+              nodes[j] + " found in LocatedBlock.";
+          LOG.info(output);
+          return output;
+        } else {
+          if (nodes[j].isInMaintenance()) {
+            //IN_MAINTENANCE node must not be in LocatedBlock.
+            output = "For block " + blk.getBlock() + " replica on " +
+                nodes[j] + " which is in maintenance state.";
+            LOG.info(output);
+            return output;
+          }
+        }
+      }
+      if (repl != nodes.length) {
+        output = "Wrong number of replicas for block " + blk.getBlock() +
+            ": expected " + repl + ", got " + nodes.length + " ,";
+        for (int j = 0; j < nodes.length; j++) { // for each replica
+          output += nodes[j] + ",";
+        }
+        output += "pending block # " + ns.getPendingReplicationBlocks() + " ,";
+        output += "under replicated # " + ns.getUnderReplicatedBlocks() + " ,";
+        if (expectedExcludedNode != null) {
+          output += "excluded node " + expectedExcludedNode;
+        }
+
+        LOG.info(output);
+        return output;
+      }
+
+      // Verify it has the expected maintenance node
+      Iterator<DatanodeStorageInfo> storageInfoIter =
+          bm.getStorages(blk.getBlock().getLocalBlock()).iterator();
+      List<DatanodeInfo> maintenanceNodes = new ArrayList<>();
+      while (storageInfoIter.hasNext()) {
+        DatanodeInfo node = storageInfoIter.next().getDatanodeDescriptor();
+        if (node.isMaintenance()) {
+          maintenanceNodes.add(node);
+        }
+      }
+
+      if (expectedMaintenanceNode != null) {
+        if (!maintenanceNodes.contains(expectedMaintenanceNode)) {
+          output = "No maintenance replica on " + expectedMaintenanceNode;
+          LOG.info(output);
+          return output;
+        }
+      } else {
+        if (maintenanceNodes.size() != 0) {
+          output = "Has maintenance replica(s)";
+          LOG.info(output);
+          return output;
+        }
       }
     }
     return null;
   }
+
+  static String checkWithRetry(FSNamesystem ns, FileSystem fileSys,
+      Path name, int repl, DatanodeInfo inMaintenanceNode)
+          throws IOException {
+    return checkWithRetry(ns, fileSys, name, repl, inMaintenanceNode,
+        inMaintenanceNode);
+  }
+
+  static String checkWithRetry(FSNamesystem ns, FileSystem fileSys,
+      Path name, int repl, DatanodeInfo excludedNode,
+      DatanodeInfo underMaintenanceNode) throws IOException {
+    int tries = 0;
+    String output = null;
+    while (tries++ < 200) {
+      try {
+        Thread.sleep(100);
+        output = checkFile(ns, fileSys, name, repl, excludedNode,
+            underMaintenanceNode);
+        if (output == null) {
+          break;
+        }
+      } catch (InterruptedException ie) {
+      }
+    }
+    return output;
+  }
+
+  static private DatanodeInfo[] getFirstBlockReplicasDatanodeInfos(
+      FileSystem fileSys, Path name) throws IOException {
+    // need a raw stream
+    assertTrue("Not HDFS:"+fileSys.getUri(),
+        fileSys instanceof DistributedFileSystem);
+    HdfsDataInputStream dis = (HdfsDataInputStream)fileSys.open(name);
+    Collection<LocatedBlock> dinfo = dis.getAllBlocks();
+    if (dinfo.iterator().hasNext()) { // for the first block
+      return dinfo.iterator().next().getLocations();
+    } else {
+      return null;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/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 2c7c720..00bea1c 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
@@ -415,9 +415,10 @@ public class TestBlockManager {
       throws Exception {
     assertEquals(0, bm.numOfUnderReplicatedBlocks());
     BlockInfo block = addBlockOnNodes(testIndex, origNodes);
-    assertFalse(bm.isNeededReconstruction(block, bm.countLiveNodes(block)));
+    assertFalse(bm.isNeededReconstruction(block,
+        bm.countNodes(block, fsn.isInStartupSafeMode())));
   }
-  
+
   @Test(timeout = 60000)
   public void testNeededReconstructionWhileAppending() throws IOException {
     Configuration conf = new HdfsConfiguration();
@@ -458,7 +459,8 @@ public class TestBlockManager {
         namenode.updatePipeline(clientName, oldBlock, newBlock,
             newLocatedBlock.getLocations(), newLocatedBlock.getStorageIDs());
         BlockInfo bi = bm.getStoredBlock(newBlock.getLocalBlock());
-        assertFalse(bm.isNeededReconstruction(bi, bm.countLiveNodes(bi)));
+        assertFalse(bm.isNeededReconstruction(bi, bm.countNodes(bi,
+            cluster.getNamesystem().isInStartupSafeMode())));
       } finally {
         IOUtils.closeStream(out);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
index 6bb6040..7f1cc9a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
@@ -26,17 +26,16 @@ import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
-import java.util.Random;
 
 import org.apache.commons.io.output.ByteArrayOutputStream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hdfs.AdminStatesBaseTest;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -111,47 +110,22 @@ public class TestDecommissioningStatus {
     if(cluster != null) cluster.shutdown();
   }
 
-  private FSDataOutputStream writeIncompleteFile(FileSystem fileSys, Path name,
-      short repl) throws IOException {
-    // create and write a file that contains three blocks of data
-    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096), repl,
-        blockSize);
-    byte[] buffer = new byte[fileSize];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    // need to make sure that we actually write out both file blocks
-    // (see FSOutputSummer#flush)
-    stm.flush();
-    // Do not close stream, return it
-    // so that it is not garbage collected
-    return stm;
-  }
-  
-  static private void cleanupFile(FileSystem fileSys, Path name)
-      throws IOException {
-    assertTrue(fileSys.exists(name));
-    fileSys.delete(name, true);
-    assertTrue(!fileSys.exists(name));
-  }
-
   /*
    * Decommissions the node at the given index
    */
-  private String decommissionNode(FSNamesystem namesystem, DFSClient client,
+  private String decommissionNode(DFSClient client,
       int nodeIndex) throws IOException {
     DatanodeInfo[] info = client.datanodeReport(DatanodeReportType.LIVE);
 
     String nodename = info[nodeIndex].getXferAddr();
-    decommissionNode(namesystem, nodename);
+    decommissionNode(nodename);
     return nodename;
   }
 
   /*
    * Decommissions the node by name
    */
-  private void decommissionNode(FSNamesystem namesystem, String dnName)
+  private void decommissionNode(String dnName)
       throws IOException {
     System.out.println("Decommissioning node: " + dnName);
 
@@ -166,14 +140,14 @@ public class TestDecommissioningStatus {
       int expectedUnderRepInOpenFiles) {
     assertEquals("Unexpected num under-replicated blocks",
         expectedUnderRep,
-        decommNode.decommissioningStatus.getUnderReplicatedBlocks());
+        decommNode.getLeavingServiceStatus().getUnderReplicatedBlocks());
     assertEquals("Unexpected number of decom-only replicas",
         expectedDecommissionOnly,
-        decommNode.decommissioningStatus.getDecommissionOnlyReplicas());
+        decommNode.getLeavingServiceStatus().getOutOfServiceOnlyReplicas());
     assertEquals(
         "Unexpected number of replicas in under-replicated open files",
         expectedUnderRepInOpenFiles,
-        decommNode.decommissioningStatus.getUnderReplicatedInOpenFiles());
+        decommNode.getLeavingServiceStatus().getUnderReplicatedInOpenFiles());
   }
 
   private void checkDFSAdminDecommissionStatus(
@@ -237,13 +211,14 @@ public class TestDecommissioningStatus {
     short replicas = numDatanodes;
     //
     // Decommission one node. Verify the decommission status
-    // 
+    //
     Path file1 = new Path("decommission.dat");
     DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
         replicas, seed);
 
     Path file2 = new Path("decommission1.dat");
-    FSDataOutputStream st1 = writeIncompleteFile(fileSys, file2, replicas);
+    FSDataOutputStream st1 = AdminStatesBaseTest.writeIncompleteFile(fileSys,
+        file2, replicas, (short)(fileSize / blockSize));
     for (DataNode d: cluster.getDataNodes()) {
       DataNodeTestUtils.triggerBlockReport(d);
     }
@@ -251,7 +226,7 @@ public class TestDecommissioningStatus {
     FSNamesystem fsn = cluster.getNamesystem();
     final DatanodeManager dm = fsn.getBlockManager().getDatanodeManager();
     for (int iteration = 0; iteration < numDatanodes; iteration++) {
-      String downnode = decommissionNode(fsn, client, iteration);
+      String downnode = decommissionNode(client, iteration);
       dm.refreshNodes(conf);
       decommissionedNodes.add(downnode);
       BlockManagerTestUtil.recheckDecommissionState(dm);
@@ -281,8 +256,8 @@ public class TestDecommissioningStatus {
     hostsFileWriter.initExcludeHost("");
     dm.refreshNodes(conf);
     st1.close();
-    cleanupFile(fileSys, file1);
-    cleanupFile(fileSys, file2);
+    AdminStatesBaseTest.cleanupFile(fileSys, file1);
+    AdminStatesBaseTest.cleanupFile(fileSys, file2);
   }
 
   /**
@@ -308,7 +283,7 @@ public class TestDecommissioningStatus {
     // Decommission the DN.
     FSNamesystem fsn = cluster.getNamesystem();
     final DatanodeManager dm = fsn.getBlockManager().getDatanodeManager();
-    decommissionNode(fsn, dnName);
+    decommissionNode(dnName);
     dm.refreshNodes(conf);
 
     // Stop the DN when decommission is in progress.
@@ -343,7 +318,7 @@ public class TestDecommissioningStatus {
     
     // Delete the under-replicated file, which should let the 
     // DECOMMISSION_IN_PROGRESS node become DECOMMISSIONED
-    cleanupFile(fileSys, f);
+    AdminStatesBaseTest.cleanupFile(fileSys, f);
     BlockManagerTestUtil.recheckDecommissionState(dm);
     assertTrue("the node should be decommissioned",
         dead.get(0).isDecommissioned());
@@ -376,7 +351,7 @@ public class TestDecommissioningStatus {
     FSNamesystem fsn = cluster.getNamesystem();
     final DatanodeManager dm = fsn.getBlockManager().getDatanodeManager();
     DatanodeDescriptor dnDescriptor = dm.getDatanode(dnID);
-    decommissionNode(fsn, dnName);
+    decommissionNode(dnName);
     dm.refreshNodes(conf);
     BlockManagerTestUtil.recheckDecommissionState(dm);
     assertTrue(dnDescriptor.isDecommissioned());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
index 38ec9f8..c9fe2c3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
@@ -195,9 +195,17 @@ public class TestNamenodeCapacityReport {
   private static final float EPSILON = 0.0001f;
   @Test
   public void testXceiverCount() throws Exception {
+    testXceiverCountInternal(0);
+    testXceiverCountInternal(1);
+  }
+
+  public void testXceiverCountInternal(int minMaintenanceR) throws Exception {
     Configuration conf = new HdfsConfiguration();
     // retry one time, if close fails
-    conf.setInt(HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_RETRIES_KEY, 1);
+    conf.setInt(
+        HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_RETRIES_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_KEY,
+        minMaintenanceR);
     MiniDFSCluster cluster = null;
 
     final int nodes = 8;
@@ -220,23 +228,23 @@ public class TestNamenodeCapacityReport {
       int expectedTotalLoad = nodes;  // xceiver server adds 1 to load
       int expectedInServiceNodes = nodes;
       int expectedInServiceLoad = nodes;
-      checkClusterHealth(nodes, namesystem, expectedTotalLoad, expectedInServiceNodes, expectedInServiceLoad);
-      
-      // shutdown half the nodes and force a heartbeat check to ensure
-      // counts are accurate
+      checkClusterHealth(nodes, namesystem, expectedTotalLoad,
+          expectedInServiceNodes, expectedInServiceLoad);
+
+      // Shutdown half the nodes followed by admin operations on those nodes.
+      // Ensure counts are accurate.
       for (int i=0; i < nodes/2; i++) {
         DataNode dn = datanodes.get(i);
         DatanodeDescriptor dnd = dnm.getDatanode(dn.getDatanodeId());
         dn.shutdown();
         DFSTestUtil.setDatanodeDead(dnd);
         BlockManagerTestUtil.checkHeartbeat(namesystem.getBlockManager());
-        //Verify decommission of dead node won't impact nodesInService metrics.
-        dnm.getDecomManager().startDecommission(dnd);
+        //Admin operations on dead nodes won't impact nodesInService metrics.
+        startDecommissionOrMaintenance(dnm, dnd, (i % 2 == 0));
         expectedInServiceNodes--;
         assertEquals(expectedInServiceNodes, namesystem.getNumLiveDataNodes());
         assertEquals(expectedInServiceNodes, getNumDNInService(namesystem));
-        //Verify recommission of dead node won't impact nodesInService metrics.
-        dnm.getDecomManager().stopDecommission(dnd);
+        stopDecommissionOrMaintenance(dnm, dnd, (i % 2 == 0));
         assertEquals(expectedInServiceNodes, getNumDNInService(namesystem));
       }
 
@@ -247,8 +255,9 @@ public class TestNamenodeCapacityReport {
       datanodes = cluster.getDataNodes();
       expectedInServiceNodes = nodes;
       assertEquals(nodes, datanodes.size());
-      checkClusterHealth(nodes, namesystem, expectedTotalLoad, expectedInServiceNodes, expectedInServiceLoad);
-      
+      checkClusterHealth(nodes, namesystem, expectedTotalLoad,
+          expectedInServiceNodes, expectedInServiceLoad);
+
       // create streams and hsync to force datastreamers to start
       DFSOutputStream[] streams = new DFSOutputStream[fileCount];
       for (int i=0; i < fileCount; i++) {
@@ -263,30 +272,32 @@ public class TestNamenodeCapacityReport {
       }
       // force nodes to send load update
       triggerHeartbeats(datanodes);
-      checkClusterHealth(nodes, namesystem, expectedTotalLoad, expectedInServiceNodes, expectedInServiceLoad);
+      checkClusterHealth(nodes, namesystem, expectedTotalLoad,
+          expectedInServiceNodes, expectedInServiceLoad);
 
-      // decomm a few nodes, substract their load from the expected load,
-      // trigger heartbeat to force load update
+      // admin operations on a few nodes, substract their load from the
+      // expected load, trigger heartbeat to force load update.
       for (int i=0; i < fileRepl; i++) {
         expectedInServiceNodes--;
         DatanodeDescriptor dnd =
             dnm.getDatanode(datanodes.get(i).getDatanodeId());
         expectedInServiceLoad -= dnd.getXceiverCount();
-        dnm.getDecomManager().startDecommission(dnd);
+        startDecommissionOrMaintenance(dnm, dnd, (i % 2 == 0));
         DataNodeTestUtils.triggerHeartbeat(datanodes.get(i));
         Thread.sleep(100);
-        checkClusterHealth(nodes, namesystem, expectedTotalLoad, expectedInServiceNodes, expectedInServiceLoad);
+        checkClusterHealth(nodes, namesystem, expectedTotalLoad,
+            expectedInServiceNodes, expectedInServiceLoad);
       }
-      
+
       // check expected load while closing each stream.  recalc expected
       // load based on whether the nodes in the pipeline are decomm
       for (int i=0; i < fileCount; i++) {
-        int decomm = 0;
+        int adminOps = 0;
         for (DatanodeInfo dni : streams[i].getPipeline()) {
           DatanodeDescriptor dnd = dnm.getDatanode(dni);
           expectedTotalLoad -= 2;
-          if (dnd.isDecommissionInProgress() || dnd.isDecommissioned()) {
-            decomm++;
+          if (!dnd.isInService()) {
+            adminOps++;
           } else {
             expectedInServiceLoad -= 2;
           }
@@ -297,16 +308,17 @@ public class TestNamenodeCapacityReport {
           // nodes will go decommissioned even if there's a UC block whose
           // other locations are decommissioned too.  we'll ignore that
           // bug for now
-          if (decomm < fileRepl) {
+          if (adminOps < fileRepl) {
             throw ioe;
           }
         }
         triggerHeartbeats(datanodes);
         // verify node count and loads 
-        checkClusterHealth(nodes, namesystem, expectedTotalLoad, expectedInServiceNodes, expectedInServiceLoad);
+        checkClusterHealth(nodes, namesystem, expectedTotalLoad,
+            expectedInServiceNodes, expectedInServiceLoad);
       }
 
-      // shutdown each node, verify node counts based on decomm state
+      // shutdown each node, verify node counts based on admin state
       for (int i=0; i < nodes; i++) {
         DataNode dn = datanodes.get(i);
         dn.shutdown();
@@ -320,13 +332,11 @@ public class TestNamenodeCapacityReport {
           expectedInServiceNodes--;
         }
         assertEquals(expectedInServiceNodes, getNumDNInService(namesystem));
-        
         // live nodes always report load of 1.  no nodes is load 0
         double expectedXceiverAvg = (i == nodes-1) ? 0.0 : 1.0;
         assertEquals((double)expectedXceiverAvg,
             getInServiceXceiverAverage(namesystem), EPSILON);
       }
-      
       // final sanity check
       checkClusterHealth(0, namesystem, 0.0, 0, 0.0);
     } finally {
@@ -336,6 +346,24 @@ public class TestNamenodeCapacityReport {
     }
   }
 
+  private void startDecommissionOrMaintenance(DatanodeManager dnm,
+      DatanodeDescriptor dnd, boolean decomm) {
+    if (decomm) {
+      dnm.getDecomManager().startDecommission(dnd);
+    } else {
+      dnm.getDecomManager().startMaintenance(dnd, Long.MAX_VALUE);
+    }
+  }
+
+  private void stopDecommissionOrMaintenance(DatanodeManager dnm,
+      DatanodeDescriptor dnd, boolean decomm) {
+    if (decomm) {
+      dnm.getDecomManager().stopDecommission(dnd);
+    } else {
+      dnm.getDecomManager().stopMaintenance(dnd);
+    }
+  }
+
   private static void checkClusterHealth(
     int numOfLiveNodes,
     FSNamesystem namesystem, double expectedTotalLoad,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b61fb267/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/HostsFileWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/HostsFileWriter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/HostsFileWriter.java
index 4c8fcef..e171e2b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/HostsFileWriter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/HostsFileWriter.java
@@ -54,6 +54,7 @@ public class HostsFileWriter {
     localFileSys = FileSystem.getLocal(conf);
     Path workingDir = new Path(MiniDFSCluster.getBaseDirectory());
     this.fullDir = new Path(workingDir, dir);
+    cleanup(); // In case there is some left over from previous run.
     assertTrue(localFileSys.mkdirs(this.fullDir));
 
     if (conf.getClass(


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


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

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


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

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

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


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

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

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

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

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

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

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

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

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

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


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


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

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


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

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

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


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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0f4afc81/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
index 75b329c..895ae0c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -22,49 +22,41 @@ import java.net.URI;
 import java.net.URISyntaxException;
 
 import org.apache.hadoop.conf.Configuration;
-
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
-import org.apache.hadoop.fs.viewfs.ConfigUtil;
-import org.apache.hadoop.fs.viewfs.InodeTree;
 import org.junit.Test;
 
-
 public class TestViewFsConfig {
-  
-  
-  @Test(expected=FileAlreadyExistsException.class)
+
+  @Test(expected = FileAlreadyExistsException.class)
   public void testInvalidConfig() throws IOException, URISyntaxException {
     Configuration conf = new Configuration();
     ConfigUtil.addLink(conf, "/internalDir/linkToDir2",
         new Path("file:///dir2").toUri());
     ConfigUtil.addLink(conf, "/internalDir/linkToDir2/linkToDir3",
         new Path("file:///dir3").toUri());
-    
-    class Foo { };
-    
-     new InodeTree<Foo>(conf, null) {
+
+    class Foo {
+    }
+
+    new InodeTree<Foo>(conf, null) {
 
       @Override
-      protected
-      Foo getTargetFileSystem(final URI uri)
-        throws URISyntaxException, UnsupportedFileSystemException {
-          return null;
+      protected Foo getTargetFileSystem(final URI uri)
+          throws URISyntaxException, UnsupportedFileSystemException {
+        return null;
       }
 
       @Override
-      protected
-      Foo getTargetFileSystem(
-          org.apache.hadoop.fs.viewfs.InodeTree.INodeDir<Foo>
-                                          dir)
-        throws URISyntaxException {
+      protected Foo getTargetFileSystem(
+          org.apache.hadoop.fs.viewfs.InodeTree.INodeDir<Foo> dir)
+          throws URISyntaxException {
         return null;
       }
 
       @Override
-      protected
-      Foo getTargetFileSystem(URI[] mergeFsURIList)
+      protected Foo getTargetFileSystem(URI[] mergeFsURIList)
           throws URISyntaxException, UnsupportedFileSystemException {
         return null;
       }


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


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

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


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

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

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


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


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


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

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


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

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

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


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

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

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

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

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

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

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

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

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

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

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

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

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


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


[43/50] [abbrv] hadoop git commit: YARN-5743. [Atsv2] Publish queue name and RMAppMetrics to ATS (Rohith Sharma K S via Varun Saxena)

Posted by sj...@apache.org.
YARN-5743. [Atsv2] Publish queue name and RMAppMetrics to ATS (Rohith Sharma K S via Varun Saxena)


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

Branch: refs/heads/HADOOP-13070
Commit: b154d3edcee95254d41c237142870f39e826a519
Parents: d26a1bb
Author: Varun Saxena <va...@apache.org>
Authored: Tue Oct 18 23:32:52 2016 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Tue Oct 18 23:32:52 2016 +0530

----------------------------------------------------------------------
 .../metrics/ApplicationMetricsConstants.java    | 16 ++++++-
 .../metrics/TimelineServiceV2Publisher.java     | 49 ++++++++++++++++++--
 .../TestSystemMetricsPublisherForV2.java        | 18 ++++---
 3 files changed, 70 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b154d3ed/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ApplicationMetricsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ApplicationMetricsConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ApplicationMetricsConstants.java
index 1774208..521e0af 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ApplicationMetricsConstants.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ApplicationMetricsConstants.java
@@ -71,10 +71,22 @@ public class ApplicationMetricsConstants {
       "YARN_APPLICATION_STATE";
   
   public static final String APP_CPU_METRICS =
-      "YARN_APPLICATION_CPU_METRIC";
+      "YARN_APPLICATION_CPU";
   
   public static final String APP_MEM_METRICS =
-      "YARN_APPLICATION_MEM_METRIC";
+      "YARN_APPLICATION_MEMORY";
+
+  public static final String APP_RESOURCE_PREEMPTED_CPU =
+      "YARN_APPLICATION_RESOURCE_PREEMPTED_CPU";
+
+  public static final String APP_RESOURCE_PREEMPTED_MEM =
+      "YARN_APPLICATION_RESOURCE_PREEMPTED_MEMORY";
+
+  public static final String APP_NON_AM_CONTAINER_PREEMPTED =
+      "YARN_APPLICATION_NON_AM_CONTAINER_PREEMPTED";
+
+  public static final String APP_AM_CONTAINER_PREEMPTED =
+      "YARN_APPLICATION_AM_CONTAINER_PREEMPTED";
 
   public static final String LATEST_APP_ATTEMPT_EVENT_INFO =
       "YARN_APPLICATION_LATEST_APP_ATTEMPT";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b154d3ed/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
index dbdc1a8..f039ebe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
@@ -19,7 +19,9 @@
 package org.apache.hadoop.yarn.server.resourcemanager.metrics;
 
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -40,6 +42,7 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity.Identifier;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants;
@@ -104,6 +107,8 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
     entityInfo.put(ApplicationMetricsConstants.TYPE_ENTITY_INFO,
         app.getApplicationType());
     entityInfo.put(ApplicationMetricsConstants.USER_ENTITY_INFO, app.getUser());
+    entityInfo.put(ApplicationMetricsConstants.QUEUE_ENTITY_INFO,
+        app.getQueue());
     entityInfo.put(ApplicationMetricsConstants.SUBMITTED_TIME_ENTITY_INFO,
         app.getSubmitTime());
     entityInfo.put(ApplicationMetricsConstants.APP_TAGS_INFO,
@@ -148,11 +153,6 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
   @Override
   public void appFinished(RMApp app, RMAppState state, long finishedTime) {
     ApplicationEntity entity = createApplicationEntity(app.getApplicationId());
-    RMAppMetrics appMetrics = app.getRMAppMetrics();
-    entity.addInfo(ApplicationMetricsConstants.APP_CPU_METRICS,
-        appMetrics.getVcoreSeconds());
-    entity.addInfo(ApplicationMetricsConstants.APP_MEM_METRICS,
-        appMetrics.getMemorySeconds());
 
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
@@ -174,10 +174,49 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
     }
     entity.setInfo(entityInfo);
 
+    RMAppMetrics appMetrics = app.getRMAppMetrics();
+    Set<TimelineMetric> entityMetrics =
+        getTimelinelineAppMetrics(appMetrics, finishedTime);
+    entity.setMetrics(entityMetrics);
+
     getDispatcher().getEventHandler().handle(new TimelineV2PublishEvent(
         SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId()));
   }
 
+  private Set<TimelineMetric> getTimelinelineAppMetrics(
+      RMAppMetrics appMetrics, long timestamp) {
+    Set<TimelineMetric> entityMetrics = new HashSet<TimelineMetric>();
+
+    entityMetrics.add(getTimelineMetric(
+        ApplicationMetricsConstants.APP_CPU_METRICS, timestamp,
+        appMetrics.getVcoreSeconds()));
+    entityMetrics.add(getTimelineMetric(
+        ApplicationMetricsConstants.APP_MEM_METRICS, timestamp,
+        appMetrics.getMemorySeconds()));
+    entityMetrics.add(getTimelineMetric(
+        ApplicationMetricsConstants.APP_RESOURCE_PREEMPTED_CPU, timestamp,
+        appMetrics.getResourcePreempted().getVirtualCores()));
+    entityMetrics.add(getTimelineMetric(
+        ApplicationMetricsConstants.APP_RESOURCE_PREEMPTED_MEM, timestamp,
+        appMetrics.getResourcePreempted().getMemorySize()));
+    entityMetrics.add(getTimelineMetric(
+        ApplicationMetricsConstants.APP_NON_AM_CONTAINER_PREEMPTED, timestamp,
+        appMetrics.getNumNonAMContainersPreempted()));
+    entityMetrics.add(getTimelineMetric(
+        ApplicationMetricsConstants.APP_AM_CONTAINER_PREEMPTED, timestamp,
+        appMetrics.getNumAMContainersPreempted()));
+
+    return entityMetrics;
+  }
+
+  private TimelineMetric getTimelineMetric(String name, long timestamp,
+      Number value) {
+    TimelineMetric metric = new TimelineMetric();
+    metric.setId(name);
+    metric.addValue(timestamp, value);
+    return metric;
+  }
+
   @SuppressWarnings("unchecked")
   @Override
   public void appStateUpdated(RMApp app, YarnApplicationState appState,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b154d3ed/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java
index 13aa806..4a19744 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java
@@ -211,7 +211,7 @@ public class TestSystemMetricsPublisherForV2 {
             + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
     File appFile = new File(outputDirApp, timelineServiceFileName);
     Assert.assertTrue(appFile.exists());
-    verifyEntity(appFile, 3, ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+    verifyEntity(appFile, 3, ApplicationMetricsConstants.CREATED_EVENT_TYPE, 6);
   }
 
   @Test(timeout = 10000)
@@ -245,7 +245,8 @@ public class TestSystemMetricsPublisherForV2 {
             + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
     File appFile = new File(outputDirApp, timelineServiceFileName);
     Assert.assertTrue(appFile.exists());
-    verifyEntity(appFile, 2, AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE);
+    verifyEntity(appFile, 2, AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE,
+        0);
   }
 
   @Test(timeout = 10000)
@@ -277,7 +278,7 @@ public class TestSystemMetricsPublisherForV2 {
     File appFile = new File(outputDirApp, timelineServiceFileName);
     Assert.assertTrue(appFile.exists());
     verifyEntity(appFile, 2,
-        ContainerMetricsConstants.CREATED_IN_RM_EVENT_TYPE);
+        ContainerMetricsConstants.CREATED_IN_RM_EVENT_TYPE, 0);
   }
 
   private RMApp createAppAndRegister(ApplicationId appId) {
@@ -291,16 +292,18 @@ public class TestSystemMetricsPublisherForV2 {
   }
 
   private static void verifyEntity(File entityFile, long expectedEvents,
-      String eventForCreatedTime) throws IOException {
+      String eventForCreatedTime, long expectedMetrics) throws IOException {
     BufferedReader reader = null;
     String strLine;
     long count = 0;
+    long metricsCount = 0;
     try {
       reader = new BufferedReader(new FileReader(entityFile));
       while ((strLine = reader.readLine()) != null) {
         if (strLine.trim().length() > 0) {
           TimelineEntity entity = FileSystemTimelineReaderImpl.
               getTimelineRecordFromJSON(strLine.trim(), TimelineEntity.class);
+          metricsCount = entity.getMetrics().size();
           for (TimelineEvent event : entity.getEvents()) {
             if (event.getId().equals(eventForCreatedTime)) {
               assertTrue(entity.getCreatedTime() > 0);
@@ -314,7 +317,9 @@ public class TestSystemMetricsPublisherForV2 {
       reader.close();
     }
     assertEquals("Expected " + expectedEvents + " events to be published",
-        count, expectedEvents);
+        expectedEvents, count);
+    assertEquals("Expected " + expectedMetrics + " metrics is incorrect",
+        expectedMetrics, metricsCount);
   }
 
   private String getTimelineEntityDir(RMApp app) {
@@ -349,7 +354,8 @@ public class TestSystemMetricsPublisherForV2 {
     when(app.getFinalApplicationStatus()).thenReturn(
         FinalApplicationStatus.UNDEFINED);
     when(app.getRMAppMetrics()).thenReturn(
-        new RMAppMetrics(null, 0, 0, Integer.MAX_VALUE, Long.MAX_VALUE));
+        new RMAppMetrics(Resource.newInstance(0, 0), 0, 0, Integer.MAX_VALUE,
+            Long.MAX_VALUE));
     when(app.getApplicationTags()).thenReturn(Collections.<String> emptySet());
     ApplicationSubmissionContext appSubmissionContext =
         mock(ApplicationSubmissionContext.class);


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


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

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


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

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

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


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

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

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

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


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


[39/50] [abbrv] hadoop git commit: HADOOP-13522. Add %A and %a formats for fs -stat command to print permissions. Contributed by Alex Garbarini.

Posted by sj...@apache.org.
HADOOP-13522. Add %A and %a formats for fs -stat command to print permissions. Contributed by Alex Garbarini.


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

Branch: refs/heads/HADOOP-13070
Commit: bedfec0c10144087168bc79501ffd5ab4fa52606
Parents: 0bc6d37
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Oct 18 14:37:32 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Oct 18 15:00:44 2016 +0900

----------------------------------------------------------------------
 .../hadoop/fs/permission/FsPermission.java      | 12 ++++++++++++
 .../java/org/apache/hadoop/fs/shell/Stat.java   | 11 ++++++++++-
 .../src/site/markdown/FileSystemShell.md        |  4 ++--
 .../src/test/resources/testConf.xml             |  6 +++++-
 .../org/apache/hadoop/hdfs/TestDFSShell.java    | 20 +++++++++++++++++---
 5 files changed, 46 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bedfec0c/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 48a5b1c..fabfc12 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
@@ -183,6 +183,18 @@ public class FsPermission implements Writable {
     return toShort();
   }
 
+  /**
+   * Returns the FsPermission in an octal format.
+   *
+   * @return short Unlike {@link #toShort()} which provides a binary
+   * representation, this method returns the standard octal style permission.
+   */
+  public short toOctal() {
+    int n = this.toShort();
+    int octal = (n>>>9&1)*1000 + (n>>>6&7)*100 + (n>>>3&7)*10 + (n&7);
+    return (short)octal;
+  }
+
   @Override
   public boolean equals(Object obj) {
     if (obj instanceof FsPermission) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bedfec0c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Stat.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Stat.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Stat.java
index 458d3ee..42f7843 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Stat.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Stat.java
@@ -31,6 +31,8 @@ import org.apache.hadoop.fs.FileStatus;
 /**
  * Print statistics about path in specified format.
  * Format sequences:<br>
+ *   %a: Permissions in octal<br>
+ *   %A: Permissions in symbolic style<br>
  *   %b: Size of file in blocks<br>
  *   %F: Type<br>
  *   %g: Group name of owner<br>
@@ -56,7 +58,8 @@ class Stat extends FsCommand {
   public static final String USAGE = "[format] <path> ...";
   public static final String DESCRIPTION =
     "Print statistics about the file/directory at <path>" + NEWLINE +
-    "in the specified format. Format accepts filesize in" + NEWLINE +
+    "in the specified format. Format accepts permissions in" + NEWLINE +
+    "octal (%a) and symbolic (%A), filesize in" + NEWLINE +
     "blocks (%b), type (%F), group name of owner (%g)," + NEWLINE +
     "name (%n), block size (%o), replication (%r), user name" + NEWLINE +
     "of owner (%u), modification date (%y, %Y)." + NEWLINE +
@@ -95,6 +98,12 @@ class Stat extends FsCommand {
         // this silently drops a trailing %?
         if (i + 1 == fmt.length) break;
         switch (fmt[++i]) {
+          case 'a':
+            buf.append(stat.getPermission().toOctal());
+            break;
+          case 'A':
+            buf.append(stat.getPermission());
+            break;
           case 'b':
             buf.append(stat.getLen());
             break;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bedfec0c/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md b/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
index ee7bc28..060c775 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
@@ -639,11 +639,11 @@ stat
 
 Usage: `hadoop fs -stat [format] <path> ...`
 
-Print statistics about the file/directory at \<path\> in the specified format. Format accepts filesize in blocks (%b), type (%F), group name of owner (%g), name (%n), block size (%o), replication (%r), user name of owner(%u), and modification date (%y, %Y). %y shows UTC date as "yyyy-MM-dd HH:mm:ss" and %Y shows milliseconds since January 1, 1970 UTC. If the format is not specified, %y is used by default.
+Print statistics about the file/directory at \<path\> in the specified format. Format accepts permissions in octal (%a) and symbolic (%A), filesize in blocks (%b), type (%F), group name of owner (%g), name (%n), block size (%o), replication (%r), user name of owner(%u), and modification date (%y, %Y). %y shows UTC date as "yyyy-MM-dd HH:mm:ss" and %Y shows milliseconds since January 1, 1970 UTC. If the format is not specified, %y is used by default.
 
 Example:
 
-* `hadoop fs -stat "%F %u:%g %b %y %n" /file`
+* `hadoop fs -stat "%F %a %u:%g %b %y %n" /file`
 
 Exit Code: Returns 0 on success and -1 on error.
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bedfec0c/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml b/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
index a31c828..d285f33 100644
--- a/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
+++ b/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
@@ -859,7 +859,11 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*in the specified format. Format accepts filesize in( )*</expected-output>
+          <expected-output>^( |\t)*in the specified format. Format accepts permissions in( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*octal \(%a\) and symbolic \(%A\), filesize in( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bedfec0c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
index 5e5b8b6..cf193c9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
@@ -2021,17 +2021,31 @@ public class TestDFSShell {
         out.toString(), String.format("%s%n%s%n", mtime1, mtime2));
 
     doFsStat(dfs.getConf(), "%F %u:%g %b %y %n");
-
     out.reset();
-    doFsStat(dfs.getConf(), "%F %u:%g %b %y %n", testDir1);
+
+    doFsStat(dfs.getConf(), "%F %a %A %u:%g %b %y %n", testDir1);
     assertTrue(out.toString(), out.toString().contains(mtime1));
     assertTrue(out.toString(), out.toString().contains("directory"));
     assertTrue(out.toString(), out.toString().contains(status1.getGroup()));
+    assertTrue(out.toString(),
+        out.toString().contains(status1.getPermission().toString()));
+
+    int n = status1.getPermission().toShort();
+    int octal = (n>>>9&1)*1000 + (n>>>6&7)*100 + (n>>>3&7)*10 + (n&7);
+    assertTrue(out.toString(),
+        out.toString().contains(String.valueOf(octal)));
 
     out.reset();
-    doFsStat(dfs.getConf(), "%F %u:%g %b %y %n", testDir1, testFile2);
+    doFsStat(dfs.getConf(), "%F %a %A %u:%g %b %y %n", testDir1, testFile2);
+
+    n = status2.getPermission().toShort();
+    octal = (n>>>9&1)*1000 + (n>>>6&7)*100 + (n>>>3&7)*10 + (n&7);
     assertTrue(out.toString(), out.toString().contains(mtime1));
     assertTrue(out.toString(), out.toString().contains("regular file"));
+    assertTrue(out.toString(),
+        out.toString().contains(status2.getPermission().toString()));
+    assertTrue(out.toString(),
+        out.toString().contains(String.valueOf(octal)));
     assertTrue(out.toString(), out.toString().contains(mtime2));
   }
 


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


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

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


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

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

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


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

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

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

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


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


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

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


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

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

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


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


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


[46/50] [abbrv] hadoop git commit: HADOOP-13560. S3ABlockOutputStream to support huge (many GB) file writes. Contributed by Steve Loughran

Posted by sj...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index cf785d5..c23e782 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -1,3 +1,4 @@
+
 <!---
   Licensed under the Apache License, Version 2.0 (the "License");
   you may not use this file except in compliance with the License.
@@ -852,40 +853,361 @@ Seoul
 If the wrong endpoint is used, the request may fail. This may be reported as a 301/redirect error,
 or as a 400 Bad Request.
 
-### S3AFastOutputStream
- **Warning: NEW in hadoop 2.7. UNSTABLE, EXPERIMENTAL: use at own risk**
 
-    <property>
-      <name>fs.s3a.fast.upload</name>
-      <value>false</value>
-      <description>Upload directly from memory instead of buffering to
-      disk first. Memory usage and parallelism can be controlled as up to
-      fs.s3a.multipart.size memory is consumed for each (part)upload actively
-      uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description>
-    </property>
 
-    <property>
-      <name>fs.s3a.fast.buffer.size</name>
-      <value>1048576</value>
-      <description>Size (in bytes) of initial memory buffer allocated for an
-      upload. No effect if fs.s3a.fast.upload is false.</description>
-    </property>
+### <a name="s3a_fast_upload"></a>Stabilizing: S3A Fast Upload
+
+
+**New in Hadoop 2.7; significantly enhanced in Hadoop 2.9**
+
+
+Because of the nature of the S3 object store, data written to an S3A `OutputStream`
+is not written incrementally \u2014instead, by default, it is buffered to disk
+until the stream is closed in its `close()` method.
+
+This can make output slow:
+
+* The execution time for `OutputStream.close()` is proportional to the amount of data
+buffered and inversely proportional to the bandwidth. That is `O(data/bandwidth)`.
+* The bandwidth is that available from the host to S3: other work in the same
+process, server or network at the time of upload may increase the upload time,
+hence the duration of the `close()` call.
+* If a process uploading data fails before `OutputStream.close()` is called,
+all data is lost.
+* The disks hosting temporary directories defined in `fs.s3a.buffer.dir` must
+have the capacity to store the entire buffered file.
+
+Put succinctly: the further the process is from the S3 endpoint, or the smaller
+the EC-hosted VM is, the longer it will take work to complete.
+
+This can create problems in application code:
+
+* Code often assumes that the `close()` call is fast;
+ the delays can create bottlenecks in operations.
+* Very slow uploads sometimes cause applications to time out. (generally,
+threads blocking during the upload stop reporting progress, so trigger timeouts)
+* Streaming very large amounts of data may consume all disk space before the upload begins.
+
+
+Work to addess this began in Hadoop 2.7 with the `S3AFastOutputStream`
+[HADOOP-11183](https://issues.apache.org/jira/browse/HADOOP-11183), and
+has continued with ` S3ABlockOutputStream`
+[HADOOP-13560](https://issues.apache.org/jira/browse/HADOOP-13560).
+
+
+This adds an alternative output stream, "S3a Fast Upload" which:
+
+1.  Always uploads large files as blocks with the size set by
+    `fs.s3a.multipart.size`. That is: the threshold at which multipart uploads
+    begin and the size of each upload are identical.
+1.  Buffers blocks to disk (default) or in on-heap or off-heap memory.
+1.  Uploads blocks in parallel in background threads.
+1.  Begins uploading blocks as soon as the buffered data exceeds this partition
+    size.
+1.  When buffering data to disk, uses the directory/directories listed in
+    `fs.s3a.buffer.dir`. The size of data which can be buffered is limited
+    to the available disk space.
+1.  Generates output statistics as metrics on the filesystem, including
+    statistics of active and pending block uploads.
+1.  Has the time to `close()` set by the amount of remaning data to upload, rather
+    than the total size of the file.
+
+With incremental writes of blocks, "S3A fast upload" offers an upload
+time at least as fast as the "classic" mechanism, with significant benefits
+on long-lived output streams, and when very large amounts of data are generated.
+The in memory buffering mechanims may also  offer speedup when running adjacent to
+S3 endpoints, as disks are not used for intermediate data storage.
+
+
+```xml
+<property>
+  <name>fs.s3a.fast.upload</name>
+  <value>true</value>
+  <description>
+    Use the incremental block upload mechanism with
+    the buffering mechanism set in fs.s3a.fast.upload.buffer.
+    The number of threads performing uploads in the filesystem is defined
+    by fs.s3a.threads.max; the queue of waiting uploads limited by
+    fs.s3a.max.total.tasks.
+    The size of each buffer is set by fs.s3a.multipart.size.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.fast.upload.buffer</name>
+  <value>disk</value>
+  <description>
+    The buffering mechanism to use when using S3A fast upload
+    (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer.
+    This configuration option has no effect if fs.s3a.fast.upload is false.
+
+    "disk" will use the directories listed in fs.s3a.buffer.dir as
+    the location(s) to save data prior to being uploaded.
+
+    "array" uses arrays in the JVM heap
+
+    "bytebuffer" uses off-heap memory within the JVM.
+
+    Both "array" and "bytebuffer" will consume memory in a single stream up to the number
+    of blocks set by:
+
+        fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks.
+
+    If using either of these mechanisms, keep this value low
+
+    The total number of threads performing work across all threads is set by
+    fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued
+    work items.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.multipart.size</name>
+  <value>104857600</value>
+  <description>
+  How big (in bytes) to split upload or copy operations up into.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.fast.upload.active.blocks</name>
+  <value>8</value>
+  <description>
+    Maximum Number of blocks a single output stream can have
+    active (uploading, or queued to the central FileSystem
+    instance's pool of queued operations.
+
+    This stops a single stream overloading the shared thread pool.
+  </description>
+</property>
+```
+
+**Notes**
+
+* If the amount of data written to a stream is below that set in `fs.s3a.multipart.size`,
+the upload is performed in the `OutputStream.close()` operation \u2014as with
+the original output stream.
+
+* The published Hadoop metrics monitor include live queue length and
+upload operation counts, so identifying when there is a backlog of work/
+a mismatch between data generation rates and network bandwidth. Per-stream
+statistics can also be logged by calling `toString()` on the current stream.
+
+* Incremental writes are not visible; the object can only be listed
+or read when the multipart operation completes in the `close()` call, which
+will block until the upload is completed.
+
+
+#### <a name="s3a_fast_upload_disk"></a>Fast Upload with Disk Buffers `fs.s3a.fast.upload.buffer=disk`
+
+When `fs.s3a.fast.upload.buffer` is set to `disk`, all data is buffered
+to local hard disks prior to upload. This minimizes the amount of memory
+consumed, and so eliminates heap size as the limiting factor in queued uploads
+\u2014exactly as the original "direct to disk" buffering used when
+`fs.s3a.fast.upload=false`.
+
+
+```xml
+<property>
+  <name>fs.s3a.fast.upload</name>
+  <value>true</value>
+</property>
+
+<property>
+  <name>fs.s3a.fast.upload.buffer</name>
+  <value>disk</value>
+</property>
+
+```
+
+
+#### <a name="s3a_fast_upload_bytebuffer"></a>Fast Upload with ByteBuffers: `fs.s3a.fast.upload.buffer=bytebuffer`
+
+When `fs.s3a.fast.upload.buffer` is set to `bytebuffer`, all data is buffered
+in "Direct" ByteBuffers prior to upload. This *may* be faster than buffering to disk,
+and, if disk space is small (for example, tiny EC2 VMs), there may not
+be much disk space to buffer with.
+
+The ByteBuffers are created in the memory of the JVM, but not in the Java Heap itself.
+The amount of data which can be buffered is
+limited by the Java runtime, the operating system, and, for YARN applications,
+the amount of memory requested for each container.
+
+The slower the write bandwidth to S3, the greater the risk of running out
+of memory \u2014and so the more care is needed in
+[tuning the upload settings](#s3a_fast_upload_thread_tuning).
+
+
+```xml
+<property>
+  <name>fs.s3a.fast.upload</name>
+  <value>true</value>
+</property>
+
+<property>
+  <name>fs.s3a.fast.upload.buffer</name>
+  <value>bytebuffer</value>
+</property>
+```
+
+#### <a name="s3a_fast_upload_array"></a>Fast Upload with Arrays: `fs.s3a.fast.upload.buffer=array`
+
+When `fs.s3a.fast.upload.buffer` is set to `array`, all data is buffered
+in byte arrays in the JVM's heap prior to upload.
+This *may* be faster than buffering to disk.
+
+This `array` option is similar to the in-memory-only stream offered in
+Hadoop 2.7 with `fs.s3a.fast.upload=true`
+
+The amount of data which can be buffered is limited by the available
+size of the JVM heap heap. The slower the write bandwidth to S3, the greater
+the risk of heap overflows. This risk can be mitigated by
+[tuning the upload settings](#s3a_fast_upload_thread_tuning).
+
+```xml
+<property>
+  <name>fs.s3a.fast.upload</name>
+  <value>true</value>
+</property>
+
+<property>
+  <name>fs.s3a.fast.upload.buffer</name>
+  <value>array</value>
+</property>
+
+```
+#### <a name="s3a_fast_upload_thread_tuning"></a>S3A Fast Upload Thread Tuning
+
+Both the [Array](#s3a_fast_upload_array) and [Byte buffer](#s3a_fast_upload_bytebuffer)
+buffer mechanisms can consume very large amounts of memory, on-heap or
+off-heap respectively. The [disk buffer](#s3a_fast_upload_disk) mechanism
+does not use much memory up, but will consume hard disk capacity.
+
+If there are many output streams being written to in a single process, the
+amount of memory or disk used is the multiple of all stream's active memory/disk use.
+
+Careful tuning may be needed to reduce the risk of running out memory, especially
+if the data is buffered in memory.
+
+There are a number parameters which can be tuned:
+
+1. The total number of threads available in the filesystem for data
+uploads *or any other queued filesystem operation*. This is set in
+`fs.s3a.threads.max`
+
+1. The number of operations which can be queued for execution:, *awaiting
+a thread*: `fs.s3a.max.total.tasks`
+
+1. The number of blocks which a single output stream can have active,
+that is: being uploaded by a thread, or queued in the filesystem thread queue:
+`fs.s3a.fast.upload.active.blocks`
+
+1. How long an idle thread can stay in the thread pool before it is retired: `fs.s3a.threads.keepalivetime`
+
+
+When the maximum allowed number of active blocks of a single stream is reached,
+no more blocks can be uploaded from that stream until one or more of those active
+blocks' uploads completes. That is: a `write()` call which would trigger an upload
+of a now full datablock, will instead block until there is capacity in the queue.
+
+How does that come together?
+
+* As the pool of threads set in `fs.s3a.threads.max` is shared (and intended
+to be used across all threads), a larger number here can allow for more
+parallel operations. However, as uploads require network bandwidth, adding more
+threads does not guarantee speedup.
+
+* The extra queue of tasks for the thread pool (`fs.s3a.max.total.tasks`)
+covers all ongoing background S3A operations (future plans include: parallelized
+rename operations, asynchronous directory operations).
+
+* When using memory buffering, a small value of `fs.s3a.fast.upload.active.blocks`
+limits the amount of memory which can be consumed per stream.
+
+* When using disk buffering a larger value of `fs.s3a.fast.upload.active.blocks`
+does not consume much memory. But it may result in a large number of blocks to
+compete with other filesystem operations.
+
+
+We recommend a low value of `fs.s3a.fast.upload.active.blocks`; enough
+to start background upload without overloading other parts of the system,
+then experiment to see if higher values deliver more throughtput \u2014especially
+from VMs running on EC2.
+
+```xml
+
+<property>
+  <name>fs.s3a.fast.upload.active.blocks</name>
+  <value>4</value>
+  <description>
+    Maximum Number of blocks a single output stream can have
+    active (uploading, or queued to the central FileSystem
+    instance's pool of queued operations.
+
+    This stops a single stream overloading the shared thread pool.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.threads.max</name>
+  <value>10</value>
+  <description>The total number of threads available in the filesystem for data
+    uploads *or any other queued filesystem operation*.</description>
+</property>
+
+<property>
+  <name>fs.s3a.max.total.tasks</name>
+  <value>5</value>
+  <description>The number of operations which can be queued for execution</description>
+</property>
+
+<property>
+  <name>fs.s3a.threads.keepalivetime</name>
+  <value>60</value>
+  <description>Number of seconds a thread can be idle before being
+    terminated.</description>
+</property>
+
+```
+
+
+#### <a name="s3a_multipart_purge"></a>Cleaning up After Incremental Upload Failures: `fs.s3a.multipart.purge`
+
+
+If an incremental streaming operation is interrupted, there may be
+intermediate partitions uploaded to S3 \u2014data which will be billed for.
+
+These charges can be reduced by enabling `fs.s3a.multipart.purge`,
+and setting a purge time in seconds, such as 86400 seconds \u201424 hours.
+When an S3A FileSystem instance is instantiated with the purge time greater
+than zero, it will, on startup, delete all outstanding partition requests
+older than this time.
+
+```xml
+<property>
+  <name>fs.s3a.multipart.purge</name>
+  <value>true</value>
+  <description>True if you want to purge existing multipart uploads that may not have been
+     completed/aborted correctly</description>
+</property>
+
+<property>
+  <name>fs.s3a.multipart.purge.age</name>
+  <value>86400</value>
+  <description>Minimum age in seconds of multipart uploads to purge</description>
+</property>
+```
+
+If an S3A client is instantited with `fs.s3a.multipart.purge=true`,
+it will delete all out of date uploads *in the entire bucket*. That is: it will affect all
+multipart uploads to that bucket, from all applications.
 
-Writes are buffered in memory instead of to a file on local disk. This
-removes the throughput bottleneck of the local disk write and read cycle
-before starting the actual upload. Furthermore, it allows handling files that
-are larger than the remaining local disk space.
-
-However, non-trivial memory tuning is needed for optimal results and careless
-settings could cause memory overflow. Up to `fs.s3a.threads.max` parallel
-(part)uploads are active. Furthermore, up to `fs.s3a.max.total.tasks`
-additional part(uploads) can be waiting (and thus memory buffers are created).
-The memory buffer is uploaded as a single upload if it is not larger than
-`fs.s3a.multipart.threshold`. Else, a multi-part upload is initiated and
-parts of size `fs.s3a.multipart.size` are used to protect against overflowing
-the available memory. These settings should be tuned to the envisioned
-workflow (some large files, many small ones, ...) and the physical
-limitations of the machine and cluster (memory, network bandwidth).
+Leaving `fs.s3a.multipart.purge` to its default, `false`,
+means that the client will not make any attempt to reset or change the partition
+rate.
+
+The best practise for using this option is to disable multipart purges in
+normal use of S3A, enabling only in manual/scheduled housekeeping operations.
 
 ### S3A Experimental "fadvise" input policy support
 
@@ -1221,7 +1543,143 @@ can be used:
 Using the explicit endpoint for the region is recommended for speed and the
 ability to use the V4 signing API.
 
-## Visible S3 Inconsistency
+
+### "Timeout waiting for connection from pool" when writing to S3A
+
+This happens when using the Block output stream, `fs.s3a.fast.upload=true` and
+the thread pool runs out of capacity.
+
+```
+[s3a-transfer-shared-pool1-t20] INFO  http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496)) - Unable to execute HTTP request: Timeout waiting for connection from poolorg.apache.http.conn.ConnectionPoolTimeoutException: Timeout waiting for connection from pool
+  at org.apache.http.impl.conn.PoolingClientConnectionManager.leaseConnection(PoolingClientConnectionManager.java:230)
+  at org.apache.http.impl.conn.PoolingClientConnectionManager$1.getConnection(PoolingClientConnectionManager.java:199)
+  at sun.reflect.GeneratedMethodAccessor13.invoke(Unknown Source)
+  at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
+  at java.lang.reflect.Method.invoke(Method.java:498)
+  at com.amazonaws.http.conn.ClientConnectionRequestFactory$Handler.invoke(ClientConnectionRequestFactory.java:70)
+  at com.amazonaws.http.conn.$Proxy10.getConnection(Unknown Source)
+  at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:424)
+  at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:884)
+  at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
+  at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55)
+  at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728)
+  at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
+  at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
+  at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
+  at com.amazonaws.services.s3.AmazonS3Client.doUploadPart(AmazonS3Client.java:2921)
+  at com.amazonaws.services.s3.AmazonS3Client.uploadPart(AmazonS3Client.java:2906)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.uploadPart(S3AFileSystem.java:1025)
+  at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:360)
+  at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:355)
+  at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239)
+  at java.util.concurrent.FutureTask.run(FutureTask.java:266)
+  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
+  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
+  at java.lang.Thread.run(Thread.java:745)
+```
+
+Make sure that `fs.s3a.connection.maximum` is at least larger
+than `fs.s3a.threads.max`.
+
+```xml
+<property>
+  <name>fs.s3a.threads.max</name>
+  <value>20</value>
+</property>
+
+<property>
+  <name>fs.s3a.connection.maximum</name>
+  <value>30</value>
+</property>
+```
+
+### "Timeout waiting for connection from pool" when reading from S3A
+
+This happens when more threads are trying to read from an S3A system than
+the maximum number of allocated HTTP connections.
+
+Set `fs.s3a.connection.maximum` to a larger value (and at least as large as
+`fs.s3a.threads.max`)
+
+### Out of heap memory when writing to S3A via Fast Upload
+
+This can happen when using the fast upload mechanism (`fs.s3a.fast.upload=true`)
+and in-memory buffering (either `fs.s3a.fast.upload.buffer=array` or
+`fs.s3a.fast.upload.buffer=bytebuffer`).
+
+More data is being generated than in the JVM than it can upload to S3 \u2014and
+so much data has been buffered that the JVM has run out of memory.
+
+Consult [S3A Fast Upload Thread Tuning](#s3a_fast_upload_thread_tuning) for
+detail on this issue and options to address it. Consider also buffering to
+disk, rather than memory.
+
+
+### When writing to S3A: "java.io.FileNotFoundException: Completing multi-part upload"
+
+
+```
+java.io.FileNotFoundException: Completing multi-part upload on fork-5/test/multipart/1c397ca6-9dfb-4ac1-9cf7-db666673246b: com.amazonaws.services.s3.model.AmazonS3Exception: The specified upload does not exist. The upload ID may be invalid, or the upload may have been aborted or completed. (Service: Amazon S3; Status Code: 404; Error Code: NoSuchUpload; Request ID: 84FF8057174D9369), S3 Extended Request ID: Ij5Yn6Eq/qIERH4Z6Io3YL2t9/qNZ7z9gjPb1FrTtTovZ8k1MXqh+zCYYjqmfJ/fCY6E1+JR9jA=
+  at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1182)
+  at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:770)
+  at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
+  at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
+  at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
+  at com.amazonaws.services.s3.AmazonS3Client.completeMultipartUpload(AmazonS3Client.java:2705)
+  at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.complete(S3ABlockOutputStream.java:473)
+  at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.access$200(S3ABlockOutputStream.java:382)
+  at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:272)
+  at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
+  at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
+```
+
+This surfaces if, while a multipart upload was taking place, all outstanding multipart
+uploads were garbage collected. The upload operation cannot complete because
+the data uploaded has been deleted.
+
+Consult [Cleaning up After Incremental Upload Failures](#s3a_multipart_purge) for
+details on how the multipart purge timeout can be set. If multipart uploads
+are failing with the message above, it may be a sign that this value is too low.
+
+### When writing to S3A, HTTP Exceptions logged at info from `AmazonHttpClient`
+
+```
+[s3a-transfer-shared-pool4-t6] INFO  http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496)) - Unable to execute HTTP request: hwdev-steve-ireland-new.s3.amazonaws.com:443 failed to respond
+org.apache.http.NoHttpResponseException: bucket.s3.amazonaws.com:443 failed to respond
+  at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:143)
+  at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:57)
+  at org.apache.http.impl.io.AbstractMessageParser.parse(AbstractMessageParser.java:261)
+  at org.apache.http.impl.AbstractHttpClientConnection.receiveResponseHeader(AbstractHttpClientConnection.java:283)
+  at org.apache.http.impl.conn.DefaultClientConnection.receiveResponseHeader(DefaultClientConnection.java:259)
+  at org.apache.http.impl.conn.ManagedClientConnectionImpl.receiveResponseHeader(ManagedClientConnectionImpl.java:209)
+  at org.apache.http.protocol.HttpRequestExecutor.doReceiveResponse(HttpRequestExecutor.java:272)
+  at com.amazonaws.http.protocol.SdkHttpRequestExecutor.doReceiveResponse(SdkHttpRequestExecutor.java:66)
+  at org.apache.http.protocol.HttpRequestExecutor.execute(HttpRequestExecutor.java:124)
+  at org.apache.http.impl.client.DefaultRequestDirector.tryExecute(DefaultRequestDirector.java:686)
+  at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:488)
+  at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:884)
+  at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
+  at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55)
+  at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728)
+  at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
+  at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
+  at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
+  at com.amazonaws.services.s3.AmazonS3Client.copyPart(AmazonS3Client.java:1731)
+  at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:41)
+  at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:28)
+  at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239)
+  at java.util.concurrent.FutureTask.run(FutureTask.java:266)
+  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
+  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
+  at java.lang.Thread.run(Thread.java:745)
+```
+
+These are HTTP I/O exceptions caught and logged inside the AWS SDK. The client
+will attempt to retry the operation; it may just be a transient event. If there
+are many such exceptions in logs, it may be a symptom of connectivity or network
+problems.
+
+### Visible S3 Inconsistency
 
 Amazon S3 is *an eventually consistent object store*. That is: not a filesystem.
 
@@ -1564,7 +2022,7 @@ tests or the `it.test` property for integration tests.
 
     mvn clean test -Dtest=TestS3AInputPolicies
 
-    mvn clean verify -Dit.test=ITestS3AFileContextStatistics
+    mvn clean verify -Dit.test=ITestS3AFileContextStatistics -Dtest=none
 
     mvn clean verify -Dtest=TestS3A* -Dit.test=ITestS3A*
 
@@ -1614,7 +2072,7 @@ An alternate endpoint may be defined in `test.fs.s3a.sts.endpoint`.
 
 The default is ""; meaning "use the amazon default value".
 
-#### CSV Data source Tests
+### CSV Data source Tests
 
 The `TestS3AInputStreamPerformance` tests require read access to a multi-MB
 text file. The default file for these tests is one published by amazon,
@@ -1661,18 +2119,89 @@ endpoint:
   <value>s3.amazonaws.com</value>
 </property>
 ```
+### Viewing Integration Test Reports
 
-#### Scale test operation count
+
+Integration test results and logs are stored in `target/failsafe-reports/`.
+An HTML report can be generated during site generation, or with the `surefire-report`
+plugin:
+
+```
+mvn surefire-report:failsafe-report-only
+```
+### Scale Tests
+
+There are a set of tests designed to measure the scalability and performance
+at scale of the S3A tests, *Scale Tests*. Tests include: creating
+and traversing directory trees, uploading large files, renaming them,
+deleting them, seeking through the files, performing random IO, and others.
+This makes them a foundational part of the benchmarking.
+
+By their very nature they are slow. And, as their execution time is often
+limited by bandwidth between the computer running the tests and the S3 endpoint,
+parallel execution does not speed these tests up.
+
+#### Enabling the Scale Tests
+
+The tests are enabled if the `scale` property is set in the maven build
+this can be done regardless of whether or not the parallel test profile
+is used
+
+```bash
+mvn verify -Dscale
+
+mvn verify -Dparallel-tests -Dscale -DtestsThreadCount=8
+```
+
+The most bandwidth intensive tests (those which upload data) always run
+sequentially; those which are slow due to HTTPS setup costs or server-side
+actionsare included in the set of parallelized tests.
+
+
+#### Maven build tuning options
+
+
+Some of the tests can be tuned from the maven build or from the
+configuration file used to run the tests.
+
+```bash
+mvn verify -Dscale -Dfs.s3a.scale.test.huge.filesize=128M
+```
+
+The algorithm is
+
+1. The value is queried from the configuration file, using a default value if
+it is not set.
+1. The value is queried from the JVM System Properties, where it is passed
+down by maven.
+1. If the system property is null, empty, or it has the value `unset`, then
+the configuration value is used. The `unset` option is used to
+[work round a quirk in maven property propagation](http://stackoverflow.com/questions/7773134/null-versus-empty-arguments-in-maven).
+
+Only a few properties can be set this way; more will be added.
+
+| Property | Meaninging |
+|-----------|-------------|
+| `fs.s3a.scale.test.timeout`| Timeout in seconds for scale tests |
+| `fs.s3a.scale.test.huge.filesize`| Size for huge file uploads |
+| `fs.s3a.scale.test.huge.huge.partitionsize`| Size for partitions in huge file uploads |
+
+The file and partition sizes are numeric values with a k/m/g/t/p suffix depending
+on the desired size. For example: 128M, 128m, 2G, 2G, 4T or even 1P.
+
+#### Scale test configuration options
 
 Some scale tests perform multiple operations (such as creating many directories).
 
 The exact number of operations to perform is configurable in the option
 `scale.test.operation.count`
 
-      <property>
-        <name>scale.test.operation.count</name>
-        <value>10</value>
-      </property>
+```xml
+<property>
+  <name>scale.test.operation.count</name>
+  <value>10</value>
+</property>
+```
 
 Larger values generate more load, and are recommended when testing locally,
 or in batch runs.
@@ -1685,19 +2214,64 @@ the width and depth of tests creating recursive directories. Larger
 values create exponentially more directories, with consequent performance
 impact.
 
-      <property>
-        <name>scale.test.directory.count</name>
-        <value>2</value>
-      </property>
+```xml
+<property>
+  <name>scale.test.directory.count</name>
+  <value>2</value>
+</property>
+```
 
 DistCp tests targeting S3A support a configurable file size.  The default is
 10 MB, but the configuration value is expressed in KB so that it can be tuned
 smaller to achieve faster test runs.
 
-      <property>
-        <name>scale.test.distcp.file.size.kb</name>
-        <value>10240</value>
-      </property>
+```xml
+<property>
+  <name>scale.test.distcp.file.size.kb</name>
+  <value>10240</value>
+</property>
+```
+
+S3A specific scale test properties are
+
+##### `fs.s3a.scale.test.huge.filesize`: size in MB for "Huge file tests".
+
+The Huge File tests validate S3A's ability to handle large files \u2014the property
+`fs.s3a.scale.test.huge.filesize` declares the file size to use.
+
+```xml
+<property>
+  <name>fs.s3a.scale.test.huge.filesize</name>
+  <value>200M</value>
+</property>
+```
+
+Amazon S3 handles files larger than 5GB differently than smaller ones.
+Setting the huge filesize to a number greater than that) validates support
+for huge files.
+
+```xml
+<property>
+  <name>fs.s3a.scale.test.huge.filesize</name>
+  <value>6G</value>
+</property>
+```
+
+Tests at this scale are slow: they are best executed from hosts running in
+the cloud infrastructure where the S3 endpoint is based.
+Otherwise, set a large timeout in `fs.s3a.scale.test.timeout`
+
+```xml
+<property>
+  <name>fs.s3a.scale.test.timeout</name>
+  <value>432000</value>
+</property>
+```
+
+
+The tests are executed in an order to only clean up created files after
+the end of all the tests. If the tests are interrupted, the test data will remain.
+
 
 
 ### Testing against non AWS S3 endpoints.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
index 28278fe..9e14ed2 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
@@ -18,24 +18,26 @@
 
 package org.apache.hadoop.fs.contract.s3a;
 
-import static org.apache.hadoop.fs.s3a.Constants.MIN_MULTIPART_THRESHOLD;
-import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.*;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
 
 /**
  * Contract test suite covering S3A integration with DistCp.
+ * Uses the block output stream, buffered to disk. This is the
+ * recommended output mechanism for DistCP due to its scalability.
  */
 public class ITestS3AContractDistCp extends AbstractContractDistCpTest {
 
-  private static final long MULTIPART_SETTING = 8 * 1024 * 1024; // 8 MB
+  private static final long MULTIPART_SETTING = MULTIPART_MIN_SIZE;
 
   @Override
   protected Configuration createConfiguration() {
     Configuration newConf = super.createConfiguration();
-    newConf.setLong(MIN_MULTIPART_THRESHOLD, MULTIPART_SETTING);
     newConf.setLong(MULTIPART_SIZE, MULTIPART_SETTING);
+    newConf.setBoolean(FAST_UPLOAD, true);
+    newConf.set(FAST_UPLOAD_BUFFER, FAST_UPLOAD_BUFFER_DISK);
     return newConf;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
index b7973b3..e049fd1 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
@@ -48,6 +48,7 @@ public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase
   @Override
   public void teardown() throws Exception {
     super.teardown();
+    describe("closing file system");
     IOUtils.closeStream(getFileSystem());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java
index b0b8a65..b1b8240 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -20,18 +20,23 @@ package org.apache.hadoop.fs.s3a;
 
 import com.google.common.util.concurrent.ListenableFuture;
 import org.apache.hadoop.util.StopWatch;
-import org.junit.*;
+
+import org.junit.AfterClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 
 /**
- * Basic unit test for S3A's blocking executor service.
+ * Basic test for S3A's blocking executor service.
  */
 public class ITestBlockingThreadPoolExecutorService {
 
@@ -47,7 +52,10 @@ public class ITestBlockingThreadPoolExecutorService {
 
   private static final Integer SOME_VALUE = 1337;
 
-  private static BlockingThreadPoolExecutorService tpe = null;
+  private static BlockingThreadPoolExecutorService tpe;
+
+  @Rule
+  public Timeout testTimeout = new Timeout(60 * 1000);
 
   @AfterClass
   public static void afterClass() throws Exception {
@@ -71,13 +79,23 @@ public class ITestBlockingThreadPoolExecutorService {
   @Test
   public void testSubmitRunnable() throws Exception {
     ensureCreated();
-    int totalTasks = NUM_ACTIVE_TASKS + NUM_WAITING_TASKS;
+    verifyQueueSize(tpe, NUM_ACTIVE_TASKS + NUM_WAITING_TASKS);
+  }
+
+  /**
+   * Verify the size of the executor's queue, by verifying that the first
+   * submission to block is {@code expectedQueueSize + 1}.
+   * @param executorService executor service to test
+   * @param expectedQueueSize size of queue
+   */
+  protected void verifyQueueSize(ExecutorService executorService,
+      int expectedQueueSize) {
     StopWatch stopWatch = new StopWatch().start();
-    for (int i = 0; i < totalTasks; i++) {
-      tpe.submit(sleeper);
+    for (int i = 0; i < expectedQueueSize; i++) {
+      executorService.submit(sleeper);
       assertDidntBlock(stopWatch);
     }
-    tpe.submit(sleeper);
+    executorService.submit(sleeper);
     assertDidBlock(stopWatch);
   }
 
@@ -93,6 +111,15 @@ public class ITestBlockingThreadPoolExecutorService {
     ensureDestroyed();
   }
 
+  @Test
+  public void testChainedQueue() throws Throwable {
+    ensureCreated();
+    int size = 2;
+    ExecutorService wrapper = new SemaphoredDelegatingExecutor(tpe,
+        size, true);
+    verifyQueueSize(wrapper, size);
+  }
+
   // Helper functions, etc.
 
   private void assertDidntBlock(StopWatch sw) {
@@ -141,8 +168,9 @@ public class ITestBlockingThreadPoolExecutorService {
   private static void ensureCreated() throws Exception {
     if (tpe == null) {
       LOG.debug("Creating thread pool");
-      tpe = new BlockingThreadPoolExecutorService(NUM_ACTIVE_TASKS,
-          NUM_WAITING_TASKS, 1, TimeUnit.SECONDS, "btpetest");
+      tpe = BlockingThreadPoolExecutorService.newInstance(
+          NUM_ACTIVE_TASKS, NUM_WAITING_TASKS,
+          1, TimeUnit.SECONDS, "btpetest");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java
new file mode 100644
index 0000000..74cad00
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.io.IOUtils;
+
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.s3a.Constants.*;
+
+/**
+ * Tests small file upload functionality for
+ * {@link S3ABlockOutputStream} with the blocks buffered in byte arrays.
+ *
+ * File sizes are kept small to reduce test duration on slow connections;
+ * multipart tests are kept in scale tests.
+ */
+public class ITestS3ABlockOutputArray extends AbstractS3ATestBase {
+
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    S3ATestUtils.disableFilesystemCaching(conf);
+    conf.setLong(MIN_MULTIPART_THRESHOLD, MULTIPART_MIN_SIZE);
+    conf.setInt(MULTIPART_SIZE, MULTIPART_MIN_SIZE);
+    conf.setBoolean(Constants.FAST_UPLOAD, true);
+    conf.set(FAST_UPLOAD_BUFFER, getBlockOutputBufferName());
+    return conf;
+  }
+
+  protected String getBlockOutputBufferName() {
+    return FAST_UPLOAD_BUFFER_ARRAY;
+  }
+
+  @Test
+  public void testZeroByteUpload() throws IOException {
+    verifyUpload("0", 0);
+  }
+
+  @Test
+  public void testRegularUpload() throws IOException {
+    verifyUpload("regular", 1024);
+  }
+
+  @Test(expected = IOException.class)
+  public void testDoubleStreamClose() throws Throwable {
+    Path dest = path("testDoubleStreamClose");
+    describe(" testDoubleStreamClose");
+    FSDataOutputStream stream = getFileSystem().create(dest, true);
+    byte[] data = ContractTestUtils.dataset(16, 'a', 26);
+    try {
+      stream.write(data);
+      stream.close();
+      stream.write(data);
+    } finally {
+      IOUtils.closeStream(stream);
+    }
+  }
+
+  public void verifyUpload(String name, int fileSize) throws IOException {
+    Path dest = path(name);
+    describe(name + " upload to " + dest);
+    ContractTestUtils.createAndVerifyFile(
+        getFileSystem(),
+        dest,
+        fileSize);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputByteBuffer.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputByteBuffer.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputByteBuffer.java
new file mode 100644
index 0000000..504426b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputByteBuffer.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+/**
+ * Use {@link Constants#FAST_UPLOAD_BYTEBUFFER} for buffering.
+ */
+public class ITestS3ABlockOutputByteBuffer extends ITestS3ABlockOutputArray {
+
+  protected String getBlockOutputBufferName() {
+    return Constants.FAST_UPLOAD_BYTEBUFFER;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputDisk.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputDisk.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputDisk.java
new file mode 100644
index 0000000..550706d
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputDisk.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+/**
+ * Use {@link Constants#FAST_UPLOAD_BUFFER_DISK} for buffering.
+ */
+public class ITestS3ABlockOutputDisk extends ITestS3ABlockOutputArray {
+
+  protected String getBlockOutputBufferName() {
+    return Constants.FAST_UPLOAD_BUFFER_DISK;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockingThreadPool.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockingThreadPool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockingThreadPool.java
index 4444d0c..991135e 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockingThreadPool.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockingThreadPool.java
@@ -72,6 +72,8 @@ public class ITestS3ABlockingThreadPool {
   @Test
   public void testFastMultiPartUpload() throws Exception {
     conf.setBoolean(Constants.FAST_UPLOAD, true);
+    conf.set(Constants.FAST_UPLOAD_BUFFER,
+        Constants.FAST_UPLOAD_BYTEBUFFER);
     fs = S3ATestUtils.createTestFileSystem(conf);
     ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 16 * 1024 *
         1024);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
index b08bfe9..30d4bf6 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
@@ -35,6 +36,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertEquals;
@@ -417,6 +419,33 @@ public class ITestS3AConfiguration {
     fs.close();
   }
 
+  @Test
+  public void testDirectoryAllocatorDefval() throws Throwable {
+    conf = new Configuration();
+    conf.unset(Constants.BUFFER_DIR);
+    fs = S3ATestUtils.createTestFileSystem(conf);
+    File tmp = fs.createTmpFileForWrite("out-", 1024, conf);
+    assertTrue("not found: " + tmp, tmp.exists());
+    tmp.delete();
+  }
+
+  @Test
+  public void testDirectoryAllocatorRR() throws Throwable {
+    File dir1 = GenericTestUtils.getRandomizedTestDir();
+    File dir2 = GenericTestUtils.getRandomizedTestDir();
+    dir1.mkdirs();
+    dir2.mkdirs();
+    conf = new Configuration();
+    conf.set(Constants.BUFFER_DIR, dir1 +", " + dir2);
+    fs = S3ATestUtils.createTestFileSystem(conf);
+    File tmp1 = fs.createTmpFileForWrite("out-", 1024, conf);
+    tmp1.delete();
+    File tmp2 = fs.createTmpFileForWrite("out-", 1024, conf);
+    tmp2.delete();
+    assertNotEquals("round robin not working",
+        tmp1.getParent(), tmp2.getParent());
+  }
+
   /**
    * Reads and returns a field from an object using reflection.  If the field
    * cannot be found, is null, or is not the expected type, then this method

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionBlockOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionBlockOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionBlockOutputStream.java
new file mode 100644
index 0000000..5239f30
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionBlockOutputStream.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Run the encryption tests against the block output stream.
+ */
+public class ITestS3AEncryptionBlockOutputStream extends ITestS3AEncryption {
+
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    conf.setBoolean(Constants.FAST_UPLOAD, true);
+    conf.set(Constants.FAST_UPLOAD_BUFFER,
+        Constants.FAST_UPLOAD_BYTEBUFFER);
+    return conf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionFastOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionFastOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionFastOutputStream.java
deleted file mode 100644
index c06fed1..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionFastOutputStream.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Run the encryption tests against the Fast output stream.
- * This verifies that both file writing paths can encrypt their data.
- */
-public class ITestS3AEncryptionFastOutputStream extends ITestS3AEncryption {
-
-  @Override
-  protected Configuration createConfiguration() {
-    Configuration conf = super.createConfiguration();
-    conf.setBoolean(Constants.FAST_UPLOAD, true);
-    return conf;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFastOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFastOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFastOutputStream.java
deleted file mode 100644
index b5fa1c3..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFastOutputStream.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-
-import java.io.IOException;
-
-/**
- * Tests regular and multi-part upload functionality for S3AFastOutputStream.
- * File sizes are kept small to reduce test duration on slow connections
- */
-public class ITestS3AFastOutputStream {
-  private FileSystem fs;
-
-
-  @Rule
-  public Timeout testTimeout = new Timeout(30 * 60 * 1000);
-
-  @Before
-  public void setUp() throws Exception {
-    Configuration conf = new Configuration();
-    conf.setLong(Constants.MIN_MULTIPART_THRESHOLD, 5 * 1024 * 1024);
-    conf.setInt(Constants.MULTIPART_SIZE, 5 * 1024 * 1024);
-    conf.setBoolean(Constants.FAST_UPLOAD, true);
-    fs = S3ATestUtils.createTestFileSystem(conf);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (fs != null) {
-      fs.delete(getTestPath(), true);
-    }
-  }
-
-  protected Path getTestPath() {
-    return new Path("/tests3a");
-  }
-
-  @Test
-  public void testRegularUpload() throws IOException {
-    ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 1024 * 1024);
-  }
-
-  @Test
-  public void testMultiPartUpload() throws IOException {
-    ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 6 * 1024 *
-        1024);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATestUtils.java
new file mode 100644
index 0000000..88204b2
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATestUtils.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
+
+/**
+ * Test the test utils. Why an integration test? it's needed to
+ * verify property pushdown.
+ */
+public class ITestS3ATestUtils extends Assert {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ITestS3ATestUtils.class);
+  public static final String KEY = "undefined.property";
+
+  @Before
+  public void clear() {
+    System.clearProperty(KEY);
+  }
+
+  @Test
+  public void testGetTestProperty() throws Throwable {
+    Configuration conf = new Configuration(false);
+    assertEquals("a", getTestProperty(conf, KEY, "a"));
+    conf.set(KEY, "\t b \n");
+    assertEquals("b", getTestProperty(conf, KEY, "a"));
+    System.setProperty(KEY, "c");
+    assertEquals("c", getTestProperty(conf, KEY, "a"));
+    unsetSysprop();
+    assertEquals("b", getTestProperty(conf, KEY, "a"));
+  }
+
+  @Test
+  public void testGetTestPropertyLong() throws Throwable {
+    Configuration conf = new Configuration(false);
+    assertEquals(1, getTestPropertyLong(conf, KEY, 1));
+    conf.setInt(KEY, 2);
+    assertEquals(2, getTestPropertyLong(conf, KEY, 1));
+    System.setProperty(KEY, "3");
+    assertEquals(3, getTestPropertyLong(conf, KEY, 1));
+  }
+
+  @Test
+  public void testGetTestPropertyInt() throws Throwable {
+    Configuration conf = new Configuration(false);
+    assertEquals(1, getTestPropertyInt(conf, KEY, 1));
+    conf.setInt(KEY, 2);
+    assertEquals(2, getTestPropertyInt(conf, KEY, 1));
+    System.setProperty(KEY, "3");
+    assertEquals(3, getTestPropertyInt(conf, KEY, 1));
+    conf.unset(KEY);
+    assertEquals(3, getTestPropertyInt(conf, KEY, 1));
+    unsetSysprop();
+    assertEquals(5, getTestPropertyInt(conf, KEY, 5));
+  }
+
+  @Test
+  public void testGetTestPropertyBool() throws Throwable {
+    Configuration conf = new Configuration(false);
+    assertTrue(getTestPropertyBool(conf, KEY, true));
+    conf.set(KEY, "\tfalse \n");
+    assertFalse(getTestPropertyBool(conf, KEY, true));
+    System.setProperty(KEY, "true");
+    assertTrue(getTestPropertyBool(conf, KEY, true));
+    unsetSysprop();
+    assertEquals("false", getTestProperty(conf, KEY, "true"));
+    conf.unset(KEY);
+    assertTrue(getTestPropertyBool(conf, KEY, true));
+  }
+
+  protected void unsetSysprop() {
+    System.setProperty(KEY, UNSET_PROPERTY);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
index 6a4e68c..6894bb0 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
@@ -44,14 +44,35 @@ public interface S3ATestConstants {
   String TEST_FS_S3A_NAME = TEST_FS_S3A + "name";
 
   /**
+   * Run the encryption tests?
+   */
+  String KEY_ENCRYPTION_TESTS = TEST_FS_S3A + "encryption.enabled";
+
+  /**
+   * Tell tests that they are being executed in parallel: {@value}.
+   */
+  String KEY_PARALLEL_TEST_EXECUTION = "test.parallel.execution";
+
+  /**
+   * A property set to true in maven if scale tests are enabled: {@value}.
+   */
+  String KEY_SCALE_TESTS_ENABLED = S3A_SCALE_TEST + "enabled";
+
+  /**
    * The number of operations to perform: {@value}.
    */
   String KEY_OPERATION_COUNT = SCALE_TEST + "operation.count";
 
   /**
+   * The number of directory operations to perform: {@value}.
+   */
+  String KEY_DIRECTORY_COUNT = SCALE_TEST + "directory.count";
+
+  /**
    * The readahead buffer: {@value}.
    */
   String KEY_READ_BUFFER_SIZE = S3A_SCALE_TEST + "read.buffer.size";
+
   int DEFAULT_READ_BUFFER_SIZE = 16384;
 
   /**
@@ -65,12 +86,62 @@ public interface S3ATestConstants {
   String DEFAULT_CSVTEST_FILE = "s3a://landsat-pds/scene_list.gz";
 
   /**
+   * Endpoint for the S3 CSV/scale tests. This defaults to
+   * being us-east.
+   */
+  String KEY_CSVTEST_ENDPOINT = S3A_SCALE_TEST + "csvfile.endpoint";
+
+  /**
+   * Endpoint for the S3 CSV/scale tests. This defaults to
+   * being us-east.
+   */
+  String DEFAULT_CSVTEST_ENDPOINT = "s3.amazonaws.com";
+
+  /**
+   * Name of the property to define the timeout for scale tests: {@value}.
+   * Measured in seconds.
+   */
+  String KEY_TEST_TIMEOUT = S3A_SCALE_TEST + "timeout";
+
+  /**
+   * Name of the property to define the file size for the huge file
+   * tests: {@value}.
+   * Measured in KB; a suffix like "M", or "G" will change the unit.
+   */
+  String KEY_HUGE_FILESIZE = S3A_SCALE_TEST + "huge.filesize";
+
+  /**
+   * Name of the property to define the partition size for the huge file
+   * tests: {@value}.
+   * Measured in KB; a suffix like "M", or "G" will change the unit.
+   */
+  String KEY_HUGE_PARTITION_SIZE = S3A_SCALE_TEST + "huge.partitionsize";
+
+  /**
+   * The default huge size is small \u2014full 5GB+ scale tests are something
+   * to run in long test runs on EC2 VMs. {@value}.
+   */
+  String DEFAULT_HUGE_FILESIZE = "10M";
+
+  /**
    * The default number of operations to perform: {@value}.
    */
   long DEFAULT_OPERATION_COUNT = 2005;
 
   /**
-   * Run the encryption tests?
+   * Default number of directories to create when performing
+   * directory performance/scale tests.
    */
-  String KEY_ENCRYPTION_TESTS = TEST_FS_S3A + "encryption.enabled";
+  int DEFAULT_DIRECTORY_COUNT = 2;
+
+  /**
+   * Default scale test timeout in seconds: {@value}.
+   */
+  int DEFAULT_TEST_TIMEOUT = 30 * 60;
+
+  /**
+   * Default policy on scale tests: {@value}.
+   */
+  boolean DEFAULT_SCALE_TESTS_ENABLED = false;
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
index 95f6d4b..c67e118 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
@@ -40,6 +40,12 @@ import static org.apache.hadoop.fs.s3a.Constants.*;
 public class S3ATestUtils {
 
   /**
+   * Value to set a system property to (in maven) to declare that
+   * a property has been unset.
+   */
+  public static final String UNSET_PROPERTY = "unset";
+
+  /**
    * Create the test filesystem.
    *
    * If the test.fs.s3a.name property is not set, this will
@@ -53,8 +59,25 @@ public class S3ATestUtils {
    */
   public static S3AFileSystem createTestFileSystem(Configuration conf)
       throws IOException {
-    String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, "");
+    return createTestFileSystem(conf, true);
+  }
 
+  /**
+   * Create the test filesystem with or without multipart purging
+   *
+   * If the test.fs.s3a.name property is not set, this will
+   * trigger a JUnit failure.
+   * @param conf configuration
+   * @param purge flag to enable Multipart purging
+   * @return the FS
+   * @throws IOException IO Problems
+   * @throws AssumptionViolatedException if the FS is not named
+   */
+  public static S3AFileSystem createTestFileSystem(Configuration conf,
+      boolean purge)
+      throws IOException {
+
+    String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, "");
 
     boolean liveTest = !StringUtils.isEmpty(fsname);
     URI testURI = null;
@@ -70,8 +93,12 @@ public class S3ATestUtils {
     }
     S3AFileSystem fs1 = new S3AFileSystem();
     //enable purging in tests
-    conf.setBoolean(PURGE_EXISTING_MULTIPART, true);
-    conf.setInt(PURGE_EXISTING_MULTIPART_AGE, 0);
+    if (purge) {
+      conf.setBoolean(PURGE_EXISTING_MULTIPART, true);
+      // but a long delay so that parallel multipart tests don't
+      // suddenly start timing out
+      conf.setInt(PURGE_EXISTING_MULTIPART_AGE, 30 * 60);
+    }
     fs1.initialize(testURI, conf);
     return fs1;
   }
@@ -149,6 +176,121 @@ public class S3ATestUtils {
   }
 
   /**
+   * Get a long test property.
+   * <ol>
+   *   <li>Look up configuration value (which can pick up core-default.xml),
+   *       using {@code defVal} as the default value (if conf != null).
+   *   </li>
+   *   <li>Fetch the system property.</li>
+   *   <li>If the system property is not empty or "(unset)":
+   *   it overrides the conf value.
+   *   </li>
+   * </ol>
+   * This puts the build properties in charge of everything. It's not a
+   * perfect design; having maven set properties based on a file, as ant let
+   * you do, is better for customization.
+   *
+   * As to why there's a special (unset) value, see
+   * {@link http://stackoverflow.com/questions/7773134/null-versus-empty-arguments-in-maven}
+   * @param conf config: may be null
+   * @param key key to look up
+   * @param defVal default value
+   * @return the evaluated test property.
+   */
+  public static long getTestPropertyLong(Configuration conf,
+      String key, long defVal) {
+    return Long.valueOf(
+        getTestProperty(conf, key, Long.toString(defVal)));
+  }
+  /**
+   * Get a test property value in bytes, using k, m, g, t, p, e suffixes.
+   * {@link org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix#string2long(String)}
+   * <ol>
+   *   <li>Look up configuration value (which can pick up core-default.xml),
+   *       using {@code defVal} as the default value (if conf != null).
+   *   </li>
+   *   <li>Fetch the system property.</li>
+   *   <li>If the system property is not empty or "(unset)":
+   *   it overrides the conf value.
+   *   </li>
+   * </ol>
+   * This puts the build properties in charge of everything. It's not a
+   * perfect design; having maven set properties based on a file, as ant let
+   * you do, is better for customization.
+   *
+   * As to why there's a special (unset) value, see
+   * {@link http://stackoverflow.com/questions/7773134/null-versus-empty-arguments-in-maven}
+   * @param conf config: may be null
+   * @param key key to look up
+   * @param defVal default value
+   * @return the evaluated test property.
+   */
+  public static long getTestPropertyBytes(Configuration conf,
+      String key, String defVal) {
+    return org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix
+        .string2long(getTestProperty(conf, key, defVal));
+  }
+
+  /**
+   * Get an integer test property; algorithm described in
+   * {@link #getTestPropertyLong(Configuration, String, long)}.
+   * @param key key to look up
+   * @param defVal default value
+   * @return the evaluated test property.
+   */
+  public static int getTestPropertyInt(Configuration conf,
+      String key, int defVal) {
+    return (int) getTestPropertyLong(conf, key, defVal);
+  }
+
+  /**
+   * Get a boolean test property; algorithm described in
+   * {@link #getTestPropertyLong(Configuration, String, long)}.
+   * @param key key to look up
+   * @param defVal default value
+   * @return the evaluated test property.
+   */
+  public static boolean getTestPropertyBool(Configuration conf,
+      String key,
+      boolean defVal) {
+    return Boolean.valueOf(
+        getTestProperty(conf, key, Boolean.toString(defVal)));
+  }
+
+  /**
+   * Get a string test property.
+   * <ol>
+   *   <li>Look up configuration value (which can pick up core-default.xml),
+   *       using {@code defVal} as the default value (if conf != null).
+   *   </li>
+   *   <li>Fetch the system property.</li>
+   *   <li>If the system property is not empty or "(unset)":
+   *   it overrides the conf value.
+   *   </li>
+   * </ol>
+   * This puts the build properties in charge of everything. It's not a
+   * perfect design; having maven set properties based on a file, as ant let
+   * you do, is better for customization.
+   *
+   * As to why there's a special (unset) value, see
+   * @see <a href="http://stackoverflow.com/questions/7773134/null-versus-empty-arguments-in-maven">
+   *   Stack Overflow</a>
+   * @param conf config: may be null
+   * @param key key to look up
+   * @param defVal default value
+   * @return the evaluated test property.
+   */
+
+  public static String getTestProperty(Configuration conf,
+      String key,
+      String defVal) {
+    String confVal = conf != null ? conf.getTrimmed(key, defVal) : defVal;
+    String propval = System.getProperty(key);
+    return StringUtils.isNotEmpty(propval) && !UNSET_PROPERTY.equals(propval)
+        ? propval : confVal;
+  }
+
+  /**
    * The exception to raise so as to exit fast from
    * {@link #eventually(int, Callable)}.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java
new file mode 100644
index 0000000..9fa95fd
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+/**
+ * Unit tests for {@link S3ADataBlocks}.
+ */
+public class TestDataBlocks extends Assert {
+
+  @Rule
+  public Timeout testTimeout = new Timeout(30 * 1000);
+
+  @Before
+  public void nameThread() {
+    Thread.currentThread().setName("JUnit");
+  }
+
+  /**
+   * Test the {@link S3ADataBlocks.ByteBufferBlockFactory}.
+   * That code implements an input stream over a ByteBuffer, and has to
+   * return the buffer to the pool after the read complete.
+   *
+   * This test verifies the basic contract of the process.
+   */
+  @Test
+  public void testByteBufferIO() throws Throwable {
+    try (S3ADataBlocks.ByteBufferBlockFactory factory =
+             new S3ADataBlocks.ByteBufferBlockFactory(null)) {
+      int limit = 128;
+      S3ADataBlocks.ByteBufferBlockFactory.ByteBufferBlock block
+          = factory.create(limit);
+      assertEquals("outstanding buffers in " + factory,
+          1, factory.getOutstandingBufferCount());
+
+      byte[] buffer = ContractTestUtils.toAsciiByteArray("test data");
+      int bufferLen = buffer.length;
+      block.write(buffer, 0, bufferLen);
+      assertEquals(bufferLen, block.dataSize());
+      assertEquals("capacity in " + block,
+          limit - bufferLen, block.remainingCapacity());
+      assertTrue("hasCapacity(64) in " + block, block.hasCapacity(64));
+      assertTrue("No capacity in " + block,
+          block.hasCapacity(limit - bufferLen));
+
+      // now start the write
+      S3ADataBlocks.ByteBufferBlockFactory.ByteBufferInputStream
+          stream = block.startUpload();
+      assertTrue("!hasRemaining() in " + stream, stream.hasRemaining());
+      int expected = bufferLen;
+      assertEquals("wrong available() in " + stream,
+          expected, stream.available());
+
+      assertEquals('t', stream.read());
+      expected--;
+      assertEquals("wrong available() in " + stream,
+          expected, stream.available());
+
+      // close the block. The buffer must remain outstanding here;
+      // the stream manages the lifecycle of it now
+      block.close();
+      assertEquals("outstanding buffers in " + factory,
+          1, factory.getOutstandingBufferCount());
+      block.close();
+
+      // read into a byte array with an offset
+      int offset = 5;
+      byte[] in = new byte[limit];
+      assertEquals(2, stream.read(in, offset, 2));
+      assertEquals('e', in[offset]);
+      assertEquals('s', in[offset + 1]);
+      expected -= 2;
+      assertEquals("wrong available() in " + stream,
+          expected, stream.available());
+
+      // read to end
+      byte[] remainder = new byte[limit];
+      int c;
+      int index = 0;
+      while ((c = stream.read()) >= 0) {
+        remainder[index++] = (byte) c;
+      }
+      assertEquals(expected, index);
+      assertEquals('a', remainder[--index]);
+
+      assertEquals("wrong available() in " + stream,
+          0, stream.available());
+      assertTrue("hasRemaining() in " + stream, !stream.hasRemaining());
+
+      // when the stream is closed, the data should be returned
+      stream.close();
+      assertEquals("outstanding buffers in " + factory,
+          0, factory.getOutstandingBufferCount());
+      stream.close();
+      assertEquals("outstanding buffers in " + factory,
+          0, factory.getOutstandingBufferCount());
+
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
index 5e88aba..e1aef75 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
@@ -34,6 +34,7 @@ public class ITestS3AFileContextStatistics extends FCStatisticsBaseTest {
     fc = S3ATestUtils.createTestFileContext(conf);
     fc.mkdir(fileContextTestHelper.getTestRootPath(fc, "test"),
         FileContext.DEFAULT_PERM, true);
+    FileContext.clearStatistics();
   }
 
   @After


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


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

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


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

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

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/adb96e10/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 8c59186..563682f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -4999,7 +4999,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         BlockInfo blk = blkIterator.next();
         final INodeFile inode = getBlockCollection(blk);
         skip++;
-        if (inode != null && blockManager.countNodes(blk).liveReplicas() == 0) {
+        if (inode != null) {
           String src = inode.getFullPathName();
           if (src.startsWith(path)){
             corruptFiles.add(new CorruptFileBlockInfo(src, blk));

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


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


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

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


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

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

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


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

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

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


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


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

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

This reverts commit 412c4c9a342b73bf1c1a7f43ea91245cbf94d02d.


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

Branch: refs/heads/HADOOP-13070
Commit: 0bc6d37f3c1e7c2a8682dffa95461a884bd6ba17
Parents: b61fb26
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Mon Oct 17 22:47:37 2016 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Mon Oct 17 22:47:37 2016 -0700

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


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

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

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

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

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

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

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

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

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

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

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

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

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


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


[50/50] [abbrv] hadoop git commit: HADOOP-13400. Update the ApplicationClassLoader implementation in line with latest Java ClassLoader implementation. Contributed by Vrushali C.

Posted by sj...@apache.org.
HADOOP-13400. Update the ApplicationClassLoader implementation in line with latest Java ClassLoader implementation. Contributed by Vrushali C.


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

Branch: refs/heads/HADOOP-13070
Commit: f0e56e36498a3bb793b882c55efe118d355eeae3
Parents: c62ae71
Author: Sangjin Lee <sj...@apache.org>
Authored: Tue Oct 18 16:44:06 2016 -0700
Committer: Sangjin Lee <sj...@apache.org>
Committed: Tue Oct 18 16:44:06 2016 -0700

----------------------------------------------------------------------
 .../hadoop/util/ApplicationClassLoader.java     | 66 ++++++++++----------
 1 file changed, 33 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0e56e36/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java
index 2f46e1f..9b89889 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ApplicationClassLoader.java
@@ -158,49 +158,49 @@ public class ApplicationClassLoader extends URLClassLoader {
   }
 
   @Override
-  protected synchronized Class<?> loadClass(String name, boolean resolve)
+  protected Class<?> loadClass(String name, boolean resolve)
       throws ClassNotFoundException {
-    
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Loading class: " + name);
-    }
+    synchronized (getClassLoadingLock(name)) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Loading class: " + name);
+      }
 
-    Class<?> c = findLoadedClass(name);
-    ClassNotFoundException ex = null;
+      Class<?> c = findLoadedClass(name);
+      ClassNotFoundException ex = null;
+
+      if (c == null && !isSystemClass(name, systemClasses)) {
+        // Try to load class from this classloader's URLs. Note that this is
+        // like the servlet spec, not the usual Java 2 behaviour where we ask
+        // the parent to attempt to load first.
+        try {
+          c = findClass(name);
+          if (LOG.isDebugEnabled() && c != null) {
+            LOG.debug("Loaded class: " + name + " ");
+          }
+        } catch (ClassNotFoundException e) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(e);
+          }
+          ex = e;
+        }
+      }
 
-    if (c == null && !isSystemClass(name, systemClasses)) {
-      // Try to load class from this classloader's URLs. Note that this is like
-      // the servlet spec, not the usual Java 2 behaviour where we ask the
-      // parent to attempt to load first.
-      try {
-        c = findClass(name);
+      if (c == null) { // try parent
+        c = parent.loadClass(name);
         if (LOG.isDebugEnabled() && c != null) {
-          LOG.debug("Loaded class: " + name + " ");
+          LOG.debug("Loaded class from parent: " + name + " ");
         }
-      } catch (ClassNotFoundException e) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(e);
-        }
-        ex = e;
       }
-    }
 
-    if (c == null) { // try parent
-      c = parent.loadClass(name);
-      if (LOG.isDebugEnabled() && c != null) {
-        LOG.debug("Loaded class from parent: " + name + " ");
+      if (c == null) {
+        throw ex != null ? ex : new ClassNotFoundException(name);
       }
-    }
 
-    if (c == null) {
-      throw ex != null ? ex : new ClassNotFoundException(name);
-    }
-
-    if (resolve) {
-      resolveClass(c);
+      if (resolve) {
+        resolveClass(c);
+      }
+      return c;
     }
-
-    return c;
   }
 
   /**


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


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

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


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

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

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


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

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


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


[47/50] [abbrv] hadoop git commit: HADOOP-13560. S3ABlockOutputStream to support huge (many GB) file writes. Contributed by Steve Loughran

Posted by sj...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java
deleted file mode 100644
index c25d0fb..0000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java
+++ /dev/null
@@ -1,410 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a;
-
-import com.amazonaws.AmazonClientException;
-import com.amazonaws.event.ProgressEvent;
-import com.amazonaws.event.ProgressListener;
-import com.amazonaws.services.s3.AmazonS3;
-import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
-import com.amazonaws.services.s3.model.CannedAccessControlList;
-import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
-import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
-import com.amazonaws.services.s3.model.ObjectMetadata;
-import com.amazonaws.services.s3.model.PartETag;
-import com.amazonaws.services.s3.model.PutObjectRequest;
-import com.amazonaws.services.s3.model.PutObjectResult;
-import com.amazonaws.services.s3.model.UploadPartRequest;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.util.Progressable;
-import org.slf4j.Logger;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.List;
-
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-
-import static org.apache.hadoop.fs.s3a.S3AUtils.*;
-import static org.apache.hadoop.fs.s3a.Statistic.*;
-
-/**
- * Upload files/parts asap directly from a memory buffer (instead of buffering
- * to a file).
- * <p>
- * Uploads are managed low-level rather than through the AWS TransferManager.
- * This allows for uploading each part of a multi-part upload as soon as
- * the bytes are in memory, rather than waiting until the file is closed.
- * <p>
- * Unstable: statistics and error handling might evolve
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class S3AFastOutputStream extends OutputStream {
-
-  private static final Logger LOG = S3AFileSystem.LOG;
-  private final String key;
-  private final String bucket;
-  private final AmazonS3 client;
-  private final int partSize;
-  private final int multiPartThreshold;
-  private final S3AFileSystem fs;
-  private final CannedAccessControlList cannedACL;
-  private final ProgressListener progressListener;
-  private final ListeningExecutorService executorService;
-  private MultiPartUpload multiPartUpload;
-  private boolean closed;
-  private ByteArrayOutputStream buffer;
-  private int bufferLimit;
-
-
-  /**
-   * Creates a fast OutputStream that uploads to S3 from memory.
-   * For MultiPartUploads, as soon as sufficient bytes have been written to
-   * the stream a part is uploaded immediately (by using the low-level
-   * multi-part upload API on the AmazonS3Client).
-   *
-   * @param client AmazonS3Client used for S3 calls
-   * @param fs S3AFilesystem
-   * @param bucket S3 bucket name
-   * @param key S3 key name
-   * @param progress report progress in order to prevent timeouts
-   * @param cannedACL used CannedAccessControlList
-   * @param partSize size of a single part in a multi-part upload (except
-   * last part)
-   * @param multiPartThreshold files at least this size use multi-part upload
-   * @param threadPoolExecutor thread factory
-   * @throws IOException on any problem
-   */
-  public S3AFastOutputStream(AmazonS3 client,
-      S3AFileSystem fs,
-      String bucket,
-      String key,
-      Progressable progress,
-      CannedAccessControlList cannedACL,
-      long partSize,
-      long multiPartThreshold,
-      ExecutorService threadPoolExecutor)
-      throws IOException {
-    this.bucket = bucket;
-    this.key = key;
-    this.client = client;
-    this.fs = fs;
-    this.cannedACL = cannedACL;
-    //Ensure limit as ByteArrayOutputStream size cannot exceed Integer.MAX_VALUE
-    if (partSize > Integer.MAX_VALUE) {
-      this.partSize = Integer.MAX_VALUE;
-      LOG.warn("s3a: MULTIPART_SIZE capped to ~2.14GB (maximum allowed size " +
-          "when using 'FAST_UPLOAD = true')");
-    } else {
-      this.partSize = (int) partSize;
-    }
-    if (multiPartThreshold > Integer.MAX_VALUE) {
-      this.multiPartThreshold = Integer.MAX_VALUE;
-      LOG.warn("s3a: MIN_MULTIPART_THRESHOLD capped to ~2.14GB (maximum " +
-          "allowed size when using 'FAST_UPLOAD = true')");
-    } else {
-      this.multiPartThreshold = (int) multiPartThreshold;
-    }
-    this.bufferLimit = this.multiPartThreshold;
-    this.closed = false;
-    int initialBufferSize = this.fs.getConf()
-        .getInt(Constants.FAST_BUFFER_SIZE, Constants.DEFAULT_FAST_BUFFER_SIZE);
-    if (initialBufferSize < 0) {
-      LOG.warn("s3a: FAST_BUFFER_SIZE should be a positive number. Using " +
-          "default value");
-      initialBufferSize = Constants.DEFAULT_FAST_BUFFER_SIZE;
-    } else if (initialBufferSize > this.bufferLimit) {
-      LOG.warn("s3a: automatically adjusting FAST_BUFFER_SIZE to not " +
-          "exceed MIN_MULTIPART_THRESHOLD");
-      initialBufferSize = this.bufferLimit;
-    }
-    this.buffer = new ByteArrayOutputStream(initialBufferSize);
-    this.executorService = MoreExecutors.listeningDecorator(threadPoolExecutor);
-    this.multiPartUpload = null;
-    this.progressListener = new ProgressableListener(progress);
-    LOG.debug("Initialized S3AFastOutputStream for bucket '{}' key '{}'",
-        bucket, key);
-  }
-
-  /**
-   * Writes a byte to the memory buffer. If this causes the buffer to reach
-   * its limit, the actual upload is submitted to the threadpool.
-   * @param b the int of which the lowest byte is written
-   * @throws IOException on any problem
-   */
-  @Override
-  public synchronized void write(int b) throws IOException {
-    buffer.write(b);
-    if (buffer.size() == bufferLimit) {
-      uploadBuffer();
-    }
-  }
-
-  /**
-   * Writes a range of bytes from to the memory buffer. If this causes the
-   * buffer to reach its limit, the actual upload is submitted to the
-   * threadpool and the remainder of the array is written to memory
-   * (recursively).
-   * @param b byte array containing
-   * @param off offset in array where to start
-   * @param len number of bytes to be written
-   * @throws IOException on any problem
-   */
-  @Override
-  public synchronized void write(byte[] b, int off, int len)
-      throws IOException {
-    if (b == null) {
-      throw new NullPointerException();
-    } else if ((off < 0) || (off > b.length) || (len < 0) ||
-        ((off + len) > b.length) || ((off + len) < 0)) {
-      throw new IndexOutOfBoundsException();
-    } else if (len == 0) {
-      return;
-    }
-    if (buffer.size() + len < bufferLimit) {
-      buffer.write(b, off, len);
-    } else {
-      int firstPart = bufferLimit - buffer.size();
-      buffer.write(b, off, firstPart);
-      uploadBuffer();
-      this.write(b, off + firstPart, len - firstPart);
-    }
-  }
-
-  private synchronized void uploadBuffer() throws IOException {
-    if (multiPartUpload == null) {
-      multiPartUpload = initiateMultiPartUpload();
-       /* Upload the existing buffer if it exceeds partSize. This possibly
-       requires multiple parts! */
-      final byte[] allBytes = buffer.toByteArray();
-      buffer = null; //earlier gc?
-      LOG.debug("Total length of initial buffer: {}", allBytes.length);
-      int processedPos = 0;
-      while ((multiPartThreshold - processedPos) >= partSize) {
-        LOG.debug("Initial buffer: processing from byte {} to byte {}",
-            processedPos, (processedPos + partSize - 1));
-        multiPartUpload.uploadPartAsync(new ByteArrayInputStream(allBytes,
-            processedPos, partSize), partSize);
-        processedPos += partSize;
-      }
-      //resize and reset stream
-      bufferLimit = partSize;
-      buffer = new ByteArrayOutputStream(bufferLimit);
-      buffer.write(allBytes, processedPos, multiPartThreshold - processedPos);
-    } else {
-      //upload next part
-      multiPartUpload.uploadPartAsync(new ByteArrayInputStream(buffer
-          .toByteArray()), partSize);
-      buffer.reset();
-    }
-  }
-
-  /**
-   * Close the stream. This will not return until the upload is complete
-   * or the attempt to perform the upload has failed.
-   * Exceptions raised in this method are indicative that the write has
-   * failed and data is at risk of being lost.
-   * @throws IOException on any failure.
-   */
-  @Override
-  public synchronized void close() throws IOException {
-    if (closed) {
-      return;
-    }
-    closed = true;
-    try {
-      if (multiPartUpload == null) {
-        putObject();
-      } else {
-        int size = buffer.size();
-        if (size > 0) {
-          fs.incrementPutStartStatistics(size);
-          //send last part
-          multiPartUpload.uploadPartAsync(new ByteArrayInputStream(buffer
-              .toByteArray()), size);
-        }
-        final List<PartETag> partETags = multiPartUpload
-            .waitForAllPartUploads();
-        multiPartUpload.complete(partETags);
-      }
-      // This will delete unnecessary fake parent directories
-      fs.finishedWrite(key);
-      LOG.debug("Upload complete for bucket '{}' key '{}'", bucket, key);
-    } finally {
-      buffer = null;
-      super.close();
-    }
-  }
-
-  /**
-   * Create the default metadata for a multipart upload operation.
-   * @return the metadata to use/extend.
-   */
-  private ObjectMetadata createDefaultMetadata() {
-    return fs.newObjectMetadata();
-  }
-
-  private MultiPartUpload initiateMultiPartUpload() throws IOException {
-    final InitiateMultipartUploadRequest initiateMPURequest =
-        new InitiateMultipartUploadRequest(bucket,
-            key,
-            createDefaultMetadata());
-    initiateMPURequest.setCannedACL(cannedACL);
-    try {
-      return new MultiPartUpload(
-          client.initiateMultipartUpload(initiateMPURequest).getUploadId());
-    } catch (AmazonClientException ace) {
-      throw translateException("initiate MultiPartUpload", key, ace);
-    }
-  }
-
-  private void putObject() throws IOException {
-    LOG.debug("Executing regular upload for bucket '{}' key '{}'",
-        bucket, key);
-    final ObjectMetadata om = createDefaultMetadata();
-    final int size = buffer.size();
-    om.setContentLength(size);
-    final PutObjectRequest putObjectRequest =
-        fs.newPutObjectRequest(key,
-            om,
-            new ByteArrayInputStream(buffer.toByteArray()));
-    putObjectRequest.setGeneralProgressListener(progressListener);
-    ListenableFuture<PutObjectResult> putObjectResult =
-        executorService.submit(new Callable<PutObjectResult>() {
-          @Override
-          public PutObjectResult call() throws Exception {
-            fs.incrementPutStartStatistics(size);
-            return client.putObject(putObjectRequest);
-          }
-        });
-    //wait for completion
-    try {
-      putObjectResult.get();
-    } catch (InterruptedException ie) {
-      LOG.warn("Interrupted object upload: {}", ie, ie);
-      Thread.currentThread().interrupt();
-    } catch (ExecutionException ee) {
-      throw extractException("regular upload", key, ee);
-    }
-  }
-
-
-  private class MultiPartUpload {
-    private final String uploadId;
-    private final List<ListenableFuture<PartETag>> partETagsFutures;
-
-    public MultiPartUpload(String uploadId) {
-      this.uploadId = uploadId;
-      this.partETagsFutures = new ArrayList<ListenableFuture<PartETag>>();
-      LOG.debug("Initiated multi-part upload for bucket '{}' key '{}' with " +
-          "id '{}'", bucket, key, uploadId);
-    }
-
-    private void uploadPartAsync(ByteArrayInputStream inputStream,
-        int partSize) {
-      final int currentPartNumber = partETagsFutures.size() + 1;
-      final UploadPartRequest request =
-          new UploadPartRequest().withBucketName(bucket).withKey(key)
-              .withUploadId(uploadId).withInputStream(inputStream)
-              .withPartNumber(currentPartNumber).withPartSize(partSize);
-      request.setGeneralProgressListener(progressListener);
-      ListenableFuture<PartETag> partETagFuture =
-          executorService.submit(new Callable<PartETag>() {
-            @Override
-            public PartETag call() throws Exception {
-              LOG.debug("Uploading part {} for id '{}'", currentPartNumber,
-                  uploadId);
-              return fs.uploadPart(request).getPartETag();
-            }
-          });
-      partETagsFutures.add(partETagFuture);
-    }
-
-    private List<PartETag> waitForAllPartUploads() throws IOException {
-      try {
-        return Futures.allAsList(partETagsFutures).get();
-      } catch (InterruptedException ie) {
-        LOG.warn("Interrupted partUpload: {}", ie, ie);
-        Thread.currentThread().interrupt();
-        return null;
-      } catch (ExecutionException ee) {
-        //there is no way of recovering so abort
-        //cancel all partUploads
-        for (ListenableFuture<PartETag> future : partETagsFutures) {
-          future.cancel(true);
-        }
-        //abort multipartupload
-        this.abort();
-        throw extractException("Multi-part upload with id '" + uploadId + "'",
-            key, ee);
-      }
-    }
-
-    private void complete(List<PartETag> partETags) throws IOException {
-      try {
-        LOG.debug("Completing multi-part upload for key '{}', id '{}'",
-            key, uploadId);
-        client.completeMultipartUpload(
-            new CompleteMultipartUploadRequest(bucket,
-                key,
-                uploadId,
-                partETags));
-      } catch (AmazonClientException e) {
-        throw translateException("Completing multi-part upload", key, e);
-      }
-    }
-
-    public void abort() {
-      LOG.warn("Aborting multi-part upload with id '{}'", uploadId);
-      try {
-        fs.incrementStatistic(OBJECT_MULTIPART_UPLOAD_ABORTED);
-        client.abortMultipartUpload(new AbortMultipartUploadRequest(bucket,
-            key, uploadId));
-      } catch (Exception e2) {
-        LOG.warn("Unable to abort multipart upload, you may need to purge  " +
-            "uploaded parts: {}", e2, e2);
-      }
-    }
-  }
-
-  private static class ProgressableListener implements ProgressListener {
-    private final Progressable progress;
-
-    public ProgressableListener(Progressable progress) {
-      this.progress = progress;
-    }
-
-    public void progressChanged(ProgressEvent progressEvent) {
-      if (progress != null) {
-        progress.progress();
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index 85d1fc7..2354819 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -37,14 +37,20 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.AmazonServiceException;
 import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
 import com.amazonaws.services.s3.model.AmazonS3Exception;
 import com.amazonaws.services.s3.model.CannedAccessControlList;
+import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
+import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
+import com.amazonaws.services.s3.model.CopyObjectRequest;
 import com.amazonaws.services.s3.model.DeleteObjectsRequest;
+import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
 import com.amazonaws.services.s3.model.ListObjectsRequest;
 import com.amazonaws.services.s3.model.ObjectListing;
 import com.amazonaws.services.s3.model.ObjectMetadata;
+import com.amazonaws.services.s3.model.PartETag;
 import com.amazonaws.services.s3.model.PutObjectRequest;
-import com.amazonaws.services.s3.model.CopyObjectRequest;
+import com.amazonaws.services.s3.model.PutObjectResult;
 import com.amazonaws.services.s3.model.S3ObjectSummary;
 import com.amazonaws.services.s3.model.UploadPartRequest;
 import com.amazonaws.services.s3.model.UploadPartResult;
@@ -55,6 +61,8 @@ import com.amazonaws.services.s3.transfer.Upload;
 import com.amazonaws.event.ProgressListener;
 import com.amazonaws.event.ProgressEvent;
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ListeningExecutorService;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -68,6 +76,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.InvalidRequestException;
+import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
@@ -118,9 +127,12 @@ public class S3AFileSystem extends FileSystem {
   private long partSize;
   private boolean enableMultiObjectsDelete;
   private TransferManager transfers;
-  private ExecutorService threadPoolExecutor;
+  private ListeningExecutorService threadPoolExecutor;
   private long multiPartThreshold;
   public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class);
+  private static final Logger PROGRESS =
+      LoggerFactory.getLogger("org.apache.hadoop.fs.s3a.S3AFileSystem.Progress");
+  private LocalDirAllocator directoryAllocator;
   private CannedAccessControlList cannedACL;
   private String serverSideEncryptionAlgorithm;
   private S3AInstrumentation instrumentation;
@@ -131,6 +143,10 @@ public class S3AFileSystem extends FileSystem {
 
   // The maximum number of entries that can be deleted in any call to s3
   private static final int MAX_ENTRIES_TO_DELETE = 1000;
+  private boolean blockUploadEnabled;
+  private String blockOutputBuffer;
+  private S3ADataBlocks.BlockFactory blockFactory;
+  private int blockOutputActiveBlocks;
 
   /** Called after a new FileSystem instance is constructed.
    * @param name a uri whose authority section names the host, port, etc.
@@ -157,18 +173,11 @@ public class S3AFileSystem extends FileSystem {
 
       maxKeys = intOption(conf, MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS, 1);
       listing = new Listing(this);
-      partSize = conf.getLong(MULTIPART_SIZE, DEFAULT_MULTIPART_SIZE);
-      if (partSize < 5 * 1024 * 1024) {
-        LOG.error(MULTIPART_SIZE + " must be at least 5 MB");
-        partSize = 5 * 1024 * 1024;
-      }
+      partSize = getMultipartSizeProperty(conf,
+          MULTIPART_SIZE, DEFAULT_MULTIPART_SIZE);
+      multiPartThreshold = getMultipartSizeProperty(conf,
+          MIN_MULTIPART_THRESHOLD, DEFAULT_MIN_MULTIPART_THRESHOLD);
 
-      multiPartThreshold = conf.getLong(MIN_MULTIPART_THRESHOLD,
-          DEFAULT_MIN_MULTIPART_THRESHOLD);
-      if (multiPartThreshold < 5 * 1024 * 1024) {
-        LOG.error(MIN_MULTIPART_THRESHOLD + " must be at least 5 MB");
-        multiPartThreshold = 5 * 1024 * 1024;
-      }
       //check but do not store the block size
       longOption(conf, FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE, 1);
       enableMultiObjectsDelete = conf.getBoolean(ENABLE_MULTI_DELETE, true);
@@ -189,14 +198,14 @@ public class S3AFileSystem extends FileSystem {
         LOG.warn(MAX_THREADS + " must be at least 2: forcing to 2.");
         maxThreads = 2;
       }
-      int totalTasks = conf.getInt(MAX_TOTAL_TASKS, DEFAULT_MAX_TOTAL_TASKS);
-      if (totalTasks < 1) {
-        LOG.warn(MAX_TOTAL_TASKS + "must be at least 1: forcing to 1.");
-        totalTasks = 1;
-      }
-      long keepAliveTime = conf.getLong(KEEPALIVE_TIME, DEFAULT_KEEPALIVE_TIME);
-      threadPoolExecutor = new BlockingThreadPoolExecutorService(maxThreads,
-          maxThreads + totalTasks, keepAliveTime, TimeUnit.SECONDS,
+      int totalTasks = intOption(conf,
+          MAX_TOTAL_TASKS, DEFAULT_MAX_TOTAL_TASKS, 1);
+      long keepAliveTime = longOption(conf, KEEPALIVE_TIME,
+          DEFAULT_KEEPALIVE_TIME, 0);
+      threadPoolExecutor = BlockingThreadPoolExecutorService.newInstance(
+          maxThreads,
+          maxThreads + totalTasks,
+          keepAliveTime, TimeUnit.SECONDS,
           "s3a-transfer-shared");
 
       initTransferManager();
@@ -209,8 +218,25 @@ public class S3AFileSystem extends FileSystem {
 
       serverSideEncryptionAlgorithm =
           conf.getTrimmed(SERVER_SIDE_ENCRYPTION_ALGORITHM);
+      LOG.debug("Using encryption {}", serverSideEncryptionAlgorithm);
       inputPolicy = S3AInputPolicy.getPolicy(
           conf.getTrimmed(INPUT_FADVISE, INPUT_FADV_NORMAL));
+
+      blockUploadEnabled = conf.getBoolean(FAST_UPLOAD, DEFAULT_FAST_UPLOAD);
+
+      if (blockUploadEnabled) {
+        blockOutputBuffer = conf.getTrimmed(FAST_UPLOAD_BUFFER,
+            DEFAULT_FAST_UPLOAD_BUFFER);
+        partSize = ensureOutputParameterInRange(MULTIPART_SIZE, partSize);
+        blockFactory = S3ADataBlocks.createFactory(this, blockOutputBuffer);
+        blockOutputActiveBlocks = intOption(conf,
+            FAST_UPLOAD_ACTIVE_BLOCKS, DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS, 1);
+        LOG.debug("Using S3ABlockOutputStream with buffer = {}; block={};" +
+                " queue limit={}",
+            blockOutputBuffer, partSize, blockOutputActiveBlocks);
+      } else {
+        LOG.debug("Using S3AOutputStream");
+      }
     } catch (AmazonClientException e) {
       throw translateException("initializing ", new Path(name), e);
     }
@@ -337,6 +363,33 @@ public class S3AFileSystem extends FileSystem {
   }
 
   /**
+   * Demand create the directory allocator, then create a temporary file.
+   * {@link LocalDirAllocator#createTmpFileForWrite(String, long, Configuration)}.
+   *  @param pathStr prefix for the temporary file
+   *  @param size the size of the file that is going to be written
+   *  @param conf the Configuration object
+   *  @return a unique temporary file
+   *  @throws IOException IO problems
+   */
+  synchronized File createTmpFileForWrite(String pathStr, long size,
+      Configuration conf) throws IOException {
+    if (directoryAllocator == null) {
+      String bufferDir = conf.get(BUFFER_DIR) != null
+          ? BUFFER_DIR : "hadoop.tmp.dir";
+      directoryAllocator = new LocalDirAllocator(bufferDir);
+    }
+    return directoryAllocator.createTmpFileForWrite(pathStr, size, conf);
+  }
+
+  /**
+   * Get the bucket of this filesystem.
+   * @return the bucket
+   */
+  public String getBucket() {
+    return bucket;
+  }
+
+  /**
    * Change the input policy for this FS.
    * @param inputPolicy new policy
    */
@@ -460,6 +513,7 @@ public class S3AFileSystem extends FileSystem {
    * @see #setPermission(Path, FsPermission)
    */
   @Override
+  @SuppressWarnings("IOResourceOpenedButNotSafelyClosed")
   public FSDataOutputStream create(Path f, FsPermission permission,
       boolean overwrite, int bufferSize, short replication, long blockSize,
       Progressable progress) throws IOException {
@@ -484,28 +538,33 @@ public class S3AFileSystem extends FileSystem {
 
     }
     instrumentation.fileCreated();
-    if (getConf().getBoolean(FAST_UPLOAD, DEFAULT_FAST_UPLOAD)) {
-      return new FSDataOutputStream(
-          new S3AFastOutputStream(s3,
-              this,
-              bucket,
+    FSDataOutputStream output;
+    if (blockUploadEnabled) {
+      output = new FSDataOutputStream(
+          new S3ABlockOutputStream(this,
               key,
+              new SemaphoredDelegatingExecutor(threadPoolExecutor,
+                  blockOutputActiveBlocks, true),
               progress,
-              cannedACL,
               partSize,
-              multiPartThreshold,
-              threadPoolExecutor),
-          statistics);
+              blockFactory,
+              instrumentation.newOutputStreamStatistics(),
+              new WriteOperationHelper(key)
+          ),
+          null);
+    } else {
+
+      // We pass null to FSDataOutputStream so it won't count writes that
+      // are being buffered to a file
+      output = new FSDataOutputStream(
+          new S3AOutputStream(getConf(),
+              this,
+              key,
+              progress
+          ),
+          null);
     }
-    // We pass null to FSDataOutputStream so it won't count writes that
-    // are being buffered to a file
-    return new FSDataOutputStream(
-        new S3AOutputStream(getConf(),
-            this,
-            key,
-            progress
-        ),
-        null);
+    return output;
   }
 
   /**
@@ -750,6 +809,33 @@ public class S3AFileSystem extends FileSystem {
   }
 
   /**
+   * Decrement a gauge by a specific value.
+   * @param statistic The operation to decrement
+   * @param count the count to decrement
+   */
+  protected void decrementGauge(Statistic statistic, long count) {
+    instrumentation.decrementGauge(statistic, count);
+  }
+
+  /**
+   * Increment a gauge by a specific value.
+   * @param statistic The operation to increment
+   * @param count the count to increment
+   */
+  protected void incrementGauge(Statistic statistic, long count) {
+    instrumentation.incrementGauge(statistic, count);
+  }
+
+  /**
+   * Get the storage statistics of this filesystem.
+   * @return the storage statistics
+   */
+  @Override
+  public S3AStorageStatistics getStorageStatistics() {
+    return storageStatistics;
+  }
+
+  /**
    * Request object metadata; increments counters in the process.
    * @param key key
    * @return the metadata
@@ -896,7 +982,9 @@ public class S3AFileSystem extends FileSystem {
    */
   public ObjectMetadata newObjectMetadata(long length) {
     final ObjectMetadata om = newObjectMetadata();
-    om.setContentLength(length);
+    if (length >= 0) {
+      om.setContentLength(length);
+    }
     return om;
   }
 
@@ -918,7 +1006,41 @@ public class S3AFileSystem extends FileSystem {
       len = putObjectRequest.getMetadata().getContentLength();
     }
     incrementPutStartStatistics(len);
-    return transfers.upload(putObjectRequest);
+    try {
+      Upload upload = transfers.upload(putObjectRequest);
+      incrementPutCompletedStatistics(true, len);
+      return upload;
+    } catch (AmazonClientException e) {
+      incrementPutCompletedStatistics(false, len);
+      throw e;
+    }
+  }
+
+  /**
+   * PUT an object directly (i.e. not via the transfer manager).
+   * Byte length is calculated from the file length, or, if there is no
+   * file, from the content length of the header.
+   * @param putObjectRequest the request
+   * @return the upload initiated
+   * @throws AmazonClientException on problems
+   */
+  public PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest)
+      throws AmazonClientException {
+    long len;
+    if (putObjectRequest.getFile() != null) {
+      len = putObjectRequest.getFile().length();
+    } else {
+      len = putObjectRequest.getMetadata().getContentLength();
+    }
+    incrementPutStartStatistics(len);
+    try {
+      PutObjectResult result = s3.putObject(putObjectRequest);
+      incrementPutCompletedStatistics(true, len);
+      return result;
+    } catch (AmazonClientException e) {
+      incrementPutCompletedStatistics(false, len);
+      throw e;
+    }
   }
 
   /**
@@ -926,10 +1048,20 @@ public class S3AFileSystem extends FileSystem {
    * Increments the write and put counters
    * @param request request
    * @return the result of the operation.
+   * @throws AmazonClientException on problems
    */
-  public UploadPartResult uploadPart(UploadPartRequest request) {
-    incrementPutStartStatistics(request.getPartSize());
-    return s3.uploadPart(request);
+  public UploadPartResult uploadPart(UploadPartRequest request)
+      throws AmazonClientException {
+    long len = request.getPartSize();
+    incrementPutStartStatistics(len);
+    try {
+      UploadPartResult uploadPartResult = s3.uploadPart(request);
+      incrementPutCompletedStatistics(true, len);
+      return uploadPartResult;
+    } catch (AmazonClientException e) {
+      incrementPutCompletedStatistics(false, len);
+      throw e;
+    }
   }
 
   /**
@@ -942,9 +1074,28 @@ public class S3AFileSystem extends FileSystem {
     LOG.debug("PUT start {} bytes", bytes);
     incrementWriteOperations();
     incrementStatistic(OBJECT_PUT_REQUESTS);
+    incrementGauge(OBJECT_PUT_REQUESTS_ACTIVE, 1);
+    if (bytes > 0) {
+      incrementGauge(OBJECT_PUT_BYTES_PENDING, bytes);
+    }
+  }
+
+  /**
+   * At the end of a put/multipart upload operation, update the
+   * relevant counters and gauges.
+   *
+   * @param success did the operation succeed?
+   * @param bytes bytes in the request.
+   */
+  public void incrementPutCompletedStatistics(boolean success, long bytes) {
+    LOG.debug("PUT completed success={}; {} bytes", success, bytes);
+    incrementWriteOperations();
     if (bytes > 0) {
       incrementStatistic(OBJECT_PUT_BYTES, bytes);
+      decrementGauge(OBJECT_PUT_BYTES_PENDING, bytes);
     }
+    incrementStatistic(OBJECT_PUT_REQUESTS_COMPLETED);
+    decrementGauge(OBJECT_PUT_REQUESTS_ACTIVE, 1);
   }
 
   /**
@@ -955,7 +1106,7 @@ public class S3AFileSystem extends FileSystem {
    * @param bytes bytes successfully uploaded.
    */
   public void incrementPutProgressStatistics(String key, long bytes) {
-    LOG.debug("PUT {}: {} bytes", key, bytes);
+    PROGRESS.debug("PUT {}: {} bytes", key, bytes);
     incrementWriteOperations();
     if (bytes > 0) {
       statistics.incrementBytesWritten(bytes);
@@ -1475,7 +1626,7 @@ public class S3AFileSystem extends FileSystem {
     LocalFileSystem local = getLocal(getConf());
     File srcfile = local.pathToFile(src);
 
-    final ObjectMetadata om = newObjectMetadata();
+    final ObjectMetadata om = newObjectMetadata(srcfile.length());
     PutObjectRequest putObjectRequest = newPutObjectRequest(key, om, srcfile);
     Upload up = putObject(putObjectRequest);
     ProgressableProgressListener listener = new ProgressableProgressListener(
@@ -1743,6 +1894,10 @@ public class S3AFileSystem extends FileSystem {
           .append(serverSideEncryptionAlgorithm)
           .append('\'');
     }
+    if (blockFactory != null) {
+      sb.append(", blockFactory=").append(blockFactory);
+    }
+    sb.append(", executor=").append(threadPoolExecutor);
     sb.append(", statistics {")
         .append(statistics)
         .append("}");
@@ -1950,4 +2105,163 @@ public class S3AFileSystem extends FileSystem {
           getFileBlockLocations(status, 0, status.getLen())
           : null);
   }
+
+  /**
+   * Helper for an ongoing write operation.
+   * <p>
+   * It hides direct access to the S3 API from the output stream,
+   * and is a location where the object upload process can be evolved/enhanced.
+   * <p>
+   * Features
+   * <ul>
+   *   <li>Methods to create and submit requests to S3, so avoiding
+   *   all direct interaction with the AWS APIs.</li>
+   *   <li>Some extra preflight checks of arguments, so failing fast on
+   *   errors.</li>
+   *   <li>Callbacks to let the FS know of events in the output stream
+   *   upload process.</li>
+   * </ul>
+   *
+   * Each instance of this state is unique to a single output stream.
+   */
+  final class WriteOperationHelper {
+    private final String key;
+
+    private WriteOperationHelper(String key) {
+      this.key = key;
+    }
+
+    /**
+     * Create a {@link PutObjectRequest} request.
+     * The metadata is assumed to have been configured with the size of the
+     * operation.
+     * @param inputStream source data.
+     * @param length size, if known. Use -1 for not known
+     * @return the request
+     */
+    PutObjectRequest newPutRequest(InputStream inputStream, long length) {
+      return newPutObjectRequest(key, newObjectMetadata(length), inputStream);
+    }
+
+    /**
+     * Callback on a successful write.
+     */
+    void writeSuccessful() {
+      finishedWrite(key);
+    }
+
+    /**
+     * Callback on a write failure.
+     * @param e Any exception raised which triggered the failure.
+     */
+    void writeFailed(Exception e) {
+      LOG.debug("Write to {} failed", this, e);
+    }
+
+    /**
+     * Create a new object metadata instance.
+     * Any standard metadata headers are added here, for example:
+     * encryption.
+     * @param length size, if known. Use -1 for not known
+     * @return a new metadata instance
+     */
+    public ObjectMetadata newObjectMetadata(long length) {
+      return S3AFileSystem.this.newObjectMetadata(length);
+    }
+
+    /**
+     * Start the multipart upload process.
+     * @return the upload result containing the ID
+     * @throws IOException IO problem
+     */
+    String initiateMultiPartUpload() throws IOException {
+      LOG.debug("Initiating Multipart upload");
+      final InitiateMultipartUploadRequest initiateMPURequest =
+          new InitiateMultipartUploadRequest(bucket,
+              key,
+              newObjectMetadata(-1));
+      initiateMPURequest.setCannedACL(cannedACL);
+      try {
+        return s3.initiateMultipartUpload(initiateMPURequest)
+            .getUploadId();
+      } catch (AmazonClientException ace) {
+        throw translateException("initiate MultiPartUpload", key, ace);
+      }
+    }
+
+    /**
+     * Complete a multipart upload operation.
+     * @param uploadId multipart operation Id
+     * @param partETags list of partial uploads
+     * @return the result
+     * @throws AmazonClientException on problems.
+     */
+    CompleteMultipartUploadResult completeMultipartUpload(String uploadId,
+        List<PartETag> partETags) throws AmazonClientException {
+      Preconditions.checkNotNull(uploadId);
+      Preconditions.checkNotNull(partETags);
+      Preconditions.checkArgument(!partETags.isEmpty(),
+          "No partitions have been uploaded");
+      return s3.completeMultipartUpload(
+          new CompleteMultipartUploadRequest(bucket,
+              key,
+              uploadId,
+              partETags));
+    }
+
+    /**
+     * Abort a multipart upload operation.
+     * @param uploadId multipart operation Id
+     * @return the result
+     * @throws AmazonClientException on problems.
+     */
+    void abortMultipartUpload(String uploadId) throws AmazonClientException {
+      s3.abortMultipartUpload(
+          new AbortMultipartUploadRequest(bucket, key, uploadId));
+    }
+
+    /**
+     * Create and initialize a part request of a multipart upload.
+     * @param uploadId ID of ongoing upload
+     * @param uploadStream source of data to upload
+     * @param partNumber current part number of the upload
+     * @param size amount of data
+     * @return the request.
+     */
+    UploadPartRequest newUploadPartRequest(String uploadId,
+        InputStream uploadStream,
+        int partNumber,
+        int size) {
+      Preconditions.checkNotNull(uploadId);
+      Preconditions.checkNotNull(uploadStream);
+      Preconditions.checkArgument(size > 0, "Invalid partition size %s", size);
+      Preconditions.checkArgument(partNumber> 0 && partNumber <=10000,
+          "partNumber must be between 1 and 10000 inclusive, but is %s",
+          partNumber);
+
+      LOG.debug("Creating part upload request for {} #{} size {}",
+          uploadId, partNumber, size);
+      return new UploadPartRequest()
+          .withBucketName(bucket)
+          .withKey(key)
+          .withUploadId(uploadId)
+          .withInputStream(uploadStream)
+          .withPartNumber(partNumber)
+          .withPartSize(size);
+    }
+
+    /**
+     * The toString method is intended to be used in logging/toString calls.
+     * @return a string description.
+     */
+    @Override
+    public String toString() {
+      final StringBuilder sb = new StringBuilder(
+          "{bucket=").append(bucket);
+      sb.append(", key='").append(key).append('\'');
+      sb.append('}');
+      return sb.toString();
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
index 26b5b51..963c53f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
@@ -18,7 +18,9 @@
 
 package org.apache.hadoop.fs.s3a;
 
-import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.metrics2.MetricStringBuilder;
@@ -29,10 +31,12 @@ import org.apache.hadoop.metrics2.lib.MutableCounterLong;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 import org.apache.hadoop.metrics2.lib.MutableMetric;
 
+import java.io.Closeable;
 import java.net.URI;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
+import java.util.concurrent.atomic.AtomicLong;
 
 import static org.apache.hadoop.fs.s3a.Statistic.*;
 
@@ -50,6 +54,9 @@ import static org.apache.hadoop.fs.s3a.Statistic.*;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class S3AInstrumentation {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      S3AInstrumentation.class);
+
   public static final String CONTEXT = "S3AFileSystem";
   private final MetricsRegistry registry =
       new MetricsRegistry("S3AFileSystem").setContext(CONTEXT);
@@ -100,7 +107,23 @@ public class S3AInstrumentation {
       OBJECT_METADATA_REQUESTS,
       OBJECT_MULTIPART_UPLOAD_ABORTED,
       OBJECT_PUT_BYTES,
-      OBJECT_PUT_REQUESTS
+      OBJECT_PUT_REQUESTS,
+      OBJECT_PUT_REQUESTS_COMPLETED,
+      STREAM_WRITE_FAILURES,
+      STREAM_WRITE_BLOCK_UPLOADS,
+      STREAM_WRITE_BLOCK_UPLOADS_COMMITTED,
+      STREAM_WRITE_BLOCK_UPLOADS_ABORTED,
+      STREAM_WRITE_TOTAL_TIME,
+      STREAM_WRITE_TOTAL_DATA,
+  };
+
+
+  private static final Statistic[] GAUGES_TO_CREATE = {
+      OBJECT_PUT_REQUESTS_ACTIVE,
+      OBJECT_PUT_BYTES_PENDING,
+      STREAM_WRITE_BLOCK_UPLOADS_ACTIVE,
+      STREAM_WRITE_BLOCK_UPLOADS_PENDING,
+      STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING,
   };
 
   public S3AInstrumentation(URI name) {
@@ -143,6 +166,9 @@ public class S3AInstrumentation {
     for (Statistic statistic : COUNTERS_TO_CREATE) {
       counter(statistic);
     }
+    for (Statistic statistic : GAUGES_TO_CREATE) {
+      gauge(statistic.getSymbol(), statistic.getDescription());
+    }
   }
 
   /**
@@ -254,13 +280,13 @@ public class S3AInstrumentation {
    * Lookup a counter by name. Return null if it is not known.
    * @param name counter name
    * @return the counter
+   * @throws IllegalStateException if the metric is not a counter
    */
   private MutableCounterLong lookupCounter(String name) {
     MutableMetric metric = lookupMetric(name);
     if (metric == null) {
       return null;
     }
-    Preconditions.checkNotNull(metric, "not found: " + name);
     if (!(metric instanceof MutableCounterLong)) {
       throw new IllegalStateException("Metric " + name
           + " is not a MutableCounterLong: " + metric);
@@ -269,6 +295,20 @@ public class S3AInstrumentation {
   }
 
   /**
+   * Look up a gauge.
+   * @param name gauge name
+   * @return the gauge or null
+   * @throws ClassCastException if the metric is not a Gauge.
+   */
+  public MutableGaugeLong lookupGauge(String name) {
+    MutableMetric metric = lookupMetric(name);
+    if (metric == null) {
+      LOG.debug("No gauge {}", name);
+    }
+    return (MutableGaugeLong) metric;
+  }
+
+  /**
    * Look up a metric from both the registered set and the lighter weight
    * stream entries.
    * @param name metric name
@@ -349,6 +389,47 @@ public class S3AInstrumentation {
       counter.incr(count);
     }
   }
+  /**
+   * Increment a specific counter.
+   * No-op if not defined.
+   * @param op operation
+   * @param count atomic long containing value
+   */
+  public void incrementCounter(Statistic op, AtomicLong count) {
+    incrementCounter(op, count.get());
+  }
+
+  /**
+   * Increment a specific gauge.
+   * No-op if not defined.
+   * @param op operation
+   * @param count increment value
+   * @throws ClassCastException if the metric is of the wrong type
+   */
+  public void incrementGauge(Statistic op, long count) {
+    MutableGaugeLong gauge = lookupGauge(op.getSymbol());
+    if (gauge != null) {
+      gauge.incr(count);
+    } else {
+      LOG.debug("No Gauge: "+ op);
+    }
+  }
+
+  /**
+   * Decrement a specific gauge.
+   * No-op if not defined.
+   * @param op operation
+   * @param count increment value
+   * @throws ClassCastException if the metric is of the wrong type
+   */
+  public void decrementGauge(Statistic op, long count) {
+    MutableGaugeLong gauge = lookupGauge(op.getSymbol());
+    if (gauge != null) {
+      gauge.decr(count);
+    } else {
+      LOG.debug("No Gauge: " + op);
+    }
+  }
 
   /**
    * Create a stream input statistics instance.
@@ -553,4 +634,165 @@ public class S3AInstrumentation {
       return sb.toString();
     }
   }
+
+  /**
+   * Create a stream output statistics instance.
+   * @return the new instance
+   */
+
+  OutputStreamStatistics newOutputStreamStatistics() {
+    return new OutputStreamStatistics();
+  }
+
+  /**
+   * Merge in the statistics of a single output stream into
+   * the filesystem-wide statistics.
+   * @param statistics stream statistics
+   */
+  private void mergeOutputStreamStatistics(OutputStreamStatistics statistics) {
+    incrementCounter(STREAM_WRITE_TOTAL_TIME, statistics.totalUploadDuration());
+    incrementCounter(STREAM_WRITE_QUEUE_DURATION, statistics.queueDuration);
+    incrementCounter(STREAM_WRITE_TOTAL_DATA, statistics.bytesUploaded);
+    incrementCounter(STREAM_WRITE_BLOCK_UPLOADS,
+        statistics.blockUploadsCompleted);
+  }
+
+  /**
+   * Statistics updated by an output stream during its actual operation.
+   * Some of these stats may be relayed. However, as block upload is
+   * spans multiple
+   */
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  public final class OutputStreamStatistics implements Closeable {
+    private final AtomicLong blocksSubmitted = new AtomicLong(0);
+    private final AtomicLong blocksInQueue = new AtomicLong(0);
+    private final AtomicLong blocksActive = new AtomicLong(0);
+    private final AtomicLong blockUploadsCompleted = new AtomicLong(0);
+    private final AtomicLong blockUploadsFailed = new AtomicLong(0);
+    private final AtomicLong bytesPendingUpload = new AtomicLong(0);
+
+    private final AtomicLong bytesUploaded = new AtomicLong(0);
+    private final AtomicLong transferDuration = new AtomicLong(0);
+    private final AtomicLong queueDuration = new AtomicLong(0);
+    private final AtomicLong exceptionsInMultipartFinalize = new AtomicLong(0);
+
+    /**
+     * Block is queued for upload.
+     */
+    void blockUploadQueued(int blockSize) {
+      blocksSubmitted.incrementAndGet();
+      blocksInQueue.incrementAndGet();
+      bytesPendingUpload.addAndGet(blockSize);
+      incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_PENDING, 1);
+      incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, blockSize);
+    }
+
+    /** Queued block has been scheduled for upload. */
+    void blockUploadStarted(long duration, int blockSize) {
+      queueDuration.addAndGet(duration);
+      blocksInQueue.decrementAndGet();
+      blocksActive.incrementAndGet();
+      incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_PENDING, -1);
+      incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, 1);
+    }
+
+    /** A block upload has completed. */
+    void blockUploadCompleted(long duration, int blockSize) {
+      this.transferDuration.addAndGet(duration);
+      incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, -1);
+      blocksActive.decrementAndGet();
+      blockUploadsCompleted.incrementAndGet();
+    }
+
+    /**
+     *  A block upload has failed.
+     *  A final transfer completed event is still expected, so this
+     *  does not decrement the active block counter.
+     */
+    void blockUploadFailed(long duration, int blockSize) {
+      blockUploadsFailed.incrementAndGet();
+    }
+
+    /** Intermediate report of bytes uploaded. */
+    void bytesTransferred(long byteCount) {
+      bytesUploaded.addAndGet(byteCount);
+      bytesPendingUpload.addAndGet(-byteCount);
+      incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, -byteCount);
+    }
+
+    /**
+     * Note an exception in a multipart complete.
+     */
+    void exceptionInMultipartComplete() {
+      exceptionsInMultipartFinalize.incrementAndGet();
+    }
+
+    /**
+     * Note an exception in a multipart abort.
+     */
+    void exceptionInMultipartAbort() {
+      exceptionsInMultipartFinalize.incrementAndGet();
+    }
+
+    /**
+     * Get the number of bytes pending upload.
+     * @return the number of bytes in the pending upload state.
+     */
+    public long getBytesPendingUpload() {
+      return bytesPendingUpload.get();
+    }
+
+    /**
+     * Output stream has closed.
+     * Trigger merge in of all statistics not updated during operation.
+     */
+    @Override
+    public void close() {
+      if (bytesPendingUpload.get() > 0) {
+        LOG.warn("Closing output stream statistics while data is still marked" +
+            " as pending upload in {}", this);
+      }
+      mergeOutputStreamStatistics(this);
+    }
+
+    long averageQueueTime() {
+      return blocksSubmitted.get() > 0 ?
+          (queueDuration.get() / blocksSubmitted.get()) : 0;
+    }
+
+    double effectiveBandwidth() {
+      double duration = totalUploadDuration() / 1000.0;
+      return duration > 0 ?
+          (bytesUploaded.get() / duration) : 0;
+    }
+
+    long totalUploadDuration() {
+      return queueDuration.get() + transferDuration.get();
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder sb = new StringBuilder(
+          "OutputStreamStatistics{");
+      sb.append("blocksSubmitted=").append(blocksSubmitted);
+      sb.append(", blocksInQueue=").append(blocksInQueue);
+      sb.append(", blocksActive=").append(blocksActive);
+      sb.append(", blockUploadsCompleted=").append(blockUploadsCompleted);
+      sb.append(", blockUploadsFailed=").append(blockUploadsFailed);
+      sb.append(", bytesPendingUpload=").append(bytesPendingUpload);
+      sb.append(", bytesUploaded=").append(bytesUploaded);
+      sb.append(", exceptionsInMultipartFinalize=").append(
+          exceptionsInMultipartFinalize);
+      sb.append(", transferDuration=").append(transferDuration).append(" ms");
+      sb.append(", queueDuration=").append(queueDuration).append(" ms");
+      sb.append(", averageQueueTime=").append(averageQueueTime()).append(" ms");
+      sb.append(", totalUploadDuration=").append(totalUploadDuration())
+          .append(" ms");
+      sb.append(", effectiveBandwidth=").append(effectiveBandwidth())
+          .append(" bytes/s");
+      sb.append('}');
+      return sb.toString();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
index 23ba682..6ebc9e4 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -35,8 +35,8 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.io.OutputStream;
+import java.util.concurrent.atomic.AtomicBoolean;
 
-import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 
 /**
@@ -45,37 +45,27 @@ import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class S3AOutputStream extends OutputStream {
-  private OutputStream backupStream;
-  private File backupFile;
-  private boolean closed;
-  private String key;
-  private Progressable progress;
-  private long partSize;
-  private long partSizeThreshold;
-  private S3AFileSystem fs;
-  private LocalDirAllocator lDirAlloc;
+  private final OutputStream backupStream;
+  private final File backupFile;
+  private final AtomicBoolean closed = new AtomicBoolean(false);
+  private final String key;
+  private final Progressable progress;
+  private final S3AFileSystem fs;
 
   public static final Logger LOG = S3AFileSystem.LOG;
 
   public S3AOutputStream(Configuration conf,
-      S3AFileSystem fs, String key, Progressable progress)
+      S3AFileSystem fs,
+      String key,
+      Progressable progress)
       throws IOException {
     this.key = key;
     this.progress = progress;
     this.fs = fs;
 
-    partSize = fs.getPartitionSize();
-    partSizeThreshold = fs.getMultiPartThreshold();
-
-    if (conf.get(BUFFER_DIR, null) != null) {
-      lDirAlloc = new LocalDirAllocator(BUFFER_DIR);
-    } else {
-      lDirAlloc = new LocalDirAllocator("${hadoop.tmp.dir}/s3a");
-    }
 
-    backupFile = lDirAlloc.createTmpFileForWrite("output-",
+    backupFile = fs.createTmpFileForWrite("output-",
         LocalDirAllocator.SIZE_UNKNOWN, conf);
-    closed = false;
 
     LOG.debug("OutputStream for key '{}' writing to tempfile: {}",
         key, backupFile);
@@ -84,25 +74,33 @@ public class S3AOutputStream extends OutputStream {
         new FileOutputStream(backupFile));
   }
 
+  /**
+   * Check for the filesystem being open.
+   * @throws IOException if the filesystem is closed.
+   */
+  void checkOpen() throws IOException {
+    if (closed.get()) {
+      throw new IOException("Output Stream closed");
+    }
+  }
+
   @Override
   public void flush() throws IOException {
+    checkOpen();
     backupStream.flush();
   }
 
   @Override
-  public synchronized void close() throws IOException {
-    if (closed) {
+  public void close() throws IOException {
+    if (closed.getAndSet(true)) {
       return;
     }
 
     backupStream.close();
     LOG.debug("OutputStream for key '{}' closed. Now beginning upload", key);
-    LOG.debug("Minimum upload part size: {} threshold {}" , partSize,
-        partSizeThreshold);
-
 
     try {
-      final ObjectMetadata om = fs.newObjectMetadata();
+      final ObjectMetadata om = fs.newObjectMetadata(backupFile.length());
       Upload upload = fs.putObject(
           fs.newPutObjectRequest(
               key,
@@ -126,18 +124,19 @@ public class S3AOutputStream extends OutputStream {
         LOG.warn("Could not delete temporary s3a file: {}", backupFile);
       }
       super.close();
-      closed = true;
     }
     LOG.debug("OutputStream for key '{}' upload complete", key);
   }
 
   @Override
   public void write(int b) throws IOException {
+    checkOpen();
     backupStream.write(b);
   }
 
   @Override
   public void write(byte[] b, int off, int len) throws IOException {
+    checkOpen();
     backupStream.write(b, off, len);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
index 93d819b..c89f690 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
@@ -49,6 +49,7 @@ import java.util.concurrent.ExecutionException;
 import static org.apache.hadoop.fs.s3a.Constants.ACCESS_KEY;
 import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER;
 import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT;
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_MIN_SIZE;
 import static org.apache.hadoop.fs.s3a.Constants.SECRET_KEY;
 
 /**
@@ -460,4 +461,42 @@ public final class S3AUtils {
             key, v, min));
     return v;
   }
+
+  /**
+   * Get a size property from the configuration: this property must
+   * be at least equal to {@link Constants#MULTIPART_MIN_SIZE}.
+   * If it is too small, it is rounded up to that minimum, and a warning
+   * printed.
+   * @param conf configuration
+   * @param property property name
+   * @param defVal default value
+   * @return the value, guaranteed to be above the minimum size
+   */
+  public static long getMultipartSizeProperty(Configuration conf,
+      String property, long defVal) {
+    long partSize = conf.getLong(property, defVal);
+    if (partSize < MULTIPART_MIN_SIZE) {
+      LOG.warn("{} must be at least 5 MB; configured value is {}",
+          property, partSize);
+      partSize = MULTIPART_MIN_SIZE;
+    }
+    return partSize;
+  }
+
+  /**
+   * Ensure that the long value is in the range of an integer.
+   * @param name property name for error messages
+   * @param size original size
+   * @return the size, guaranteed to be less than or equal to the max
+   * value of an integer.
+   */
+  public static int ensureOutputParameterInRange(String name, long size) {
+    if (size > Integer.MAX_VALUE) {
+      LOG.warn("s3a: {} capped to ~2.14GB" +
+          " (maximum allowed size with current output mechanism)", name);
+      return Integer.MAX_VALUE;
+    } else {
+      return (int)size;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SemaphoredDelegatingExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SemaphoredDelegatingExecutor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SemaphoredDelegatingExecutor.java
new file mode 100644
index 0000000..6b21912
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SemaphoredDelegatingExecutor.java
@@ -0,0 +1,230 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import com.google.common.util.concurrent.ForwardingListeningExecutorService;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * This ExecutorService blocks the submission of new tasks when its queue is
+ * already full by using a semaphore. Task submissions require permits, task
+ * completions release permits.
+ * <p>
+ * This is a refactoring of {@link BlockingThreadPoolExecutorService}; that code
+ * contains the thread pool logic, whereas this isolates the semaphore
+ * and submit logic for use with other thread pools and delegation models.
+ * In particular, it <i>permits multiple per stream executors to share a
+ * single per-FS-instance executor; the latter to throttle overall
+ * load from the the FS, the others to limit the amount of load which
+ * a single output stream can generate.</i>
+ * <p>
+ * This is inspired by <a href="https://github.com/apache/incubator-s4/blob/master/subprojects/s4-comm/src/main/java/org/apache/s4/comm/staging/BlockingThreadPoolExecutorService.java">
+ * this s4 threadpool</a>
+ */
+@SuppressWarnings("NullableProblems")
+@InterfaceAudience.Private
+class SemaphoredDelegatingExecutor extends
+    ForwardingListeningExecutorService {
+
+  private final Semaphore queueingPermits;
+  private final ListeningExecutorService executorDelegatee;
+  private final int permitCount;
+
+  /**
+   * Instantiate.
+   * @param executorDelegatee Executor to delegate to
+   * @param permitCount number of permits into the queue permitted
+   * @param fair should the semaphore be "fair"
+   */
+  SemaphoredDelegatingExecutor(ListeningExecutorService executorDelegatee,
+      int permitCount,
+      boolean fair) {
+    this.permitCount = permitCount;
+    queueingPermits = new Semaphore(permitCount, fair);
+    this.executorDelegatee = executorDelegatee;
+  }
+
+  @Override
+  protected ListeningExecutorService delegate() {
+    return executorDelegatee;
+  }
+
+
+  @Override
+  public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
+      throws InterruptedException {
+    throw new RuntimeException("Not implemented");
+  }
+
+  @Override
+  public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks,
+      long timeout, TimeUnit unit) throws InterruptedException {
+    throw new RuntimeException("Not implemented");
+  }
+
+  @Override
+  public <T> T invokeAny(Collection<? extends Callable<T>> tasks)
+      throws InterruptedException, ExecutionException {
+    throw new RuntimeException("Not implemented");
+  }
+
+  @Override
+  public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout,
+      TimeUnit unit)
+      throws InterruptedException, ExecutionException, TimeoutException {
+    throw new RuntimeException("Not implemented");
+  }
+
+  @Override
+  public <T> ListenableFuture<T> submit(Callable<T> task) {
+    try {
+      queueingPermits.acquire();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      return Futures.immediateFailedCheckedFuture(e);
+    }
+    return super.submit(new CallableWithPermitRelease<>(task));
+  }
+
+  @Override
+  public <T> ListenableFuture<T> submit(Runnable task, T result) {
+    try {
+      queueingPermits.acquire();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      return Futures.immediateFailedCheckedFuture(e);
+    }
+    return super.submit(new RunnableWithPermitRelease(task), result);
+  }
+
+  @Override
+  public ListenableFuture<?> submit(Runnable task) {
+    try {
+      queueingPermits.acquire();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      return Futures.immediateFailedCheckedFuture(e);
+    }
+    return super.submit(new RunnableWithPermitRelease(task));
+  }
+
+  @Override
+  public void execute(Runnable command) {
+    try {
+      queueingPermits.acquire();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    }
+    super.execute(new RunnableWithPermitRelease(command));
+  }
+
+  /**
+   * Get the number of permits available; guaranteed to be
+   * {@code 0 <= availablePermits <= size}.
+   * @return the number of permits available at the time of invocation.
+   */
+  public int getAvailablePermits() {
+    return queueingPermits.availablePermits();
+  }
+
+  /**
+   * Get the number of threads waiting to acquire a permit.
+   * @return snapshot of the length of the queue of blocked threads.
+   */
+  public int getWaitingCount() {
+    return queueingPermits.getQueueLength();
+  }
+
+  /**
+   * Total number of permits.
+   * @return the number of permits as set in the constructor
+   */
+  public int getPermitCount() {
+    return permitCount;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "SemaphoredDelegatingExecutor{");
+    sb.append("permitCount=").append(getPermitCount());
+    sb.append(", available=").append(getAvailablePermits());
+    sb.append(", waiting=").append(getWaitingCount());
+    sb.append('}');
+    return sb.toString();
+  }
+
+  /**
+   * Releases a permit after the task is executed.
+   */
+  class RunnableWithPermitRelease implements Runnable {
+
+    private Runnable delegatee;
+
+    public RunnableWithPermitRelease(Runnable delegatee) {
+      this.delegatee = delegatee;
+    }
+
+    @Override
+    public void run() {
+      try {
+        delegatee.run();
+      } finally {
+        queueingPermits.release();
+      }
+
+    }
+  }
+
+  /**
+   * Releases a permit after the task is completed.
+   */
+  class CallableWithPermitRelease<T> implements Callable<T> {
+
+    private Callable<T> delegatee;
+
+    public CallableWithPermitRelease(Callable<T> delegatee) {
+      this.delegatee = delegatee;
+    }
+
+    @Override
+    public T call() throws Exception {
+      try {
+        return delegatee.call();
+      } finally {
+        queueingPermits.release();
+      }
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c348c56/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
index d84a355..36ec50b 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
@@ -81,10 +81,16 @@ public enum Statistic {
       "Object multipart upload aborted"),
   OBJECT_PUT_REQUESTS("object_put_requests",
       "Object put/multipart upload count"),
+  OBJECT_PUT_REQUESTS_COMPLETED("object_put_requests_completed",
+      "Object put/multipart upload completed count"),
+  OBJECT_PUT_REQUESTS_ACTIVE("object_put_requests_active",
+      "Current number of active put requests"),
   OBJECT_PUT_BYTES("object_put_bytes", "number of bytes uploaded"),
+  OBJECT_PUT_BYTES_PENDING("object_put_bytes_pending",
+      "number of bytes queued for upload/being actively uploaded"),
   STREAM_ABORTED("stream_aborted",
       "Count of times the TCP stream was aborted"),
-  STREAM_BACKWARD_SEEK_OPERATIONS("stream_backward_seek_pperations",
+  STREAM_BACKWARD_SEEK_OPERATIONS("stream_backward_seek_operations",
       "Number of executed seek operations which went backwards in a stream"),
   STREAM_CLOSED("streamClosed", "Count of times the TCP stream was closed"),
   STREAM_CLOSE_OPERATIONS("stream_close_operations",
@@ -112,7 +118,29 @@ public enum Statistic {
   STREAM_CLOSE_BYTES_READ("stream_bytes_read_in_close",
       "Count of bytes read when closing streams during seek operations."),
   STREAM_ABORT_BYTES_DISCARDED("stream_bytes_discarded_in_abort",
-      "Count of bytes discarded by aborting the stream");
+      "Count of bytes discarded by aborting the stream"),
+  STREAM_WRITE_FAILURES("stream_write_failures",
+      "Count of stream write failures reported"),
+  STREAM_WRITE_BLOCK_UPLOADS("stream_write_block_uploads",
+      "Count of block/partition uploads completed"),
+  STREAM_WRITE_BLOCK_UPLOADS_ACTIVE("stream_write_block_uploads_active",
+      "Count of block/partition uploads completed"),
+  STREAM_WRITE_BLOCK_UPLOADS_COMMITTED("stream_write_block_uploads_committed",
+      "Count of number of block uploads committed"),
+  STREAM_WRITE_BLOCK_UPLOADS_ABORTED("stream_write_block_uploads_aborted",
+      "Count of number of block uploads aborted"),
+
+  STREAM_WRITE_BLOCK_UPLOADS_PENDING("stream_write_block_uploads_pending",
+      "Gauge of block/partitions uploads queued to be written"),
+  STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING(
+      "stream_write_block_uploads_data_pending",
+      "Gauge of block/partitions data uploads queued to be written"),
+  STREAM_WRITE_TOTAL_TIME("stream_write_total_time",
+      "Count of total time taken for uploads to complete"),
+  STREAM_WRITE_TOTAL_DATA("stream_write_total_data",
+      "Count of total data uploaded in block output"),
+  STREAM_WRITE_QUEUE_DURATION("stream_write_queue_duration",
+      "Total queue duration of all block uploads");
 
   private static final Map<String, Statistic> SYMBOL_MAP =
       new HashMap<>(Statistic.values().length);


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


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

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


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

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

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


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

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


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


[40/50] [abbrv] hadoop git commit: HADOOP-13061. Refactor erasure coders. Contributed by Kai Sasaki

Posted by sj...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
index afaaf24..6e679c3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
@@ -21,7 +21,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
@@ -32,15 +31,11 @@ import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
  * It implements {@link ErasureCoder}.
  */
 @InterfaceAudience.Private
-public class RSErasureDecoder extends AbstractErasureDecoder {
+public class RSErasureDecoder extends ErasureDecoder {
   private RawErasureDecoder rsRawDecoder;
 
-  public RSErasureDecoder(int numDataUnits, int numParityUnits) {
-    super(numDataUnits, numParityUnits);
-  }
-
-  public RSErasureDecoder(ECSchema schema) {
-    super(schema);
+  public RSErasureDecoder(ErasureCoderOptions options) {
+    super(options);
   }
 
   @Override
@@ -56,11 +51,8 @@ public class RSErasureDecoder extends AbstractErasureDecoder {
 
   private RawErasureDecoder checkCreateRSRawDecoder() {
     if (rsRawDecoder == null) {
-      // TODO: we should create the raw coder according to codec.
-      ErasureCoderOptions coderOptions = new ErasureCoderOptions(
-          getNumDataUnits(), getNumParityUnits());
       rsRawDecoder = CodecUtil.createRawDecoder(getConf(),
-          ErasureCodeConstants.RS_DEFAULT_CODEC_NAME, coderOptions);
+          ErasureCodeConstants.RS_DEFAULT_CODEC_NAME, getOptions());
     }
     return rsRawDecoder;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
index 2139113..7a09b92 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
@@ -21,7 +21,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
@@ -32,15 +31,11 @@ import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
  * It implements {@link ErasureCoder}.
  */
 @InterfaceAudience.Private
-public class RSErasureEncoder extends AbstractErasureEncoder {
+public class RSErasureEncoder extends ErasureEncoder {
   private RawErasureEncoder rawEncoder;
 
-  public RSErasureEncoder(int numDataUnits, int numParityUnits) {
-    super(numDataUnits, numParityUnits);
-  }
-
-  public RSErasureEncoder(ECSchema schema) {
-    super(schema);
+  public RSErasureEncoder(ErasureCoderOptions options) {
+    super(options);
   }
 
   @Override
@@ -57,10 +52,8 @@ public class RSErasureEncoder extends AbstractErasureEncoder {
   private RawErasureEncoder checkCreateRSRawEncoder() {
     if (rawEncoder == null) {
       // TODO: we should create the raw coder according to codec.
-      ErasureCoderOptions coderOptions = new ErasureCoderOptions(
-          getNumDataUnits(), getNumParityUnits());
       rawEncoder = CodecUtil.createRawEncoder(getConf(),
-          ErasureCodeConstants.RS_DEFAULT_CODEC_NAME, coderOptions);
+          ErasureCodeConstants.RS_DEFAULT_CODEC_NAME, getOptions());
     }
     return rawEncoder;
   }
@@ -71,4 +64,9 @@ public class RSErasureEncoder extends AbstractErasureEncoder {
       rawEncoder.release();
     }
   }
+
+  @Override
+  public boolean preferDirectBuffer() {
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java
index 47fb8da..1a0e5c0 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java
@@ -21,7 +21,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
@@ -32,23 +31,17 @@ import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
  * It implements {@link ErasureCoder}.
  */
 @InterfaceAudience.Private
-public class XORErasureDecoder extends AbstractErasureDecoder {
+public class XORErasureDecoder extends ErasureDecoder {
 
-  public XORErasureDecoder(int numDataUnits, int numParityUnits) {
-    super(numDataUnits, numParityUnits);
-  }
-
-  public XORErasureDecoder(ECSchema schema) {
-    super(schema);
+  public XORErasureDecoder(ErasureCoderOptions options) {
+    super(options);
   }
 
   @Override
   protected ErasureCodingStep prepareDecodingStep(
       final ECBlockGroup blockGroup) {
-    ErasureCoderOptions coderOptions = new ErasureCoderOptions(
-        getNumDataUnits(), getNumParityUnits());
     RawErasureDecoder rawDecoder = CodecUtil.createRawDecoder(getConf(),
-        ErasureCodeConstants.XOR_CODEC_NAME, coderOptions);
+        ErasureCodeConstants.XOR_CODEC_NAME, getOptions());
 
     ECBlock[] inputBlocks = getInputBlocks(blockGroup);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java
index 1735179..89da617 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java
@@ -21,7 +21,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
@@ -32,28 +31,21 @@ import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
  * It implements {@link ErasureCoder}.
  */
 @InterfaceAudience.Private
-public class XORErasureEncoder extends AbstractErasureEncoder {
+public class XORErasureEncoder extends ErasureEncoder {
 
-  public XORErasureEncoder(int numDataUnits, int numParityUnits) {
-    super(numDataUnits, numParityUnits);
-  }
-
-  public XORErasureEncoder(ECSchema schema) {
-    super(schema);
+  public XORErasureEncoder(ErasureCoderOptions options) {
+    super(options);
   }
 
   @Override
   protected ErasureCodingStep prepareEncodingStep(
       final ECBlockGroup blockGroup) {
-    ErasureCoderOptions coderOptions = new ErasureCoderOptions(
-        getNumDataUnits(), getNumParityUnits());
     RawErasureEncoder rawEncoder = CodecUtil.createRawEncoder(getConf(),
-        ErasureCodeConstants.XOR_CODEC_NAME, coderOptions);
+        ErasureCodeConstants.XOR_CODEC_NAME, getOptions());
 
     ECBlock[] inputBlocks = getInputBlocks(blockGroup);
 
     return new ErasureEncodingStep(inputBlocks,
         getOutputBlocks(blockGroup), rawEncoder);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/package-info.java
new file mode 100644
index 0000000..8ea40c5
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/package-info.java
@@ -0,0 +1,28 @@
+/**
+ * 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.
+ */
+
+/**
+ * Erasure coders framework.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.io.erasurecode.coder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/CoderUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/CoderUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/CoderUtil.java
index ef34639..6072f13 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/CoderUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/CoderUtil.java
@@ -28,7 +28,7 @@ import java.util.Arrays;
  * Helpful utilities for implementing some raw erasure coders.
  */
 @InterfaceAudience.Private
-final class CoderUtil {
+public final class CoderUtil {
 
   private CoderUtil() {
     // No called

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
index 0727157..72316ad 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.fs.local.LocalConfigKeys;
 import org.apache.hadoop.ha.SshFenceByTcpPort;
 import org.apache.hadoop.ha.ZKFailoverController;
 import org.apache.hadoop.http.HttpServer2;
+import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.security.CompositeGroupsMapping;
 import org.apache.hadoop.security.HttpCrossOriginFilterInitializer;
@@ -49,6 +50,7 @@ import org.apache.hadoop.security.ssl.SSLFactory;
  * {@link org.apache.hadoop.security.LdapGroupsMapping}
  * {@link org.apache.hadoop.security.http.CrossOriginFilter}
  * {@link org.apache.hadoop.security.ssl.SSLFactory}
+ * {@link org.apache.hadoop.io.erasurecode.rawcoder.CoderUtil}
  * <p></p>
  * against core-site.xml for missing properties.  Currently only
  * throws an error if the class is missing a property.
@@ -71,7 +73,8 @@ public class TestCommonConfigurationFields extends TestConfigurationFieldsBase {
         LdapGroupsMapping.class,
         ZKFailoverController.class,
         SSLFactory.class,
-        CompositeGroupsMapping.class
+        CompositeGroupsMapping.class,
+        CodecUtil.class
         };
 
     // Initialize used variables

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCodecRawCoderMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCodecRawCoderMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCodecRawCoderMapping.java
index 81dc458..0db001a 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCodecRawCoderMapping.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCodecRawCoderMapping.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.io.erasurecode;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoderLegacy;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
@@ -73,7 +72,7 @@ public class TestCodecRawCoderMapping {
     String dummyFactName = "DummyNoneExistingFactory";
     // set the dummy factory to rs-legacy and create a raw coder
     // with rs-default, which is OK as the raw coder key is not used
-    conf.set(CommonConfigurationKeys.
+    conf.set(CodecUtil.
         IO_ERASURECODE_CODEC_RS_LEGACY_RAWCODER_KEY, dummyFactName);
     RawErasureEncoder encoder = CodecUtil.createRawEncoder(conf,
         ErasureCodeConstants.RS_DEFAULT_CODEC_NAME, coderOptions);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/codec/TestHHXORErasureCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/codec/TestHHXORErasureCodec.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/codec/TestHHXORErasureCodec.java
index c980b87..d5a3384 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/codec/TestHHXORErasureCodec.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/codec/TestHHXORErasureCodec.java
@@ -17,7 +17,9 @@
  */
 package org.apache.hadoop.io.erasurecode.codec;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.ErasureCodecOptions;
 import org.apache.hadoop.io.erasurecode.coder.ErasureCoder;
 import org.junit.Test;
 
@@ -25,10 +27,12 @@ import static org.junit.Assert.assertEquals;
 
 public class TestHHXORErasureCodec {
   private ECSchema schema = new ECSchema("hhxor", 10, 4);
+  private ErasureCodecOptions options = new ErasureCodecOptions(schema);
 
   @Test
   public void testGoodCodec() {
-    HHXORErasureCodec codec = new HHXORErasureCodec(schema);
+    HHXORErasureCodec codec
+        = new HHXORErasureCodec(new Configuration(), options);
     ErasureCoder encoder = codec.createEncoder();
     assertEquals(10, encoder.getNumDataUnits());
     assertEquals(4, encoder.getNumParityUnits());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
index 261be57..acbc136 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.io.erasurecode.coder;
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
 import org.apache.hadoop.io.erasurecode.ECChunk;
+import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
 import org.apache.hadoop.io.erasurecode.TestCoderBase;
 
 import java.lang.reflect.Constructor;
@@ -158,10 +159,12 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
   protected ErasureCoder createEncoder() {
     ErasureCoder encoder;
     try {
+      ErasureCoderOptions options = new ErasureCoderOptions(
+          numDataUnits, numParityUnits, allowChangeInputs, allowDump);
       Constructor<? extends ErasureCoder> constructor =
           (Constructor<? extends ErasureCoder>)
-              encoderClass.getConstructor(int.class, int.class);
-      encoder = constructor.newInstance(numDataUnits, numParityUnits);
+              encoderClass.getConstructor(ErasureCoderOptions.class);
+      encoder = constructor.newInstance(options);
     } catch (Exception e) {
       throw new RuntimeException("Failed to create encoder", e);
     }
@@ -177,10 +180,12 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
   protected ErasureCoder createDecoder() {
     ErasureCoder decoder;
     try {
+      ErasureCoderOptions options = new ErasureCoderOptions(
+          numDataUnits, numParityUnits, allowChangeInputs, allowDump);
       Constructor<? extends ErasureCoder> constructor =
           (Constructor<? extends ErasureCoder>)
-              decoderClass.getConstructor(int.class, int.class);
-      decoder = constructor.newInstance(numDataUnits, numParityUnits);
+              decoderClass.getConstructor(ErasureCoderOptions.class);
+      decoder = constructor.newInstance(options);
     } catch (Exception e) {
       throw new RuntimeException("Failed to create decoder", e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestHHXORErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestHHXORErasureCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestHHXORErasureCoder.java
index 09d1ec2..1eca1e1 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestHHXORErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestHHXORErasureCoder.java
@@ -18,7 +18,7 @@
 package org.apache.hadoop.io.erasurecode.coder;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactory;
 import org.junit.Before;
 import org.junit.Test;
@@ -50,7 +50,7 @@ public class TestHHXORErasureCoder extends TestHHErasureCoderBase {
      * This tests if the configuration items work or not.
      */
     Configuration conf = new Configuration();
-    conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_DEFAULT_RAWCODER_KEY,
+    conf.set(CodecUtil.IO_ERASURECODE_CODEC_RS_DEFAULT_RAWCODER_KEY,
         RSRawErasureCoderFactory.class.getCanonicalName());
     prepare(conf, 10, 4, new int[]{0}, new int[0]);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
index b0fe8f8..5f36c99 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
@@ -18,7 +18,7 @@
 package org.apache.hadoop.io.erasurecode.coder;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactory;
 import org.junit.Before;
 import org.junit.Rule;
@@ -57,7 +57,7 @@ public class TestRSErasureCoder extends TestErasureCoderBase {
      * This tests if the configuration items work or not.
      */
     Configuration conf = new Configuration();
-    conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_DEFAULT_RAWCODER_KEY,
+    conf.set(CodecUtil.IO_ERASURECODE_CODEC_RS_DEFAULT_RAWCODER_KEY,
         RSRawErasureCoderFactory.class.getCanonicalName());
     prepare(conf, 10, 4, new int[]{0}, new int[0]);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
index 1e27745..6af9e7c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
@@ -83,7 +82,7 @@ public class TestDFSStripedInputStream {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
     if (ErasureCodeNative.isNativeCodeLoaded()) {
       conf.set(
-          CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_DEFAULT_RAWCODER_KEY,
+          CodecUtil.IO_ERASURECODE_CODEC_RS_DEFAULT_RAWCODER_KEY,
           NativeRSRawErasureCoderFactory.class.getCanonicalName());
     }
     SimulatedFSDataset.setFactory(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
index 8d54f08..4f0a36c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
@@ -19,15 +19,13 @@ package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.io.erasurecode.ErasureCodeNative;
 import org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawErasureCoderFactory;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -70,7 +68,7 @@ public class TestDFSStripedOutputStream {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
     if (ErasureCodeNative.isNativeCodeLoaded()) {
       conf.set(
-          CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_DEFAULT_RAWCODER_KEY,
+          CodecUtil.IO_ERASURECODE_CODEC_RS_DEFAULT_RAWCODER_KEY,
           NativeRSRawErasureCoderFactory.class.getCanonicalName());
     }
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
index 11036a7..23809c3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
@@ -22,7 +22,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
@@ -37,6 +36,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.io.erasurecode.ErasureCodeNative;
 import org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawErasureCoderFactory;
 import org.apache.hadoop.security.token.Token;
@@ -188,7 +188,7 @@ public class TestDFSStripedOutputStreamWithFailure {
     final int numDNs = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
     if (ErasureCodeNative.isNativeCodeLoaded()) {
       conf.set(
-          CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_DEFAULT_RAWCODER_KEY,
+          CodecUtil.IO_ERASURECODE_CODEC_RS_DEFAULT_RAWCODER_KEY,
           NativeRSRawErasureCoderFactory.class.getCanonicalName());
     }
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c023c748/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFile.java
index 59e9f87..10fc43e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFile.java
@@ -33,7 +33,6 @@ import java.util.Random;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -48,6 +47,7 @@ import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.io.erasurecode.ErasureCodeNative;
 import org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawErasureCoderFactory;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -97,7 +97,7 @@ public class TestReconstructStripedFile {
         false);
     if (ErasureCodeNative.isNativeCodeLoaded()) {
       conf.set(
-          CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_DEFAULT_RAWCODER_KEY,
+          CodecUtil.IO_ERASURECODE_CODEC_RS_DEFAULT_RAWCODER_KEY,
           NativeRSRawErasureCoderFactory.class.getCanonicalName());
     }
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(dnNum).build();


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


[44/50] [abbrv] hadoop git commit: YARN-5718. TimelineClient (and other places in YARN) shouldn't over-write HDFS client retry settings which could cause unexpected behavior. Contributed by Junping Du.

Posted by sj...@apache.org.
YARN-5718. TimelineClient (and other places in YARN) shouldn't over-write HDFS client retry settings which could cause unexpected behavior. Contributed by Junping Du.


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

Branch: refs/heads/HADOOP-13070
Commit: b733a6f86262522e535cebc972baecbe6a6eab50
Parents: b154d3e
Author: Xuan <xg...@apache.org>
Authored: Tue Oct 18 11:04:49 2016 -0700
Committer: Xuan <xg...@apache.org>
Committed: Tue Oct 18 11:06:47 2016 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     | 17 +----------------
 .../api/impl/FileSystemTimelineWriter.java      |  7 -------
 .../nodelabels/FileSystemNodeLabelsStore.java   |  7 +------
 .../src/main/resources/yarn-default.xml         | 20 --------------------
 .../recovery/FileSystemRMStateStore.java        |  5 -----
 .../recovery/TestFSRMStateStore.java            |  4 ----
 6 files changed, 2 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b733a6f8/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 3bd0dcc..1a30c32 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
@@ -695,10 +695,6 @@ public class YarnConfiguration extends Configuration {
   /** URI for FileSystemRMStateStore */
   public static final String FS_RM_STATE_STORE_URI = RM_PREFIX
       + "fs.state-store.uri";
-  public static final String FS_RM_STATE_STORE_RETRY_POLICY_SPEC = RM_PREFIX
-      + "fs.state-store.retry-policy-spec";
-  public static final String DEFAULT_FS_RM_STATE_STORE_RETRY_POLICY_SPEC =
-      "2000, 500";
 
   public static final String FS_RM_STATE_STORE_NUM_RETRIES =
       RM_PREFIX + "fs.state-store.num-retries";
@@ -1974,13 +1970,6 @@ public class YarnConfiguration extends Configuration {
       TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_UNKNOWN_ACTIVE_SECONDS_DEFAULT
       = 24 * 60 * 60;
 
-  public static final String
-      TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_RETRY_POLICY_SPEC =
-      TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_PREFIX + "retry-policy-spec";
-  public static final String
-      DEFAULT_TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_RETRY_POLICY_SPEC =
-      "2000, 500";
-
   public static final String TIMELINE_SERVICE_LEVELDB_CACHE_READ_CACHE_SIZE =
       TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_PREFIX
           + "leveldb-cache-read-cache-size";
@@ -2600,11 +2589,7 @@ public class YarnConfiguration extends Configuration {
   /** URI for NodeLabelManager */
   public static final String FS_NODE_LABELS_STORE_ROOT_DIR = NODE_LABELS_PREFIX
       + "fs-store.root-dir";
-  public static final String FS_NODE_LABELS_STORE_RETRY_POLICY_SPEC =
-      NODE_LABELS_PREFIX + "fs-store.retry-policy-spec";
-  public static final String DEFAULT_FS_NODE_LABELS_STORE_RETRY_POLICY_SPEC =
-      "2000, 500";
-  
+
   /**
    * Flag to indicate if the node labels feature enabled, by default it's
    * disabled

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b733a6f8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
index 55d6bd2..b1284e1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
@@ -106,13 +106,6 @@ public class FileSystemTimelineWriter extends TimelineWriter{
     super(authUgi, client, resURI);
 
     Configuration fsConf = new Configuration(conf);
-    fsConf.setBoolean("dfs.client.retry.policy.enabled", true);
-    String retryPolicy =
-        fsConf.get(YarnConfiguration.
-            TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_RETRY_POLICY_SPEC,
-          YarnConfiguration.
-              DEFAULT_TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_RETRY_POLICY_SPEC);
-    fsConf.set("dfs.client.retry.policy.spec", retryPolicy);
 
     activePath = new Path(fsConf.get(
       YarnConfiguration

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b733a6f8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java
index a65349b..9a2b8be 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java
@@ -95,13 +95,8 @@ public class FileSystemNodeLabelsStore extends NodeLabelsStore {
 
   void setFileSystem(Configuration conf) throws IOException {
     Configuration confCopy = new Configuration(conf);
-    confCopy.setBoolean("dfs.client.retry.policy.enabled", true);
-    String retryPolicy =
-        confCopy.get(YarnConfiguration.FS_NODE_LABELS_STORE_RETRY_POLICY_SPEC,
-            YarnConfiguration.DEFAULT_FS_NODE_LABELS_STORE_RETRY_POLICY_SPEC);
-    confCopy.set("dfs.client.retry.policy.spec", retryPolicy);
     fs = fsWorkingPath.getFileSystem(confCopy);
-    
+
     // if it's local file system, use RawLocalFileSystem instead of
     // LocalFileSystem, the latter one doesn't support append.
     if (fs.getScheme().equals("file")) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b733a6f8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index f37c689..72e026d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -522,16 +522,6 @@
   </property>
 
   <property>
-    <description>hdfs client retry policy specification. hdfs client retry
-    is always enabled. Specified in pairs of sleep-time and number-of-retries
-    and (t0, n0), (t1, n1), ..., the first n0 retries sleep t0 milliseconds on
-    average, the following n1 retries sleep t1 milliseconds on average, and so on.
-    </description>
-    <name>yarn.resourcemanager.fs.state-store.retry-policy-spec</name>
-    <value>2000, 500</value>
-  </property>
-
-  <property>
     <description>the number of retries to recover from IOException in
     FileSystemRMStateStore.
     </description>
@@ -2483,16 +2473,6 @@
 
   <property>
     <description>
-    Retry policy used for FileSystem node label store. The policy is
-    specified by N pairs of sleep-time in milliseconds and number-of-retries
-    "s1,n1,s2,n2,...".
-    </description>
-    <name>yarn.node-labels.fs-store.retry-policy-spec</name>
-    <value>2000, 500</value>
-  </property>
-
-  <property>
-    <description>
     URI for NodeLabelManager.  The default value is
     /tmp/hadoop-yarn-${user}/node-labels/ in the local filesystem.
     </description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b733a6f8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
index a228037..b9a4374 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
@@ -142,11 +142,6 @@ public class FileSystemRMStateStore extends RMStateStore {
     // authenticated with kerberos so we are good to create a file-system
     // handle.
     fsConf = new Configuration(getConfig());
-    fsConf.setBoolean("dfs.client.retry.policy.enabled", true);
-    String retryPolicy =
-        fsConf.get(YarnConfiguration.FS_RM_STATE_STORE_RETRY_POLICY_SPEC,
-          YarnConfiguration.DEFAULT_FS_RM_STATE_STORE_RETRY_POLICY_SPEC);
-    fsConf.set("dfs.client.retry.policy.spec", retryPolicy);
 
     String scheme = fsWorkingPath.toUri().getScheme();
     if (scheme == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b733a6f8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.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/recovery/TestFSRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
index 61088e1..6f0d53f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
@@ -113,8 +113,6 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
       YarnConfiguration conf = new YarnConfiguration();
       conf.set(YarnConfiguration.FS_RM_STATE_STORE_URI,
           workingDirPathURI.toString());
-      conf.set(YarnConfiguration.FS_RM_STATE_STORE_RETRY_POLICY_SPEC,
-              "100,6000");
       conf.setInt(YarnConfiguration.FS_RM_STATE_STORE_NUM_RETRIES, 8);
       conf.setLong(YarnConfiguration.FS_RM_STATE_STORE_RETRY_INTERVAL_MS,
               900L);
@@ -325,8 +323,6 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
           YarnConfiguration conf = new YarnConfiguration();
           conf.set(YarnConfiguration.FS_RM_STATE_STORE_URI,
               workingDirPathURI.toString());
-          conf.set(YarnConfiguration.FS_RM_STATE_STORE_RETRY_POLICY_SPEC,
-              "100,6000");
           this.store = new TestFileSystemRMStore(conf) {
             Version storedVersion = null;
 


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


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

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


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

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

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


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

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

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

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


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


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

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


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

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

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


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

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

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


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