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 vv...@apache.org on 2015/07/01 12:45:32 UTC

[01/50] hadoop git commit: HDFS-8640. Make reserved RBW space visible through JMX. (Contributed by kanaka kumar avvaru)

Repository: hadoop
Updated Branches:
  refs/heads/YARN-2139 c906a1de7 -> 7405c5979


HDFS-8640. Make reserved RBW space visible through JMX. (Contributed by kanaka kumar avvaru)


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

Branch: refs/heads/YARN-2139
Commit: 67a62da5c5f592b07d083440ced3666c7709b20d
Parents: bc43390
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Jun 25 10:13:22 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 25 10:13:22 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |  5 +++-
 .../fsdataset/impl/TestRbwSpaceReservation.java | 28 ++++++++++++++++++++
 3 files changed, 35 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/67a62da5/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e5c30bd..01bb92d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -669,6 +669,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8462. Implement GETXATTRS and LISTXATTRS operations for WebImageViewer.
     (Jagadesh Kiran N via aajisaka)
 
+    HDFS-8640. Make reserved RBW space visible through JMX. (kanaka kumar
+    avvaru via Arpit Agarwal)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67a62da5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index a1ff918..a2bb2c0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -2559,13 +2559,15 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     final String directory;
     final long usedSpace; // size of space used by HDFS
     final long freeSpace; // size of free space excluding reserved space
-    final long reservedSpace; // size of space reserved for non-HDFS and RBW
+    final long reservedSpace; // size of space reserved for non-HDFS
+    final long reservedSpaceForRBW; // size of space reserved RBW
 
     VolumeInfo(FsVolumeImpl v, long usedSpace, long freeSpace) {
       this.directory = v.toString();
       this.usedSpace = usedSpace;
       this.freeSpace = freeSpace;
       this.reservedSpace = v.getReserved();
+      this.reservedSpaceForRBW = v.getReservedForRbw();
     }
   }  
 
@@ -2599,6 +2601,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       innerInfo.put("usedSpace", v.usedSpace);
       innerInfo.put("freeSpace", v.freeSpace);
       innerInfo.put("reservedSpace", v.reservedSpace);
+      innerInfo.put("reservedSpaceForRBW", v.reservedSpaceForRBW);
       info.put(v.directory, innerInfo);
     }
     return info;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67a62da5/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestRbwSpaceReservation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestRbwSpaceReservation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestRbwSpaceReservation.java
index 44bf81b..a647d96 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestRbwSpaceReservation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestRbwSpaceReservation.java
@@ -49,11 +49,15 @@ import org.mockito.Mockito;
 
 import java.io.IOException;
 import java.io.OutputStream;
+import java.lang.management.ManagementFactory;
 import java.lang.reflect.Field;
 import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.TimeoutException;
 
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+
 /**
  * Ensure that the DN reserves disk space equivalent to a full block for
  * replica being written (RBW).
@@ -324,6 +328,30 @@ public class TestRbwSpaceReservation {
         fsVolumeImpl.getReservedForRbw() == 0);
   }
 
+  @Test(timeout = 30000)
+  public void testRBWInJMXBean() throws Exception {
+
+    final short replication = 1;
+    startCluster(BLOCK_SIZE, replication, -1);
+
+    final String methodName = GenericTestUtils.getMethodName();
+    final Path file = new Path("/" + methodName + ".01.dat");
+
+    try (FSDataOutputStream os = fs.create(file, replication)) {
+      // Write 1 byte to the file
+      os.write(new byte[1]);
+      os.hsync();
+
+      final MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+      final ObjectName mxbeanName = new ObjectName(
+          "Hadoop:service=DataNode,name=DataNodeInfo");
+      final String volumeInfo = (String) mbs.getAttribute(mxbeanName,
+          "VolumeInfo");
+
+      assertTrue(volumeInfo.contains("reservedSpaceForRBW"));
+    }
+  }
+
   /**
    * Stress test to ensure we are not leaking reserved space.
    * @throws IOException


[22/50] hadoop git commit: HDFS-8653. Code cleanup for DatanodeManager, DatanodeDescriptor and DatanodeStorageInfo. Contributed by Zhe Zhang.

Posted by vv...@apache.org.
HDFS-8653. Code cleanup for DatanodeManager, DatanodeDescriptor and DatanodeStorageInfo. Contributed by Zhe Zhang.


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

Branch: refs/heads/YARN-2139
Commit: 2ffd84273ac490724fe7e7825664bb6d09ef0e99
Parents: d3fed8e
Author: Andrew Wang <wa...@apache.org>
Authored: Mon Jun 29 12:12:34 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Mon Jun 29 12:12:41 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../CacheReplicationMonitor.java                |  4 +--
 .../blockmanagement/DatanodeDescriptor.java     | 33 ++++++++++----------
 .../server/blockmanagement/DatanodeManager.java | 26 +++++++--------
 .../blockmanagement/DatanodeStorageInfo.java    | 13 ++++++--
 5 files changed, 45 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ffd8427/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e55f340..0c56f2b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -682,6 +682,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8623. Refactor NameNode handling of invalid, corrupt, and under-recovery
     blocks. (Zhe Zhang via jing9)
 
+    HDFS-8653. Code cleanup for DatanodeManager, DatanodeDescriptor and
+    DatanodeStorageInfo. (Zhe Zhang via wang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ffd8427/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 a0f3503..2f81ddf 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
@@ -452,8 +452,8 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
         file.getFullPathName(), cachedTotal, neededTotal);
   }
 
-  private String findReasonForNotCaching(CachedBlock cblock, 
-          BlockInfo blockInfo) {
+  private String findReasonForNotCaching(CachedBlock cblock,
+      BlockInfo blockInfo) {
     if (blockInfo == null) {
       // Somehow, a cache report with the block arrived, but the block
       // reports from the DataNode haven't (yet?) described such a block.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ffd8427/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 dd7b301..99def6b 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
@@ -64,7 +64,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();
+  public final DecommissioningStatus decommissioningStatus =
+      new DecommissioningStatus();
 
   private long curBlockReportId = 0;
 
@@ -115,7 +116,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
         return null;
       }
 
-      List<E> results = new ArrayList<E>();
+      List<E> results = new ArrayList<>();
       for(; !blockq.isEmpty() && numBlocks > 0; numBlocks--) {
         results.add(blockq.poll());
       }
@@ -135,7 +136,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
 
   private final Map<String, DatanodeStorageInfo> storageMap = 
-      new HashMap<String, DatanodeStorageInfo>();
+      new HashMap<>();
 
   /**
    * A list of CachedBlock objects on this datanode.
@@ -217,12 +218,14 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private long bandwidth;
 
   /** A queue of blocks to be replicated by this datanode */
-  private final BlockQueue<BlockTargetPair> replicateBlocks = new BlockQueue<BlockTargetPair>();
+  private final BlockQueue<BlockTargetPair> replicateBlocks =
+      new BlockQueue<>();
   /** A queue of blocks to be recovered by this datanode */
   private final BlockQueue<BlockInfoUnderConstruction> recoverBlocks =
-                                new BlockQueue<BlockInfoUnderConstruction>();
+      new BlockQueue<>();
   /** A set of blocks to be invalidated by this datanode */
-  private final LightWeightHashSet<Block> invalidateBlocks = new LightWeightHashSet<Block>();
+  private final LightWeightHashSet<Block> invalidateBlocks =
+      new LightWeightHashSet<>();
 
   /* Variables for maintaining number of blocks scheduled to be written to
    * this storage. This count is approximate and might be slightly bigger
@@ -230,9 +233,9 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * while writing the block).
    */
   private EnumCounters<StorageType> currApproxBlocksScheduled
-      = new EnumCounters<StorageType>(StorageType.class);
+      = new EnumCounters<>(StorageType.class);
   private EnumCounters<StorageType> prevApproxBlocksScheduled
-      = new EnumCounters<StorageType>(StorageType.class);
+      = new EnumCounters<>(StorageType.class);
   private long lastBlocksScheduledRollTime = 0;
   private static final int BLOCKS_SCHEDULED_ROLL_INTERVAL = 600*1000; //10min
   private int volumeFailures = 0;
@@ -276,6 +279,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
       return storageMap.get(storageID);
     }
   }
+
   DatanodeStorageInfo[] getStorageInfos() {
     synchronized (storageMap) {
       final Collection<DatanodeStorageInfo> storages = storageMap.values();
@@ -321,7 +325,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
               Long.toHexString(curBlockReportId));
           iter.remove();
           if (zombies == null) {
-            zombies = new LinkedList<DatanodeStorageInfo>();
+            zombies = new LinkedList<>();
           }
           zombies.add(storageInfo);
         }
@@ -350,10 +354,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
    */
   boolean removeBlock(String storageID, BlockInfo b) {
     DatanodeStorageInfo s = getStorageInfo(storageID);
-    if (s != null) {
-      return s.removeBlock(b);
-    }
-    return false;
+    return s != null && s.removeBlock(b);
   }
 
   public void resetBlocks() {
@@ -449,7 +450,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
           + this.volumeFailures + " to " + volFailures);
       synchronized (storageMap) {
         failedStorageInfos =
-            new HashSet<DatanodeStorageInfo>(storageMap.values());
+            new HashSet<>(storageMap.values());
       }
     }
 
@@ -505,7 +506,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
       HashMap<String, DatanodeStorageInfo> excessStorages;
 
       // Init excessStorages with all known storages.
-      excessStorages = new HashMap<String, DatanodeStorageInfo>(storageMap);
+      excessStorages = new HashMap<>(storageMap);
 
       // Remove storages that the DN reported in the heartbeat.
       for (final StorageReport report : reports) {
@@ -542,7 +543,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
     private final List<Iterator<BlockInfo>> iterators;
     
     private BlockIterator(final DatanodeStorageInfo... storages) {
-      List<Iterator<BlockInfo>> iterators = new ArrayList<Iterator<BlockInfo>>();
+      List<Iterator<BlockInfo>> iterators = new ArrayList<>();
       for (DatanodeStorageInfo e : storages) {
         iterators.add(e.getBlockIterator());
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ffd8427/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 8143fb4..4266004 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
@@ -85,7 +85,7 @@ public class DatanodeManager {
    * Mapping: StorageID -> DatanodeDescriptor
    */
   private final Map<String, DatanodeDescriptor> datanodeMap
-      = new HashMap<String, DatanodeDescriptor>();
+      = new HashMap<>();
 
   /** Cluster network topology */
   private final NetworkTopology networktopology;
@@ -162,7 +162,7 @@ public class DatanodeManager {
    * Software version -> Number of datanodes with this version
    */
   private HashMap<String, Integer> datanodesSoftwareVersions =
-    new HashMap<String, Integer>(4, 0.75f);
+    new HashMap<>(4, 0.75f);
   
   /**
    * The minimum time between resending caching directives to Datanodes,
@@ -217,7 +217,7 @@ public class DatanodeManager {
     // locations of those hosts in the include list and store the mapping
     // in the cache; so future calls to resolve will be fast.
     if (dnsToSwitchMapping instanceof CachedDNSToSwitchMapping) {
-      final ArrayList<String> locations = new ArrayList<String>();
+      final ArrayList<String> locations = new ArrayList<>();
       for (InetSocketAddress addr : hostFileManager.getIncludes()) {
         locations.add(addr.getAddress().getHostAddress());
       }
@@ -370,7 +370,7 @@ public class DatanodeManager {
     // here we should get node but not datanode only .
     Node client = getDatanodeByHost(targethost);
     if (client == null) {
-      List<String> hosts = new ArrayList<String> (1);
+      List<String> hosts = new ArrayList<> (1);
       hosts.add(targethost);
       List<String> resolvedHosts = dnsToSwitchMapping.resolve(hosts);
       if (resolvedHosts != null && !resolvedHosts.isEmpty()) {
@@ -522,7 +522,7 @@ public class DatanodeManager {
   void datanodeDump(final PrintWriter out) {
     synchronized (datanodeMap) {
       Map<String,DatanodeDescriptor> sortedDatanodeMap =
-          new TreeMap<String,DatanodeDescriptor>(datanodeMap);
+          new TreeMap<>(datanodeMap);
       out.println("Metasave: Number of datanodes: " + datanodeMap.size());
       for (DatanodeDescriptor node : sortedDatanodeMap.values()) {
         out.println(node.dumpDatanode());
@@ -660,7 +660,7 @@ public class DatanodeManager {
 
   private void countSoftwareVersions() {
     synchronized(datanodeMap) {
-      HashMap<String, Integer> versionCount = new HashMap<String, Integer>();
+      HashMap<String, Integer> versionCount = new HashMap<>();
       for(DatanodeDescriptor dn: datanodeMap.values()) {
         // Check isAlive too because right after removeDatanode(),
         // isDatanodeDead() is still true 
@@ -677,7 +677,7 @@ public class DatanodeManager {
 
   public HashMap<String, Integer> getDatanodesSoftwareVersions() {
     synchronized(datanodeMap) {
-      return new HashMap<String, Integer> (this.datanodesSoftwareVersions);
+      return new HashMap<> (this.datanodesSoftwareVersions);
     }
   }
   
@@ -710,7 +710,7 @@ public class DatanodeManager {
    */
   private String resolveNetworkLocation (DatanodeID node) 
       throws UnresolvedTopologyException {
-    List<String> names = new ArrayList<String>(1);
+    List<String> names = new ArrayList<>(1);
     if (dnsToSwitchMapping instanceof CachedDNSToSwitchMapping) {
       names.add(node.getIpAddr());
     } else {
@@ -1000,7 +1000,7 @@ public class DatanodeManager {
       // If the network location is invalid, clear the cached mappings
       // so that we have a chance to re-add this DataNode with the
       // correct network location later.
-      List<String> invalidNodeNames = new ArrayList<String>(3);
+      List<String> invalidNodeNames = new ArrayList<>(3);
       // clear cache for nodes in IP or Hostname
       invalidNodeNames.add(nodeReg.getIpAddr());
       invalidNodeNames.add(nodeReg.getHostName());
@@ -1275,7 +1275,7 @@ public class DatanodeManager {
     final HostFileManager.HostSet excludedNodes = hostFileManager.getExcludes();
 
     synchronized(datanodeMap) {
-      nodes = new ArrayList<DatanodeDescriptor>(datanodeMap.size());
+      nodes = new ArrayList<>(datanodeMap.size());
       for (DatanodeDescriptor dn : datanodeMap.values()) {
         final boolean isDead = isDatanodeDead(dn);
         final boolean isDecommissioning = dn.isDecommissionInProgress();
@@ -1351,7 +1351,7 @@ public class DatanodeManager {
       VolumeFailureSummary volumeFailureSummary) throws IOException {
     synchronized (heartbeatManager) {
       synchronized (datanodeMap) {
-        DatanodeDescriptor nodeinfo = null;
+        DatanodeDescriptor nodeinfo;
         try {
           nodeinfo = getDatanode(nodeReg);
         } catch(UnregisteredNodeException e) {
@@ -1389,7 +1389,7 @@ public class DatanodeManager {
             final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations();
             // Skip stale nodes during recovery - not heart beated for some time (30s by default).
             final List<DatanodeStorageInfo> recoveryLocations =
-                new ArrayList<DatanodeStorageInfo>(storages.length);
+                new ArrayList<>(storages.length);
             for (int i = 0; i < storages.length; i++) {
               if (!storages[i].getDatanodeDescriptor().isStale(staleInterval)) {
                 recoveryLocations.add(storages[i]);
@@ -1431,7 +1431,7 @@ public class DatanodeManager {
           return new DatanodeCommand[] { brCommand };
         }
 
-        final List<DatanodeCommand> cmds = new ArrayList<DatanodeCommand>();
+        final List<DatanodeCommand> cmds = new ArrayList<>();
         //check pending replication
         List<BlockTargetPair> pendingList = nodeinfo.getReplicationCommand(
               maxTransfers);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ffd8427/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 65b83e1..92841a6 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
@@ -37,8 +37,9 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 public class DatanodeStorageInfo {
   public static final DatanodeStorageInfo[] EMPTY_ARRAY = {};
 
-  public static DatanodeInfo[] toDatanodeInfos(DatanodeStorageInfo[] storages) {
-    return toDatanodeInfos(Arrays.asList(storages));
+  public static DatanodeInfo[] toDatanodeInfos(
+      DatanodeStorageInfo[] storages) {
+    return storages == null ? null: toDatanodeInfos(Arrays.asList(storages));
   }
   static DatanodeInfo[] toDatanodeInfos(List<DatanodeStorageInfo> storages) {
     final DatanodeInfo[] datanodes = new DatanodeInfo[storages.size()];
@@ -58,6 +59,9 @@ public class DatanodeStorageInfo {
   }
 
   public static String[] toStorageIDs(DatanodeStorageInfo[] storages) {
+    if (storages == null) {
+      return null;
+    }
     String[] storageIDs = new String[storages.length];
     for(int i = 0; i < storageIDs.length; i++) {
       storageIDs[i] = storages[i].getStorageID();
@@ -66,6 +70,9 @@ public class DatanodeStorageInfo {
   }
 
   public static StorageType[] toStorageTypes(DatanodeStorageInfo[] storages) {
+    if (storages == null) {
+      return null;
+    }
     StorageType[] storageTypes = new StorageType[storages.length];
     for(int i = 0; i < storageTypes.length; i++) {
       storageTypes[i] = storages[i].getStorageType();
@@ -380,6 +387,6 @@ public class DatanodeStorageInfo {
   }
 
   static enum AddBlockResult {
-    ADDED, REPLACED, ALREADY_EXIST;
+    ADDED, REPLACED, ALREADY_EXIST
   }
 }


[31/50] hadoop git commit: HDFS-8687. Remove the duplicate usage message from Dfsck.java. (Contributed by Brahma Reddy Battula)

Posted by vv...@apache.org.
HDFS-8687. Remove the duplicate usage message from Dfsck.java. (Contributed by Brahma Reddy Battula)


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

Branch: refs/heads/YARN-2139
Commit: 8e333720e13428a4d0d0f65692102f8f2e5da98d
Parents: 43a1288
Author: Arpit Agarwal <ar...@apache.org>
Authored: Mon Jun 29 14:49:00 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Mon Jun 29 14:56:42 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java         | 1 -
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8e333720/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index eb006eb..108a6c0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -977,6 +977,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8628. Update missing command option for fetchdt
     (J.Andreina via vinayakumarb)
 
+    HDFS-8687. Remove the duplicate usage message from Dfsck.java. (Brahma
+    Reddy Battula via Arpit Agarwal)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8e333720/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
index 33033e7..6bb6603 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
@@ -377,7 +377,6 @@ public class DFSck extends Configured implements Tool {
     int res = -1;
     if ((args.length == 0) || ("-files".equals(args[0]))) {
       printUsage(System.err);
-      ToolRunner.printGenericCommandUsage(System.err);
     } else if (DFSUtil.parseHelpArgument(args, USAGE, System.out, true)) {
       res = 0;
     } else {


[34/50] hadoop git commit: HADOOP-12154. FileSystem#getUsed() returns the file length only from root '/' (Contributed by J.Andreina)

Posted by vv...@apache.org.
HADOOP-12154. FileSystem#getUsed() returns the file length only from root '/' (Contributed by J.Andreina)


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

Branch: refs/heads/YARN-2139
Commit: 6d99017f38f5a158b5cb65c74688b4c833e4e35f
Parents: 77588e1
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Jun 30 15:25:20 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Tue Jun 30 15:25:20 2015 +0530

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +++
 .../java/org/apache/hadoop/fs/FileSystem.java   |  6 ++---
 .../org/apache/hadoop/hdfs/TestDFSShell.java    | 26 ++++++++++++++++++++
 3 files changed, 32 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d99017f/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 50192ae..2cf9509 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -899,6 +899,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12089. StorageException complaining " no lease ID" when updating
     FolderLastModifiedTime in WASB. (Duo Xu via cnauroth)
 
+    HADOOP-12154. FileSystem#getUsed() returns the file length only from root '/'
+    (J.Andreina via vinayakumarb)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d99017f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
index 1d7bc87..c98074a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
@@ -2085,9 +2085,9 @@ public abstract class FileSystem extends Configured implements Closeable {
   /** Return the total size of all files in the filesystem.*/
   public long getUsed() throws IOException{
     long used = 0;
-    FileStatus[] files = listStatus(new Path("/"));
-    for(FileStatus file:files){
-      used += file.getLen();
+    RemoteIterator<LocatedFileStatus> files = listFiles(new Path("/"), true);
+    while (files.hasNext()) {
+      used += files.next().getLen();
     }
     return used;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d99017f/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 2df31c4..1386124 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
@@ -913,6 +913,32 @@ public class TestDFSShell {
       cluster.shutdown();
     }
   }
+
+  @Test(timeout = 30000)
+  public void testTotalSizeOfAllFiles() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+      FileSystem fs = cluster.getFileSystem();
+      // create file under root
+      FSDataOutputStream File1 = fs.create(new Path("/File1"));
+      File1.write("hi".getBytes());
+      File1.close();
+      // create file under sub-folder
+      FSDataOutputStream File2 = fs.create(new Path("/Folder1/File2"));
+      File2.write("hi".getBytes());
+      File2.close();
+      // getUsed() should return total length of all the files in Filesystem
+      assertEquals(4, fs.getUsed());
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+        cluster = null;
+      }
+    }
+  }
+
   private static void runCount(String path, long dirs, long files, FsShell shell
     ) throws IOException {
     ByteArrayOutputStream bytes = new ByteArrayOutputStream();


[50/50] hadoop git commit: YARN-3823. Fix mismatch in default values for yarn.scheduler.maximum-allocation-vcores property. Contributed by Ray Chiang.

Posted by vv...@apache.org.
YARN-3823. Fix mismatch in default values for
yarn.scheduler.maximum-allocation-vcores property. Contributed by Ray
Chiang.


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

Branch: refs/heads/YARN-2139
Commit: 7405c59799ed1b8ad1a7c6f1b18fabf49d0b92b2
Parents: 532e38c
Author: Devaraj K <de...@apache.org>
Authored: Wed Jul 1 11:11:26 2015 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Wed Jul 1 11:11:26 2015 +0530

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7405c597/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 5e9243d..8c07e48 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -580,6 +580,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3768. ArrayIndexOutOfBoundsException with empty environment variables.
     (Zhihai Xu via gera)
 
+    YARN-3823. Fix mismatch in default values for 
+    yarn.scheduler.maximum-allocation-vcores property. (Ray Chiang via devaraj)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7405c597/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 621198c..4987084 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
@@ -276,7 +276,7 @@
     in terms of virtual CPU cores. Requests higher than this will throw a
     InvalidResourceRequestException.</description>
     <name>yarn.scheduler.maximum-allocation-vcores</name>
-    <value>32</value>
+    <value>4</value>
   </property>
 
   <property>


[23/50] hadoop git commit: HADOOP-12104. Migrate Hadoop Pipes native build to new CMake framework (Alan Burlison via Colin P. McCabe)

Posted by vv...@apache.org.
HADOOP-12104. Migrate Hadoop Pipes native build to new CMake framework (Alan Burlison via Colin P. McCabe)


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

Branch: refs/heads/YARN-2139
Commit: 5a27c3fd7616215937264c2b1f015205e60f2d73
Parents: 2ffd842
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Mon Jun 29 12:01:17 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Mon Jun 29 12:31:26 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 ++
 hadoop-tools/hadoop-pipes/src/CMakeLists.txt    | 49 +++++++++-----------
 2 files changed, 24 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a27c3fd/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 0a964a3..e8e85a0 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -685,6 +685,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11885. hadoop-dist dist-layout-stitching.sh does not work with dash.
     (wang)
 
+    HADOOP-12104. Migrate Hadoop Pipes native build to new CMake framework
+    (alanburlison via cmccabe)
+
     HADOOP-12036. Consolidate all of the cmake extensions in one directory
     (alanburlison via cmccabe)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a27c3fd/hadoop-tools/hadoop-pipes/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-pipes/src/CMakeLists.txt b/hadoop-tools/hadoop-pipes/src/CMakeLists.txt
index 170af49..3b0b28c 100644
--- a/hadoop-tools/hadoop-pipes/src/CMakeLists.txt
+++ b/hadoop-tools/hadoop-pipes/src/CMakeLists.txt
@@ -6,7 +6,7 @@
 # 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
@@ -17,25 +17,11 @@
 #
 
 cmake_minimum_required(VERSION 2.6 FATAL_ERROR)
-find_package(OpenSSL REQUIRED)
-
-set(CMAKE_BUILD_TYPE, Release)
 
-set(PIPES_FLAGS "-g -Wall -O2 -D_REENTRANT -D_GNU_SOURCE")
-set(PIPES_FLAGS "${PIPES_FLAGS} -D_LARGEFILE_SOURCE -D_FILE_OFFSET_BITS=64")
-set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${PIPES_FLAGS}")
-set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${PIPES_FLAGS}")
+list(APPEND CMAKE_MODULE_PATH ${CMAKE_SOURCE_DIR}/../../../hadoop-common-project/hadoop-common)
+include(HadoopCommon)
 
-include(../../../hadoop-common-project/hadoop-common/src/JNIFlags.cmake NO_POLICY_SCOPE)
-
-function(output_directory TGT DIR)
-    SET_TARGET_PROPERTIES(${TGT} PROPERTIES
-        RUNTIME_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-   SET_TARGET_PROPERTIES(${TGT} PROPERTIES
-        ARCHIVE_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-    SET_TARGET_PROPERTIES(${TGT} PROPERTIES
-        LIBRARY_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-endfunction(output_directory TGT DIR)
+find_package(OpenSSL REQUIRED)
 
 include_directories(
     main/native/utils/api
@@ -47,19 +33,19 @@ include_directories(
 # Example programs
 add_executable(wordcount-simple main/native/examples/impl/wordcount-simple.cc)
 target_link_libraries(wordcount-simple hadooppipes hadooputils)
-output_directory(wordcount-simple examples)
+hadoop_output_directory(wordcount-simple examples)
 
 add_executable(wordcount-part main/native/examples/impl/wordcount-part.cc)
 target_link_libraries(wordcount-part hadooppipes hadooputils)
-output_directory(wordcount-part examples)
+hadoop_output_directory(wordcount-part examples)
 
 add_executable(wordcount-nopipe main/native/examples/impl/wordcount-nopipe.cc)
 target_link_libraries(wordcount-nopipe hadooppipes hadooputils)
-output_directory(wordcount-nopipe examples)
+hadoop_output_directory(wordcount-nopipe examples)
 
 add_executable(pipes-sort main/native/examples/impl/sort.cc)
 target_link_libraries(pipes-sort hadooppipes hadooputils)
-output_directory(pipes-sort examples)
+hadoop_output_directory(pipes-sort examples)
 
 add_library(hadooputils STATIC
     main/native/utils/impl/StringUtils.cc
@@ -70,15 +56,22 @@ add_library(hadooppipes STATIC
     main/native/pipes/impl/HadoopPipes.cc
 )
 
-INCLUDE(CheckLibraryExists)
-CHECK_LIBRARY_EXISTS(dl dlopen "" NEED_LINK_DL)
+include(CheckLibraryExists)
+check_library_exists(dl dlopen "" NEED_LINK_DL)
+
+if(NEED_LINK_DL)
+    set(LIB_DL "dl")
+endif()
 
-if (NEED_LINK_DL)
-    set(LIB_DL dl)
-endif (NEED_LINK_DL)
+if(${CMAKE_SYSTEM_NAME} MATCHES "SunOS")
+    exec_program("uname" ARGS "-r" OUTPUT_VARIABLE OS_VERSION)
+    if(OS_VERSION VERSION_LESS "5.12")
+        set(LIB_NET "socket" "nsl")
+    endif()
+endif()
 
 target_link_libraries(hadooppipes
     ${OPENSSL_LIBRARIES}
     ${LIB_DL}
-    pthread
+    ${LIB_NET}
 )


[32/50] hadoop git commit: HDFS-8493. Consolidate truncate() related implementation in a single class. Contributed by Rakesh R.

Posted by vv...@apache.org.
HDFS-8493. Consolidate truncate() related implementation in a single class. 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/d3797f9f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d3797f9f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d3797f9f

Branch: refs/heads/YARN-2139
Commit: d3797f9f3cf502b7bfee3b64c641807b276c6faf
Parents: 8e33372
Author: Haohui Mai <wh...@apache.org>
Authored: Mon Jun 29 16:40:46 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Mon Jun 29 16:45:35 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hdfs/server/namenode/FSDirTruncateOp.java   | 361 +++++++++++++++++++
 .../hdfs/server/namenode/FSDirectory.java       |  95 -----
 .../hdfs/server/namenode/FSEditLogLoader.java   |   6 +-
 .../hdfs/server/namenode/FSNamesystem.java      | 237 ++----------
 .../hdfs/server/namenode/TestFileTruncate.java  |   8 +-
 6 files changed, 402 insertions(+), 308 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3797f9f/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 108a6c0..3535f90 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -685,6 +685,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8653. Code cleanup for DatanodeManager, DatanodeDescriptor and
     DatanodeStorageInfo. (Zhe Zhang via wang)
 
+    HDFS-8493. Consolidate truncate() related implementation in a single class.
+    (Rakesh R via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3797f9f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
new file mode 100644
index 0000000..9fc9def
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.IOException;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.RecoverLeaseOp;
+import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Helper class to perform truncate operation.
+ */
+final class FSDirTruncateOp {
+
+  /**
+   * Private constructor for preventing FSDirTruncateOp object creation.
+   * Static-only class.
+   */
+  private FSDirTruncateOp() {}
+
+  /**
+   * Truncate a file to a given size.
+   *
+   * @param fsn namespace
+   * @param srcArg path name
+   * @param newLength the target file size
+   * @param clientName client name
+   * @param clientMachine client machine info
+   * @param mtime modified time
+   * @param toRemoveBlocks to be removed blocks
+   * @param pc permission checker to check fs permission
+   * @return tuncate result
+   * @throws IOException
+   */
+  static TruncateResult truncate(final FSNamesystem fsn, final String srcArg,
+      final long newLength, final String clientName,
+      final String clientMachine, final long mtime,
+      final BlocksMapUpdateInfo toRemoveBlocks, final FSPermissionChecker pc)
+      throws IOException, UnresolvedLinkException {
+    assert fsn.hasWriteLock();
+
+    FSDirectory fsd = fsn.getFSDirectory();
+    byte[][] pathComponents = FSDirectory
+        .getPathComponentsForReservedPath(srcArg);
+    final String src;
+    final INodesInPath iip;
+    final boolean onBlockBoundary;
+    Block truncateBlock = null;
+    fsd.writeLock();
+    try {
+      src = fsd.resolvePath(pc, srcArg, pathComponents);
+      iip = fsd.getINodesInPath4Write(src, true);
+      if (fsn.isPermissionEnabled()) {
+        fsd.checkPathAccess(pc, iip, FsAction.WRITE);
+      }
+      INodeFile file = INodeFile.valueOf(iip.getLastINode(), src);
+      final BlockStoragePolicy lpPolicy = fsn.getBlockManager()
+          .getStoragePolicy("LAZY_PERSIST");
+
+      if (lpPolicy != null && lpPolicy.getId() == file.getStoragePolicyID()) {
+        throw new UnsupportedOperationException(
+            "Cannot truncate lazy persist file " + src);
+      }
+
+      // Check if the file is already being truncated with the same length
+      final BlockInfo last = file.getLastBlock();
+      if (last != null && last.getBlockUCState()
+          == BlockUCState.UNDER_RECOVERY) {
+        final Block truncatedBlock = ((BlockInfoUnderConstruction) last)
+            .getTruncateBlock();
+        if (truncatedBlock != null) {
+          final long truncateLength = file.computeFileSize(false, false)
+              + truncatedBlock.getNumBytes();
+          if (newLength == truncateLength) {
+            return new TruncateResult(false, fsd.getAuditFileInfo(iip));
+          }
+        }
+      }
+
+      // Opening an existing file for truncate. May need lease recovery.
+      fsn.recoverLeaseInternal(RecoverLeaseOp.TRUNCATE_FILE, iip, src,
+          clientName, clientMachine, false);
+      // Truncate length check.
+      long oldLength = file.computeFileSize();
+      if (oldLength == newLength) {
+        return new TruncateResult(true, fsd.getAuditFileInfo(iip));
+      }
+      if (oldLength < newLength) {
+        throw new HadoopIllegalArgumentException(
+            "Cannot truncate to a larger file size. Current size: " + oldLength
+                + ", truncate size: " + newLength + ".");
+      }
+      // Perform INodeFile truncation.
+      final QuotaCounts delta = new QuotaCounts.Builder().build();
+      onBlockBoundary = unprotectedTruncate(fsn, iip, newLength,
+          toRemoveBlocks, mtime, delta);
+      if (!onBlockBoundary) {
+        // Open file for write, but don't log into edits
+        long lastBlockDelta = file.computeFileSize() - newLength;
+        assert lastBlockDelta > 0 : "delta is 0 only if on block bounday";
+        truncateBlock = prepareFileForTruncate(fsn, iip, clientName,
+            clientMachine, lastBlockDelta, null);
+      }
+
+      // update the quota: use the preferred block size for UC block
+      fsd.updateCountNoQuotaCheck(iip, iip.length() - 1, delta);
+    } finally {
+      fsd.writeUnlock();
+    }
+
+    fsn.getEditLog().logTruncate(src, clientName, clientMachine, newLength,
+        mtime, truncateBlock);
+    return new TruncateResult(onBlockBoundary, fsd.getAuditFileInfo(iip));
+  }
+
+  /**
+   * Unprotected truncate implementation. Unlike
+   * {@link FSDirTruncateOp#truncate}, this will not schedule block recovery.
+   *
+   * @param fsn namespace
+   * @param src path name
+   * @param clientName client name
+   * @param clientMachine client machine info
+   * @param newLength the target file size
+   * @param mtime modified time
+   * @param truncateBlock truncate block
+   * @throws IOException
+   */
+  static void unprotectedTruncate(final FSNamesystem fsn, final String src,
+      final String clientName, final String clientMachine,
+      final long newLength, final long mtime, final Block truncateBlock)
+      throws UnresolvedLinkException, QuotaExceededException,
+      SnapshotAccessControlException, IOException {
+    assert fsn.hasWriteLock();
+
+    FSDirectory fsd = fsn.getFSDirectory();
+    INodesInPath iip = fsd.getINodesInPath(src, true);
+    INodeFile file = iip.getLastINode().asFile();
+    BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
+    boolean onBlockBoundary = unprotectedTruncate(fsn, iip, newLength,
+        collectedBlocks, mtime, null);
+
+    if (!onBlockBoundary) {
+      BlockInfo oldBlock = file.getLastBlock();
+      Block tBlk = prepareFileForTruncate(fsn, iip, clientName, clientMachine,
+          file.computeFileSize() - newLength, truncateBlock);
+      assert Block.matchingIdAndGenStamp(tBlk, truncateBlock) &&
+          tBlk.getNumBytes() == truncateBlock.getNumBytes() :
+          "Should be the same block.";
+      if (oldBlock.getBlockId() != tBlk.getBlockId()
+          && !file.isBlockInLatestSnapshot(oldBlock)) {
+        fsn.getBlockManager().removeBlockFromMap(oldBlock);
+      }
+    }
+    assert onBlockBoundary == (truncateBlock == null) :
+      "truncateBlock is null iff on block boundary: " + truncateBlock;
+    fsn.removeBlocksAndUpdateSafemodeTotal(collectedBlocks);
+  }
+
+  /**
+   * Convert current INode to UnderConstruction. Recreate lease. Create new
+   * block for the truncated copy. Schedule truncation of the replicas.
+   *
+   * @param fsn namespace
+   * @param iip inodes in the path containing the file
+   * @param leaseHolder lease holder
+   * @param clientMachine client machine info
+   * @param lastBlockDelta last block delta size
+   * @param newBlock new block
+   * @return the returned block will be written to editLog and passed back
+   *         into this method upon loading.
+   * @throws IOException
+   */
+  @VisibleForTesting
+  static Block prepareFileForTruncate(FSNamesystem fsn, INodesInPath iip,
+      String leaseHolder, String clientMachine, long lastBlockDelta,
+      Block newBlock) throws IOException {
+    assert fsn.hasWriteLock();
+
+    INodeFile file = iip.getLastINode().asFile();
+    file.recordModification(iip.getLatestSnapshotId());
+    file.toUnderConstruction(leaseHolder, clientMachine);
+    assert file.isUnderConstruction() : "inode should be under construction.";
+    fsn.getLeaseManager().addLease(
+        file.getFileUnderConstructionFeature().getClientName(), file.getId());
+    boolean shouldRecoverNow = (newBlock == null);
+    BlockInfo oldBlock = file.getLastBlock();
+    boolean shouldCopyOnTruncate = shouldCopyOnTruncate(fsn, file, oldBlock);
+    if (newBlock == null) {
+      newBlock = (shouldCopyOnTruncate) ? fsn.createNewBlock() : new Block(
+          oldBlock.getBlockId(), oldBlock.getNumBytes(),
+          fsn.nextGenerationStamp(fsn.getBlockIdManager().isLegacyBlock(
+              oldBlock)));
+    }
+
+    BlockInfoUnderConstruction truncatedBlockUC;
+    if (shouldCopyOnTruncate) {
+      // Add new truncateBlock into blocksMap and
+      // use oldBlock as a source for copy-on-truncate recovery
+      truncatedBlockUC = new BlockInfoUnderConstructionContiguous(newBlock,
+          file.getPreferredBlockReplication());
+      truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta);
+      truncatedBlockUC.setTruncateBlock(oldBlock);
+      file.setLastBlock(truncatedBlockUC,
+          fsn.getBlockManager().getStorages(oldBlock));
+      fsn.getBlockManager().addBlockCollection(truncatedBlockUC, file);
+
+      NameNode.stateChangeLog.debug(
+          "BLOCK* prepareFileForTruncate: Scheduling copy-on-truncate to new"
+              + " size {}  new block {} old block {}",
+          truncatedBlockUC.getNumBytes(), newBlock,
+          truncatedBlockUC.getTruncateBlock());
+    } else {
+      // Use new generation stamp for in-place truncate recovery
+      fsn.getBlockManager().convertLastBlockToUnderConstruction(file,
+          lastBlockDelta);
+      oldBlock = file.getLastBlock();
+      assert !oldBlock.isComplete() : "oldBlock should be under construction";
+      truncatedBlockUC = (BlockInfoUnderConstruction) oldBlock;
+      truncatedBlockUC.setTruncateBlock(new Block(oldBlock));
+      truncatedBlockUC.getTruncateBlock().setNumBytes(
+          oldBlock.getNumBytes() - lastBlockDelta);
+      truncatedBlockUC.getTruncateBlock().setGenerationStamp(
+          newBlock.getGenerationStamp());
+
+      NameNode.stateChangeLog.debug(
+          "BLOCK* prepareFileForTruncate: {} Scheduling in-place block "
+              + "truncate to new size {}", truncatedBlockUC.getTruncateBlock()
+              .getNumBytes(), truncatedBlockUC);
+    }
+    if (shouldRecoverNow) {
+      truncatedBlockUC.initializeBlockRecovery(newBlock.getGenerationStamp());
+    }
+
+    return newBlock;
+  }
+
+  /**
+   * Truncate has the following properties:
+   * 1.) Any block deletions occur now.
+   * 2.) INode length is truncated now - new clients can only read up to
+   *     the truncated length.
+   * 3.) INode will be set to UC and lastBlock set to UNDER_RECOVERY.
+   * 4.) NN will trigger DN truncation recovery and waits for DNs to report.
+   * 5.) File is considered UNDER_RECOVERY until truncation recovery
+   *     completes.
+   * 6.) Soft and hard Lease expiration require truncation recovery to
+   *     complete.
+   *
+   * @return true if on the block boundary or false if recovery is need
+   */
+  private static boolean unprotectedTruncate(FSNamesystem fsn,
+      INodesInPath iip, long newLength, BlocksMapUpdateInfo collectedBlocks,
+      long mtime, QuotaCounts delta) throws IOException {
+    assert fsn.hasWriteLock();
+
+    INodeFile file = iip.getLastINode().asFile();
+    int latestSnapshot = iip.getLatestSnapshotId();
+    file.recordModification(latestSnapshot, true);
+
+    verifyQuotaForTruncate(fsn, iip, file, newLength, delta);
+
+    long remainingLength =
+        file.collectBlocksBeyondMax(newLength, collectedBlocks);
+    file.excludeSnapshotBlocks(latestSnapshot, collectedBlocks);
+    file.setModificationTime(mtime);
+    // return whether on a block boundary
+    return (remainingLength - newLength) == 0;
+  }
+
+  private static void verifyQuotaForTruncate(FSNamesystem fsn,
+      INodesInPath iip, INodeFile file, long newLength, QuotaCounts delta)
+      throws QuotaExceededException {
+    FSDirectory fsd = fsn.getFSDirectory();
+    if (!fsn.isImageLoaded() || fsd.shouldSkipQuotaChecks()) {
+      // Do not check quota if edit log is still being processed
+      return;
+    }
+    final BlockStoragePolicy policy = fsd.getBlockStoragePolicySuite()
+        .getPolicy(file.getStoragePolicyID());
+    file.computeQuotaDeltaForTruncate(newLength, policy, delta);
+    fsd.readLock();
+    try {
+      FSDirectory.verifyQuota(iip, iip.length() - 1, delta, null);
+    } finally {
+      fsd.readUnlock();
+    }
+  }
+
+  /**
+   * Defines if a replica needs to be copied on truncate or
+   * can be truncated in place.
+   */
+  private static boolean shouldCopyOnTruncate(FSNamesystem fsn, INodeFile file,
+      BlockInfo blk) {
+    if (!fsn.isUpgradeFinalized()) {
+      return true;
+    }
+    if (fsn.isRollingUpgrade()) {
+      return true;
+    }
+    return file.isBlockInLatestSnapshot(blk);
+  }
+
+  /**
+   * Result of truncate operation.
+   */
+  static class TruncateResult {
+    private final boolean result;
+    private final HdfsFileStatus stat;
+
+    public TruncateResult(boolean result, HdfsFileStatus stat) {
+      this.result = result;
+      this.stat = stat;
+    }
+
+    /**
+     * @return true if client does not need to wait for block recovery,
+     *          false if client needs to wait for block recovery.
+     */
+    boolean getResult() {
+      return result;
+    }
+
+    /**
+     * @return file information.
+     */
+    HdfsFileStatus getFileStatus() {
+      return stat;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3797f9f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index c807fba..ccee1ae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -38,7 +38,6 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.XAttrHelper;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
@@ -49,11 +48,9 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
-import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.util.ByteArray;
 import org.apache.hadoop.hdfs.util.EnumCounters;
 import org.apache.hadoop.security.AccessControlException;
@@ -908,98 +905,6 @@ public class FSDirectory implements Closeable {
   }
 
   /**
-   * FSEditLogLoader implementation.
-   * Unlike FSNamesystem.truncate, this will not schedule block recovery.
-   */
-  void unprotectedTruncate(String src, String clientName, String clientMachine,
-                           long newLength, long mtime, Block truncateBlock)
-      throws UnresolvedLinkException, QuotaExceededException,
-      SnapshotAccessControlException, IOException {
-    INodesInPath iip = getINodesInPath(src, true);
-    INodeFile file = iip.getLastINode().asFile();
-    BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
-    boolean onBlockBoundary =
-        unprotectedTruncate(iip, newLength, collectedBlocks, mtime, null);
-
-    if(! onBlockBoundary) {
-      BlockInfo oldBlock = file.getLastBlock();
-      Block tBlk =
-      getFSNamesystem().prepareFileForTruncate(iip,
-          clientName, clientMachine, file.computeFileSize() - newLength,
-          truncateBlock);
-      assert Block.matchingIdAndGenStamp(tBlk, truncateBlock) &&
-          tBlk.getNumBytes() == truncateBlock.getNumBytes() :
-          "Should be the same block.";
-      if(oldBlock.getBlockId() != tBlk.getBlockId() &&
-         !file.isBlockInLatestSnapshot(oldBlock)) {
-        getBlockManager().removeBlockFromMap(oldBlock);
-      }
-    }
-    assert onBlockBoundary == (truncateBlock == null) :
-      "truncateBlock is null iff on block boundary: " + truncateBlock;
-    getFSNamesystem().removeBlocksAndUpdateSafemodeTotal(collectedBlocks);
-  }
-
-  boolean truncate(INodesInPath iip, long newLength,
-                   BlocksMapUpdateInfo collectedBlocks,
-                   long mtime, QuotaCounts delta)
-      throws IOException {
-    writeLock();
-    try {
-      return unprotectedTruncate(iip, newLength, collectedBlocks, mtime, delta);
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  /**
-   * Truncate has the following properties:
-   * 1.) Any block deletions occur now.
-   * 2.) INode length is truncated now – new clients can only read up to
-   * the truncated length.
-   * 3.) INode will be set to UC and lastBlock set to UNDER_RECOVERY.
-   * 4.) NN will trigger DN truncation recovery and waits for DNs to report.
-   * 5.) File is considered UNDER_RECOVERY until truncation recovery completes.
-   * 6.) Soft and hard Lease expiration require truncation recovery to complete.
-   *
-   * @return true if on the block boundary or false if recovery is need
-   */
-  boolean unprotectedTruncate(INodesInPath iip, long newLength,
-                              BlocksMapUpdateInfo collectedBlocks,
-                              long mtime, QuotaCounts delta) throws IOException {
-    assert hasWriteLock();
-    INodeFile file = iip.getLastINode().asFile();
-    int latestSnapshot = iip.getLatestSnapshotId();
-    file.recordModification(latestSnapshot, true);
-
-    verifyQuotaForTruncate(iip, file, newLength, delta);
-
-    long remainingLength =
-        file.collectBlocksBeyondMax(newLength, collectedBlocks);
-    file.excludeSnapshotBlocks(latestSnapshot, collectedBlocks);
-    file.setModificationTime(mtime);
-    // return whether on a block boundary
-    return (remainingLength - newLength) == 0;
-  }
-
-  private void verifyQuotaForTruncate(INodesInPath iip, INodeFile file,
-      long newLength, QuotaCounts delta) throws QuotaExceededException {
-    if (!getFSNamesystem().isImageLoaded() || shouldSkipQuotaChecks()) {
-      // Do not check quota if edit log is still being processed
-      return;
-    }
-    final BlockStoragePolicy policy = getBlockStoragePolicySuite()
-        .getPolicy(file.getStoragePolicyID());
-    file.computeQuotaDeltaForTruncate(newLength, policy, delta);
-    readLock();
-    try {
-      verifyQuota(iip, iip.length() - 1, delta, null);
-    } finally {
-      readUnlock();
-    }
-  }
-
-  /**
    * This method is always called with writeLock of FSDirectory held.
    */
   public final void addToInodeMap(INode inode) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3797f9f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index df01edd..63ef985 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -901,9 +901,9 @@ public class FSEditLogLoader {
     }
     case OP_TRUNCATE: {
       TruncateOp truncateOp = (TruncateOp) op;
-      fsDir.unprotectedTruncate(truncateOp.src, truncateOp.clientName,
-          truncateOp.clientMachine, truncateOp.newLength, truncateOp.timestamp,
-          truncateOp.truncateBlock);
+      FSDirTruncateOp.unprotectedTruncate(fsNamesys, truncateOp.src,
+          truncateOp.clientName, truncateOp.clientMachine,
+          truncateOp.newLength, truncateOp.timestamp, truncateOp.truncateBlock);
       break;
     }
     case OP_SET_STORAGE_POLICY: {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3797f9f/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 e95007b..7c6d6a1 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
@@ -201,7 +201,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -1831,218 +1830,44 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * block recovery to truncate the last block of the file.
    *
    * @return true if client does not need to wait for block recovery,
-   * false if client needs to wait for block recovery.
+   *         false if client needs to wait for block recovery.
    */
-  boolean truncate(String src, long newLength,
-                   String clientName, String clientMachine,
-                   long mtime)
-      throws IOException, UnresolvedLinkException {
+  boolean truncate(String src, long newLength, String clientName,
+      String clientMachine, long mtime) throws IOException,
+      UnresolvedLinkException {
+
     requireEffectiveLayoutVersionForFeature(Feature.TRUNCATE);
-    boolean ret;
+    final FSDirTruncateOp.TruncateResult r;
     try {
-      ret = truncateInt(src, newLength, clientName, clientMachine, mtime);
+      NameNode.stateChangeLog.debug(
+          "DIR* NameSystem.truncate: src={} newLength={}", src, newLength);
+      if (newLength < 0) {
+        throw new HadoopIllegalArgumentException(
+            "Cannot truncate to a negative file size: " + newLength + ".");
+      }
+      final FSPermissionChecker pc = getPermissionChecker();
+      checkOperation(OperationCategory.WRITE);
+      writeLock();
+      BlocksMapUpdateInfo toRemoveBlocks = new BlocksMapUpdateInfo();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot truncate for " + src);
+        r = FSDirTruncateOp.truncate(this, src, newLength, clientName,
+            clientMachine, mtime, toRemoveBlocks, pc);
+      } finally {
+        writeUnlock();
+      }
+      getEditLog().logSync();
+      if (!toRemoveBlocks.getToDeleteList().isEmpty()) {
+        removeBlocks(toRemoveBlocks);
+        toRemoveBlocks.clear();
+      }
+      logAuditEvent(true, "truncate", src, null, r.getFileStatus());
     } catch (AccessControlException e) {
       logAuditEvent(false, "truncate", src);
       throw e;
     }
-    return ret;
-  }
-
-  boolean truncateInt(String srcArg, long newLength,
-                      String clientName, String clientMachine,
-                      long mtime)
-      throws IOException, UnresolvedLinkException {
-    String src = srcArg;
-    NameNode.stateChangeLog.debug(
-        "DIR* NameSystem.truncate: src={} newLength={}", src, newLength);
-    if (newLength < 0) {
-      throw new HadoopIllegalArgumentException(
-          "Cannot truncate to a negative file size: " + newLength + ".");
-    }
-    HdfsFileStatus stat = null;
-    FSPermissionChecker pc = getPermissionChecker();
-    checkOperation(OperationCategory.WRITE);
-    boolean res;
-    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
-    writeLock();
-    BlocksMapUpdateInfo toRemoveBlocks = new BlocksMapUpdateInfo();
-    try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot truncate for " + src);
-      src = dir.resolvePath(pc, src, pathComponents);
-      res = truncateInternal(src, newLength, clientName,
-          clientMachine, mtime, pc, toRemoveBlocks);
-      stat = dir.getAuditFileInfo(dir.getINodesInPath4Write(src, false));
-    } finally {
-      writeUnlock();
-    }
-    getEditLog().logSync();
-    if (!toRemoveBlocks.getToDeleteList().isEmpty()) {
-      removeBlocks(toRemoveBlocks);
-      toRemoveBlocks.clear();
-    }
-    logAuditEvent(true, "truncate", src, null, stat);
-    return res;
-  }
-
-  /**
-   * Truncate a file to a given size
-   * Update the count at each ancestor directory with quota
-   */
-  boolean truncateInternal(String src, long newLength,
-                           String clientName, String clientMachine,
-                           long mtime, FSPermissionChecker pc,
-                           BlocksMapUpdateInfo toRemoveBlocks)
-      throws IOException, UnresolvedLinkException {
-    assert hasWriteLock();
-    INodesInPath iip = dir.getINodesInPath4Write(src, true);
-    if (isPermissionEnabled) {
-      dir.checkPathAccess(pc, iip, FsAction.WRITE);
-    }
-    INodeFile file = INodeFile.valueOf(iip.getLastINode(), src);
-    final BlockStoragePolicy lpPolicy =
-        blockManager.getStoragePolicy("LAZY_PERSIST");
-
-    if (lpPolicy != null &&
-        lpPolicy.getId() == file.getStoragePolicyID()) {
-      throw new UnsupportedOperationException(
-          "Cannot truncate lazy persist file " + src);
-    }
-
-    // Check if the file is already being truncated with the same length
-    final BlockInfo last = file.getLastBlock();
-    if (last != null && last.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
-      final Block truncateBlock
-          = ((BlockInfoUnderConstruction)last).getTruncateBlock();
-      if (truncateBlock != null) {
-        final long truncateLength = file.computeFileSize(false, false)
-            + truncateBlock.getNumBytes();
-        if (newLength == truncateLength) {
-          return false;
-        }
-      }
-    }
-
-    // Opening an existing file for truncate. May need lease recovery.
-    recoverLeaseInternal(RecoverLeaseOp.TRUNCATE_FILE,
-        iip, src, clientName, clientMachine, false);
-    // Truncate length check.
-    long oldLength = file.computeFileSize();
-    if(oldLength == newLength) {
-      return true;
-    }
-    if(oldLength < newLength) {
-      throw new HadoopIllegalArgumentException(
-          "Cannot truncate to a larger file size. Current size: " + oldLength +
-              ", truncate size: " + newLength + ".");
-    }
-    // Perform INodeFile truncation.
-    final QuotaCounts delta = new QuotaCounts.Builder().build();
-    boolean onBlockBoundary = dir.truncate(iip, newLength, toRemoveBlocks,
-        mtime, delta);
-    Block truncateBlock = null;
-    if(!onBlockBoundary) {
-      // Open file for write, but don't log into edits
-      long lastBlockDelta = file.computeFileSize() - newLength;
-      assert lastBlockDelta > 0 : "delta is 0 only if on block bounday";
-      truncateBlock = prepareFileForTruncate(iip, clientName, clientMachine,
-          lastBlockDelta, null);
-    }
-
-    // update the quota: use the preferred block size for UC block
-    dir.writeLock();
-    try {
-      dir.updateCountNoQuotaCheck(iip, iip.length() - 1, delta);
-    } finally {
-      dir.writeUnlock();
-    }
-
-    getEditLog().logTruncate(src, clientName, clientMachine, newLength, mtime,
-        truncateBlock);
-    return onBlockBoundary;
-  }
-
-  /**
-   * Convert current INode to UnderConstruction.
-   * Recreate lease.
-   * Create new block for the truncated copy.
-   * Schedule truncation of the replicas.
-   *
-   * @return the returned block will be written to editLog and passed back into
-   * this method upon loading.
-   */
-  Block prepareFileForTruncate(INodesInPath iip,
-                               String leaseHolder,
-                               String clientMachine,
-                               long lastBlockDelta,
-                               Block newBlock)
-      throws IOException {
-    INodeFile file = iip.getLastINode().asFile();
-    file.recordModification(iip.getLatestSnapshotId());
-    file.toUnderConstruction(leaseHolder, clientMachine);
-    assert file.isUnderConstruction() : "inode should be under construction.";
-    leaseManager.addLease(
-        file.getFileUnderConstructionFeature().getClientName(), file.getId());
-    boolean shouldRecoverNow = (newBlock == null);
-    BlockInfo oldBlock = file.getLastBlock();
-    boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file, oldBlock);
-    if(newBlock == null) {
-      newBlock = (shouldCopyOnTruncate) ? createNewBlock() :
-          new Block(oldBlock.getBlockId(), oldBlock.getNumBytes(),
-              nextGenerationStamp(blockIdManager.isLegacyBlock(oldBlock)));
-    }
-
-    BlockInfoUnderConstruction truncatedBlockUC;
-    if(shouldCopyOnTruncate) {
-      // Add new truncateBlock into blocksMap and
-      // use oldBlock as a source for copy-on-truncate recovery
-      truncatedBlockUC = new BlockInfoUnderConstructionContiguous(newBlock,
-          file.getPreferredBlockReplication());
-      truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta);
-      truncatedBlockUC.setTruncateBlock(oldBlock);
-      file.setLastBlock(truncatedBlockUC, blockManager.getStorages(oldBlock));
-      getBlockManager().addBlockCollection(truncatedBlockUC, file);
-
-      NameNode.stateChangeLog.debug(
-          "BLOCK* prepareFileForTruncate: Scheduling copy-on-truncate to new" +
-          " size {}  new block {} old block {}", truncatedBlockUC.getNumBytes(),
-          newBlock, truncatedBlockUC.getTruncateBlock());
-    } else {
-      // Use new generation stamp for in-place truncate recovery
-      blockManager.convertLastBlockToUnderConstruction(file, lastBlockDelta);
-      oldBlock = file.getLastBlock();
-      assert !oldBlock.isComplete() : "oldBlock should be under construction";
-      truncatedBlockUC = (BlockInfoUnderConstruction) oldBlock;
-      truncatedBlockUC.setTruncateBlock(new Block(oldBlock));
-      truncatedBlockUC.getTruncateBlock().setNumBytes(
-          oldBlock.getNumBytes() - lastBlockDelta);
-      truncatedBlockUC.getTruncateBlock().setGenerationStamp(
-          newBlock.getGenerationStamp());
-
-      NameNode.stateChangeLog.debug(
-          "BLOCK* prepareFileForTruncate: {} Scheduling in-place block " +
-          "truncate to new size {}",
-          truncatedBlockUC.getTruncateBlock().getNumBytes(), truncatedBlockUC);
-    }
-    if (shouldRecoverNow) {
-      truncatedBlockUC.initializeBlockRecovery(newBlock.getGenerationStamp());
-    }
-
-    return newBlock;
-  }
-
-  /**
-   * Defines if a replica needs to be copied on truncate or
-   * can be truncated in place.
-   */
-  boolean shouldCopyOnTruncate(INodeFile file, BlockInfo blk) {
-    if(!isUpgradeFinalized()) {
-      return true;
-    }
-    if (isRollingUpgrade()) {
-      return true;
-    }
-    return file.isBlockInLatestSnapshot(blk);
+    return r.getResult();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3797f9f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index df920e0..e0f9ad2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -1008,8 +1008,8 @@ public class TestFileTruncate {
     fsn.writeLock();
     try {
       Block oldBlock = file.getLastBlock();
-      Block truncateBlock =
-          fsn.prepareFileForTruncate(iip, client, clientMachine, 1, null);
+      Block truncateBlock = FSDirTruncateOp.prepareFileForTruncate(fsn, iip,
+          client, clientMachine, 1, null);
       // In-place truncate uses old block id with new genStamp.
       assertThat(truncateBlock.getBlockId(),
           is(equalTo(oldBlock.getBlockId())));
@@ -1041,8 +1041,8 @@ public class TestFileTruncate {
     fsn.writeLock();
     try {
       Block oldBlock = file.getLastBlock();
-      Block truncateBlock =
-          fsn.prepareFileForTruncate(iip, client, clientMachine, 1, null);
+      Block truncateBlock = FSDirTruncateOp.prepareFileForTruncate(fsn, iip,
+          client, clientMachine, 1, null);
       // Copy-on-write truncate makes new block with new id and genStamp
       assertThat(truncateBlock.getBlockId(),
           is(not(equalTo(oldBlock.getBlockId()))));


[04/50] hadoop git commit: HDFS-8546. Use try with resources in DataStorage and Storage.

Posted by vv...@apache.org.
HDFS-8546. Use try with resources in DataStorage and Storage.


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

Branch: refs/heads/YARN-2139
Commit: 1403b84b122fb76ef2b085a728b5402c32499c1f
Parents: ff0e5e5
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Jun 25 17:50:32 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Jun 25 17:50:32 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  2 +
 .../hadoop/hdfs/server/common/Storage.java      | 24 ++++-------
 .../hdfs/server/datanode/DataStorage.java       | 45 ++++++--------------
 3 files changed, 25 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1403b84b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index dcedb9f..7b97f41 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -674,6 +674,8 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8665. Fix replication check in DFSTestUtils#waitForReplication. (wang)
 
+    HDFS-8546. Use try with resources in DataStorage and Storage. (wang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1403b84b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
index e6f0999..c630206 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
@@ -709,6 +709,7 @@ public abstract class Storage extends StorageInfo {
       try {
         res = file.getChannel().tryLock();
         if (null == res) {
+          LOG.error("Unable to acquire file lock on path " + lockF.toString());
           throw new OverlappingFileLockException();
         }
         file.write(jvmName.getBytes(Charsets.UTF_8));
@@ -972,35 +973,28 @@ public abstract class Storage extends StorageInfo {
   public void writeProperties(File to, StorageDirectory sd) throws IOException {
     Properties props = new Properties();
     setPropertiesFromFields(props, sd);
-    writeProperties(to, sd, props);
+    writeProperties(to, props);
   }
 
-  public static void writeProperties(File to, StorageDirectory sd,
-      Properties props) throws IOException {
-    RandomAccessFile file = new RandomAccessFile(to, "rws");
-    FileOutputStream out = null;
-    try {
+  public static void writeProperties(File to, Properties props)
+      throws IOException {
+    try (RandomAccessFile file = new RandomAccessFile(to, "rws");
+        FileOutputStream out = new FileOutputStream(file.getFD())) {
       file.seek(0);
-      out = new FileOutputStream(file.getFD());
       /*
-       * If server is interrupted before this line, 
+       * If server is interrupted before this line,
        * the version file will remain unchanged.
        */
       props.store(out, null);
       /*
-       * Now the new fields are flushed to the head of the file, but file 
-       * length can still be larger then required and therefore the file can 
+       * Now the new fields are flushed to the head of the file, but file
+       * length can still be larger then required and therefore the file can
        * contain whole or corrupted fields from its old contents in the end.
        * If server is interrupted here and restarted later these extra fields
        * either should not effect server behavior or should be handled
        * by the server correctly.
        */
       file.setLength(out.getChannel().position());
-    } finally {
-      if (out != null) {
-        out.close();
-      }
-      file.close();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1403b84b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
index 0bd08dd..76789f9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
@@ -44,17 +44,15 @@ import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DiskChecker;
 
 import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.nio.channels.FileLock;
+import java.nio.channels.OverlappingFileLockException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -82,7 +80,6 @@ import java.util.concurrent.Future;
 public class DataStorage extends Storage {
 
   public final static String BLOCK_SUBDIR_PREFIX = "subdir";
-  final static String COPY_FILE_PREFIX = "dncp_";
   final static String STORAGE_DIR_DETACHED = "detach";
   public final static String STORAGE_DIR_RBW = "rbw";
   public final static String STORAGE_DIR_FINALIZED = "finalized";
@@ -614,20 +611,22 @@ public class DataStorage extends Storage {
   @Override
   public boolean isPreUpgradableLayout(StorageDirectory sd) throws IOException {
     File oldF = new File(sd.getRoot(), "storage");
-    if (!oldF.exists())
+    if (!oldF.exists()) {
       return false;
+    }
     // check the layout version inside the storage file
     // Lock and Read old storage file
-    RandomAccessFile oldFile = new RandomAccessFile(oldF, "rws");
-    FileLock oldLock = oldFile.getChannel().tryLock();
-    try {
+    try (RandomAccessFile oldFile = new RandomAccessFile(oldF, "rws");
+      FileLock oldLock = oldFile.getChannel().tryLock()) {
+      if (null == oldLock) {
+        LOG.error("Unable to acquire file lock on path " + oldF.toString());
+        throw new OverlappingFileLockException();
+      }
       oldFile.seek(0);
       int oldVersion = oldFile.readInt();
-      if (oldVersion < LAST_PRE_UPGRADE_LAYOUT_VERSION)
+      if (oldVersion < LAST_PRE_UPGRADE_LAYOUT_VERSION) {
         return false;
-    } finally {
-      oldLock.release();
-      oldFile.close();
+      }
     }
     return true;
   }
@@ -1218,23 +1217,8 @@ public class DataStorage extends Storage {
       return;
     }
     if (!from.isDirectory()) {
-      if (from.getName().startsWith(COPY_FILE_PREFIX)) {
-        FileInputStream in = new FileInputStream(from);
-        try {
-          FileOutputStream out = new FileOutputStream(to);
-          try {
-            IOUtils.copyBytes(in, out, 16*1024);
-            hl.linkStats.countPhysicalFileCopies++;
-          } finally {
-            out.close();
-          }
-        } finally {
-          in.close();
-        }
-      } else {
-        HardLink.createHardLink(from, to);
-        hl.linkStats.countSingleLinks++;
-      }
+      HardLink.createHardLink(from, to);
+      hl.linkStats.countSingleLinks++;
       return;
     }
     // from is a directory
@@ -1285,8 +1269,7 @@ public class DataStorage extends Storage {
     String[] otherNames = from.list(new java.io.FilenameFilter() {
         @Override
         public boolean accept(File dir, String name) {
-          return name.startsWith(BLOCK_SUBDIR_PREFIX) 
-            || name.startsWith(COPY_FILE_PREFIX);
+          return name.startsWith(BLOCK_SUBDIR_PREFIX);
         }
       });
     for(int i = 0; i < otherNames.length; i++)


[21/50] hadoop git commit: HDFS-7390. Provide JMX metrics per storage type. (Benoy Antony)

Posted by vv...@apache.org.
HDFS-7390. Provide JMX metrics per storage type. (Benoy Antony)


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

Branch: refs/heads/YARN-2139
Commit: d3fed8e653ed9e18d3a29a11c4b24a628ac770bb
Parents: fde20ff
Author: Benoy Antony <be...@apache.org>
Authored: Mon Jun 29 11:00:22 2015 -0700
Committer: Benoy Antony <be...@apache.org>
Committed: Mon Jun 29 11:00:22 2015 -0700

----------------------------------------------------------------------
 .../server/blockmanagement/BlockManager.java    |  15 +-
 .../blockmanagement/BlockStatsMXBean.java       |  36 +++++
 .../blockmanagement/DatanodeStatistics.java     |   6 +
 .../blockmanagement/HeartbeatManager.java       |  96 +++++++++++-
 .../blockmanagement/StorageTypeStats.java       | 115 +++++++++++++++
 .../blockmanagement/TestBlockStatsMXBean.java   | 146 +++++++++++++++++++
 6 files changed, 412 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fed8e6/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 5bd4980..0b60a97 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -39,6 +39,8 @@ import java.util.TreeSet;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicLong;
 
+import javax.management.ObjectName;
+
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
@@ -85,6 +87,7 @@ import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
+import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Daemon;
@@ -94,6 +97,7 @@ import org.apache.hadoop.util.Time;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Sets;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -101,7 +105,7 @@ import org.slf4j.LoggerFactory;
  * Keeps information related to the blocks stored in the Hadoop cluster.
  */
 @InterfaceAudience.Private
-public class BlockManager {
+public class BlockManager implements BlockStatsMXBean {
 
   public static final Logger LOG = LoggerFactory.getLogger(BlockManager.class);
   public static final Logger blockLog = NameNode.blockStateChangeLog;
@@ -129,6 +133,7 @@ public class BlockManager {
   private final AtomicLong postponedMisreplicatedBlocksCount = new AtomicLong(0L);
   private final long startupDelayBlockDeletionInMs;
   private final BlockReportLeaseManager blockReportLeaseManager;
+  private ObjectName mxBeanName;
 
   /** Used by metrics */
   public long getPendingReplicationBlocksCount() {
@@ -468,6 +473,7 @@ public class BlockManager {
     pendingReplications.start();
     datanodeManager.activate(conf);
     this.replicationThread.start();
+    mxBeanName = MBeans.register("NameNode", "BlockStats", this);
   }
 
   public void close() {
@@ -3944,6 +3950,8 @@ public class BlockManager {
   public void shutdown() {
     stopReplicationInitializer();
     blocksMap.close();
+    MBeans.unregister(mxBeanName);
+    mxBeanName = null;
   }
   
   public void clear() {
@@ -3954,4 +3962,9 @@ public class BlockManager {
   public BlockReportLeaseManager getBlockReportLeaseManager() {
     return blockReportLeaseManager;
   }
+
+  @Override // BlockStatsMXBean
+  public Map<StorageType, StorageTypeStats> getStorageTypeStats() {
+    return  datanodeManager.getDatanodeStatistics().getStorageTypeStats();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fed8e6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStatsMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStatsMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStatsMXBean.java
new file mode 100644
index 0000000..f22c537
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStatsMXBean.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.hdfs.server.blockmanagement;
+
+import java.util.Map;
+
+import org.apache.hadoop.fs.StorageType;
+
+/**
+ * This is an interface used to retrieve statistic information related to
+ * block management.
+ */
+public interface BlockStatsMXBean {
+
+  /**
+   * The statistics of storage types.
+   *
+   * @return get storage statistics per storage type
+   */
+  Map<StorageType, StorageTypeStats> getStorageTypeStats();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fed8e6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStatistics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStatistics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStatistics.java
index c9bc3e5..33eca2e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStatistics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStatistics.java
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import java.util.Map;
+
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 
 /** Datanode statistics */
@@ -71,4 +74,7 @@ public interface DatanodeStatistics {
 
   /** @return the expired heartbeats */
   public int getExpiredHeartbeats();
+
+  /** @return Storage Tier statistics*/
+  Map<StorageType, StorageTypeStats> getStorageTypeStats();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fed8e6/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 9017fe1..cc9365d 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
@@ -18,9 +18,15 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
@@ -189,6 +195,11 @@ class HeartbeatManager implements DatanodeStatistics {
     return stats.expiredHeartbeats;
   }
 
+  @Override
+  public  Map<StorageType, StorageTypeStats> getStorageTypeStats() {
+    return stats.statsMap.get();
+  }
+
   synchronized void register(final DatanodeDescriptor d) {
     if (!d.isAlive) {
       addDatanode(d);
@@ -393,6 +404,9 @@ class HeartbeatManager implements DatanodeStatistics {
    * For decommissioning/decommissioned nodes, only used capacity is counted.
    */
   private static class Stats {
+
+    private final StorageTypeStatsMap statsMap = new StorageTypeStatsMap();
+
     private long capacityTotal = 0L;
     private long capacityUsed = 0L;
     private long capacityRemaining = 0L;
@@ -420,6 +434,14 @@ class HeartbeatManager implements DatanodeStatistics {
       }
       cacheCapacity += node.getCacheCapacity();
       cacheUsed += node.getCacheUsed();
+      Set<StorageType> storageTypes = new HashSet<>();
+      for (DatanodeStorageInfo storageInfo : node.getStorageInfos()) {
+        statsMap.addStorage(storageInfo, node);
+        storageTypes.add(storageInfo.getStorageType());
+      }
+      for (StorageType storageType : storageTypes) {
+        statsMap.addNode(storageType, node);
+      }
     }
 
     private void subtract(final DatanodeDescriptor node) {
@@ -436,6 +458,14 @@ class HeartbeatManager implements DatanodeStatistics {
       }
       cacheCapacity -= node.getCacheCapacity();
       cacheUsed -= node.getCacheUsed();
+      Set<StorageType> storageTypes = new HashSet<>();
+      for (DatanodeStorageInfo storageInfo : node.getStorageInfos()) {
+        statsMap.subtractStorage(storageInfo, node);
+        storageTypes.add(storageInfo.getStorageType());
+      }
+      for (StorageType storageType : storageTypes) {
+        statsMap.subtractNode(storageType, node);
+      }
     }
     
     /** Increment expired heartbeat counter. */
@@ -443,5 +473,69 @@ class HeartbeatManager implements DatanodeStatistics {
       expiredHeartbeats++;
     }
   }
-}
 
+  /** StorageType specific statistics.
+   * For decommissioning/decommissioned nodes, only used capacity is counted.
+   */
+
+  static final class StorageTypeStatsMap {
+
+    private Map<StorageType, StorageTypeStats> storageTypeStatsMap =
+          new IdentityHashMap<>();
+
+    private StorageTypeStatsMap() {}
+
+    private StorageTypeStatsMap(StorageTypeStatsMap other) {
+      storageTypeStatsMap =
+          new IdentityHashMap<>(other.storageTypeStatsMap);
+      for (Map.Entry<StorageType, StorageTypeStats> entry :
+          storageTypeStatsMap.entrySet()) {
+        entry.setValue(new StorageTypeStats(entry.getValue()));
+      }
+    }
+
+    private Map<StorageType, StorageTypeStats> get() {
+      return Collections.unmodifiableMap(storageTypeStatsMap);
+    }
+
+    private void addNode(StorageType storageType,
+        final DatanodeDescriptor node) {
+      StorageTypeStats storageTypeStats =
+          storageTypeStatsMap.get(storageType);
+      if (storageTypeStats == null) {
+        storageTypeStats = new StorageTypeStats();
+        storageTypeStatsMap.put(storageType, storageTypeStats);
+      }
+      storageTypeStats.addNode(node);
+    }
+
+    private void addStorage(final DatanodeStorageInfo info,
+        final DatanodeDescriptor node) {
+      StorageTypeStats storageTypeStats =
+          storageTypeStatsMap.get(info.getStorageType());
+      if (storageTypeStats == null) {
+        storageTypeStats = new StorageTypeStats();
+        storageTypeStatsMap.put(info.getStorageType(), storageTypeStats);
+      }
+      storageTypeStats.addStorage(info, node);
+    }
+
+    private void subtractStorage(final DatanodeStorageInfo info,
+        final DatanodeDescriptor node) {
+      StorageTypeStats storageTypeStats =
+          storageTypeStatsMap.get(info.getStorageType());
+      if (storageTypeStats != null) {
+        storageTypeStats.subtractStorage(info, node);
+      }
+    }
+
+    private void subtractNode(StorageType storageType,
+        final DatanodeDescriptor node) {
+      StorageTypeStats storageTypeStats =
+          storageTypeStatsMap.get(storageType);
+      if (storageTypeStats != null) {
+        storageTypeStats.subtractNode(node);
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fed8e6/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
new file mode 100644
index 0000000..45dcc8d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/StorageTypeStats.java
@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import java.beans.ConstructorProperties;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Statistics per StorageType.
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class StorageTypeStats {
+  private long capacityTotal = 0L;
+  private long capacityUsed = 0L;
+  private long capacityRemaining = 0L;
+  private long blockPoolUsed = 0L;
+  private int nodesInService = 0;
+
+  @ConstructorProperties({"capacityTotal",
+      "capacityUsed", "capacityRemaining",  "blockPoolUsed", "nodesInService"})
+  public StorageTypeStats(long capacityTotal, long capacityUsed,
+      long capacityRemaining, long blockPoolUsed, int nodesInService) {
+    this.capacityTotal = capacityTotal;
+    this.capacityUsed = capacityUsed;
+    this.capacityRemaining = capacityRemaining;
+    this.blockPoolUsed = blockPoolUsed;
+    this.nodesInService = nodesInService;
+  }
+
+  public long getCapacityTotal() {
+    return capacityTotal;
+  }
+
+  public long getCapacityUsed() {
+    return capacityUsed;
+  }
+
+  public long getCapacityRemaining() {
+    return capacityRemaining;
+  }
+
+  public long getBlockPoolUsed() {
+    return blockPoolUsed;
+  }
+
+  public int getNodesInService() {
+    return nodesInService;
+  }
+
+  StorageTypeStats() {}
+
+  StorageTypeStats(StorageTypeStats other) {
+    capacityTotal = other.capacityTotal;
+    capacityUsed = other.capacityUsed;
+    capacityRemaining = other.capacityRemaining;
+    blockPoolUsed = other.blockPoolUsed;
+    nodesInService = other.nodesInService;
+  }
+
+  void addStorage(final DatanodeStorageInfo info,
+      final DatanodeDescriptor node) {
+    capacityUsed += info.getDfsUsed();
+    blockPoolUsed += info.getBlockPoolUsed();
+    if (!(node.isDecommissionInProgress() || node.isDecommissioned())) {
+      capacityTotal += info.getCapacity();
+      capacityRemaining += info.getRemaining();
+    } else {
+      capacityTotal += info.getDfsUsed();
+    }
+  }
+
+  void addNode(final DatanodeDescriptor node) {
+    if (!(node.isDecommissionInProgress() || node.isDecommissioned())) {
+      nodesInService++;
+    }
+  }
+
+  void subtractStorage(final DatanodeStorageInfo info,
+      final DatanodeDescriptor node) {
+    capacityUsed -= info.getDfsUsed();
+    blockPoolUsed -= info.getBlockPoolUsed();
+    if (!(node.isDecommissionInProgress() || node.isDecommissioned())) {
+      capacityTotal -= info.getCapacity();
+      capacityRemaining -= info.getRemaining();
+    } else {
+      capacityTotal -= info.getDfsUsed();
+    }
+  }
+
+  void subtractNode(final DatanodeDescriptor node) {
+    if (!(node.isDecommissionInProgress() || node.isDecommissioned())) {
+      nodesInService--;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fed8e6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
new file mode 100644
index 0000000..43d983d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
@@ -0,0 +1,146 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mortbay.util.ajax.JSON;
+
+/**
+ * Class for testing {@link BlockStatsMXBean} implementation
+ */
+public class TestBlockStatsMXBean {
+
+  private MiniDFSCluster cluster;
+
+  @Before
+  public void setup() throws IOException {
+    HdfsConfiguration conf = new HdfsConfiguration();
+    cluster = null;
+    StorageType[][] types = new StorageType[6][];
+    for (int i=0; i<3; i++) {
+      types[i] = new StorageType[] {StorageType.RAM_DISK, StorageType.DISK};
+    }
+    for (int i=3; i< 5; i++) {
+      types[i] = new StorageType[] {StorageType.RAM_DISK, StorageType.ARCHIVE};
+    }
+    types[5] = new StorageType[] {StorageType.RAM_DISK, StorageType.ARCHIVE,
+        StorageType.ARCHIVE};
+
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(6).
+        storageTypes(types).storagesPerDatanode(3).build();
+    cluster.waitActive();
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testStorageTypeStats() throws Exception {
+    Map<StorageType, StorageTypeStats> storageTypeStatsMap =
+        cluster.getNamesystem().getBlockManager().getStorageTypeStats();
+    assertTrue(storageTypeStatsMap.containsKey(StorageType.RAM_DISK));
+    assertTrue(storageTypeStatsMap.containsKey(StorageType.DISK));
+    assertTrue(storageTypeStatsMap.containsKey(StorageType.ARCHIVE));
+
+    StorageTypeStats storageTypeStats =
+        storageTypeStatsMap.get(StorageType.RAM_DISK);
+    assertEquals(6, storageTypeStats.getNodesInService());
+
+    storageTypeStats = storageTypeStatsMap.get(StorageType.DISK);
+    assertEquals(3, storageTypeStats.getNodesInService());
+
+    storageTypeStats = storageTypeStatsMap.get(StorageType.ARCHIVE);
+    assertEquals(3, storageTypeStats.getNodesInService());
+  }
+
+  protected static String readOutput(URL url) throws IOException {
+    StringBuilder out = new StringBuilder();
+    InputStream in = url.openConnection().getInputStream();
+    byte[] buffer = new byte[64 * 1024];
+    int len = in.read(buffer);
+    while (len > 0) {
+      out.append(new String(buffer, 0, len));
+      len = in.read(buffer);
+    }
+    return out.toString();
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testStorageTypeStatsJMX() throws Exception {
+    URL baseUrl = new URL (cluster.getHttpUri(0));
+    String result = readOutput(new URL(baseUrl, "/jmx"));
+    System.out.println(result);
+
+    Map<String, Object> stat = (Map<String, Object>) JSON.parse(result);
+    Object[] beans =(Object[]) stat.get("beans");
+    Map<String, Object> blockStats  = null;
+    for (Object bean : beans) {
+      Map<String, Object> map = (Map<String, Object>) bean;
+      if (map.get("name").equals("Hadoop:service=NameNode,name=BlockStats")) {
+        blockStats = map;
+      }
+    }
+    assertNotNull(blockStats);
+    Object[] storageTypeStatsList =
+        (Object[])blockStats.get("StorageTypeStats");
+    assertNotNull(storageTypeStatsList);
+    assertEquals (3, storageTypeStatsList.length);
+
+    Set<String> typesPresent = new HashSet<> ();
+    for (Object obj : storageTypeStatsList) {
+      Map<String, Object> entry = (Map<String, Object>)obj;
+      String storageType = (String)entry.get("key");
+      Map<String,Object> storageTypeStats = (Map<String,Object>)entry.get("value");
+      typesPresent.add(storageType);
+      if (storageType.equals("ARCHIVE") || storageType.equals("DISK") ) {
+        assertEquals(3l, storageTypeStats.get("nodesInService"));
+      } else if (storageType.equals("RAM_DISK")) {
+        assertEquals(6l, storageTypeStats.get("nodesInService"));
+      }
+      else {
+        fail();
+      }
+    }
+
+    assertTrue(typesPresent.contains("ARCHIVE"));
+    assertTrue(typesPresent.contains("DISK"));
+    assertTrue(typesPresent.contains("RAM_DISK"));
+  }
+}


[43/50] hadoop git commit: YARN-3827. Migrate YARN native build to new CMake framework (Alan Burlison via Colin P. McCabe)

Posted by vv...@apache.org.
YARN-3827. Migrate YARN native build to new CMake framework (Alan Burlison via Colin P. McCabe)


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

Branch: refs/heads/YARN-2139
Commit: d0cc0380b57db5fdeb41775bb9ca42dac65928b8
Parents: 9c63825
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Tue Jun 30 16:24:19 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Tue Jun 30 16:24:19 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +++
 .../src/CMakeLists.txt                          | 23 ++++++++++----------
 2 files changed, 14 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0cc0380/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 94bc8fc..5e9243d 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -317,6 +317,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3834. Scrub debug logging of tokens during resource localization.
     (Chris Nauroth via xgong)
 
+    YARN-3827. Migrate YARN native build to new CMake framework (Alan Burlison
+    via Colin P. McCabe)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0cc0380/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
index f691a9c..d4d6ae1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
@@ -16,25 +16,24 @@
 
 cmake_minimum_required(VERSION 2.6 FATAL_ERROR)
 
-set(CMAKE_BUILD_TYPE, Release)
+list(APPEND CMAKE_MODULE_PATH ${CMAKE_SOURCE_DIR}/../../../../../hadoop-common-project/hadoop-common)
+include(HadoopCommon)
 
-include(../../../../../hadoop-common-project/hadoop-common/src/JNIFlags.cmake NO_POLICY_SCOPE)
+# Note: can't use -D_FILE_OFFSET_BITS=64, see MAPREDUCE-4258
+string(REPLACE "-D_FILE_OFFSET_BITS=64" "" CMAKE_C_FLAGS "${CMAKE_C_FLAGS}")
+string(REPLACE "-D_FILE_OFFSET_BITS=64" "" CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}")
 
 include(CheckFunctionExists)
-CHECK_FUNCTION_EXISTS(fcloseall HAVE_FCLOSEALL)
+check_function_exists(fcloseall HAVE_FCLOSEALL)
 
 function(output_directory TGT DIR)
-    SET_TARGET_PROPERTIES(${TGT} PROPERTIES
+    set_target_properties(${TGT} PROPERTIES
         RUNTIME_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-    SET_TARGET_PROPERTIES(${TGT} PROPERTIES
+    set_target_properties(${TGT} PROPERTIES
         ARCHIVE_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-    SET_TARGET_PROPERTIES(${TGT} PROPERTIES
+    set_target_properties(${TGT} PROPERTIES
         LIBRARY_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-endfunction(output_directory TGT DIR)
-
-set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -g -Wall -O2 -D_GNU_SOURCE")
-# note: can't enable -D_LARGEFILE: see MAPREDUCE-4258
-set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_REENTRANT")
+endfunction()
 
 include_directories(
     ${CMAKE_CURRENT_SOURCE_DIR}
@@ -42,7 +41,7 @@ include_directories(
     main/native/container-executor
     main/native/container-executor/impl
 )
-CONFIGURE_FILE(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h)
+configure_file(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h)
 
 add_library(container
     main/native/container-executor/impl/configuration.c


[02/50] hadoop git commit: MAPREDUCE-6413. TestLocalJobSubmission is failing with unknown host. Contributed by zhihai xu

Posted by vv...@apache.org.
MAPREDUCE-6413. TestLocalJobSubmission is failing with unknown host. Contributed by zhihai xu


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

Branch: refs/heads/YARN-2139
Commit: aa5b15b03be61ebb76a226e0de485d5228c8e3d0
Parents: 67a62da
Author: Jason Lowe <jl...@apache.org>
Authored: Thu Jun 25 19:50:07 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Thu Jun 25 19:50:07 2015 +0000

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa5b15b0/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 6c65032..4df3b44 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -509,6 +509,9 @@ Release 2.8.0 - UNRELEASED
     too early (Brahma Reddy Battula, Akira AJISAKA, and Gera Shegalov via
     jlowe)
 
+    MAPREDUCE-6413. TestLocalJobSubmission is failing with unknown host
+    (zhihai xu via jlowe)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa5b15b0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLocalJobSubmission.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLocalJobSubmission.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLocalJobSubmission.java
index e06551a..d73ee4b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLocalJobSubmission.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLocalJobSubmission.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
+import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.SleepJob;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.After;
@@ -63,7 +64,8 @@ public class TestLocalJobSubmission {
     Path jarPath = makeJar(new Path(TEST_ROOT_DIR, "test.jar"));
 
     Configuration conf = new Configuration();
-    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "hdfs://testcluster");
+    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "hdfs://localhost:9000");
+    conf.set(MRConfig.FRAMEWORK_NAME, "local");
     final String[] args = {
         "-jt" , "local", "-libjars", jarPath.toString(),
         "-m", "1", "-r", "1", "-mt", "1", "-rt", "1"


[14/50] hadoop git commit: HADOOP-12009 Clarify FileSystem.listStatus() sorting order & fix FileSystemContractBaseTest:testListStatus. (J.Andreina via stevel)

Posted by vv...@apache.org.
HADOOP-12009 Clarify FileSystem.listStatus() sorting order & fix  FileSystemContractBaseTest:testListStatus. (J.Andreina via stevel)


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

Branch: refs/heads/YARN-2139
Commit: 3dfa8161f9412bcb040f3c29c471344f25f24337
Parents: b543d1a
Author: Steve Loughran <st...@apache.org>
Authored: Sun Jun 28 19:13:48 2015 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Sun Jun 28 19:14:00 2015 +0100

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt    |  4 ++++
 .../main/java/org/apache/hadoop/fs/FileSystem.java | 17 ++++++++++++++++-
 .../src/site/markdown/filesystem/filesystem.md     |  4 ++++
 .../hadoop/fs/FileSystemContractBaseTest.java      | 11 ++++++++---
 4 files changed, 32 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3dfa8161/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 18c73c0..132e4c0 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -661,6 +661,10 @@ Release 2.8.0 - UNRELEASED
 
     HADOOP-11958. MetricsSystemImpl fails to show backtrace when an error
     occurs (Jason Lowe via jeagles)
+
+    HADOOP-12009 Clarify FileSystem.listStatus() sorting order & fix
+    FileSystemContractBaseTest:testListStatus. (J.Andreina via stevel)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3dfa8161/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
index 01d4b27..3f9e3bd 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
@@ -1498,7 +1498,9 @@ public abstract class FileSystem extends Configured implements Closeable {
   /**
    * List the statuses of the files/directories in the given path if the path is
    * a directory.
-   * 
+   * <p>
+   * Does not guarantee to return the List of files/directories status in a
+   * sorted order.
    * @param f given path
    * @return the statuses of the files/directories in the given patch
    * @throws FileNotFoundException when the path does not exist;
@@ -1540,6 +1542,9 @@ public abstract class FileSystem extends Configured implements Closeable {
   /**
    * Filter files/directories in the given path using the user-supplied path
    * filter.
+   * <p>
+   * Does not guarantee to return the List of files/directories status in a
+   * sorted order.
    * 
    * @param f
    *          a path name
@@ -1560,6 +1565,9 @@ public abstract class FileSystem extends Configured implements Closeable {
   /**
    * Filter files/directories in the given list of paths using default
    * path filter.
+   * <p>
+   * Does not guarantee to return the List of files/directories status in a
+   * sorted order.
    * 
    * @param files
    *          a list of paths
@@ -1576,6 +1584,9 @@ public abstract class FileSystem extends Configured implements Closeable {
   /**
    * Filter files/directories in the given list of paths using user-supplied
    * path filter.
+   * <p>
+   * Does not guarantee to return the List of files/directories status in a
+   * sorted order.
    * 
    * @param files
    *          a list of paths
@@ -1736,6 +1747,8 @@ public abstract class FileSystem extends Configured implements Closeable {
    * while consuming the entries. Each file system implementation should
    * override this method and provide a more efficient implementation, if
    * possible. 
+   * Does not guarantee to return the iterator that traverses statuses
+   * of the files in a sorted order.
    *
    * @param p target path
    * @return remote iterator
@@ -1763,6 +1776,8 @@ public abstract class FileSystem extends Configured implements Closeable {
 
   /**
    * List the statuses and block locations of the files in the given path.
+   * Does not guarantee to return the iterator that traverses statuses
+   * of the files in a sorted order.
    * 
    * If the path is a directory, 
    *   if recursive is false, returns files in the directory;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3dfa8161/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 84e3755..f323374 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
@@ -183,6 +183,10 @@ to the same path:
     forall fs in listStatus(Path) :
       fs == getFileStatus(fs.path)
 
+**Ordering of results**: there is no guarantee of ordering of the listed entries.
+While HDFS currently returns an alphanumerically sorted list, neither the Posix `readdir()`
+nor Java's `File.listFiles()` API calls define any ordering of returned values. Applications
+which require a uniform sort order on the results must perform the sorting themselves.
 
 ### Atomicity and Consistency
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3dfa8161/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
index 2ca81e9..495af00 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.ArrayList;
 
 import junit.framework.TestCase;
 
@@ -224,9 +225,13 @@ public abstract class FileSystemContractBaseTest extends TestCase {
 
     paths = fs.listStatus(path("/test/hadoop"));
     assertEquals(3, paths.length);
-    assertEquals(path("/test/hadoop/a"), paths[0].getPath());
-    assertEquals(path("/test/hadoop/b"), paths[1].getPath());
-    assertEquals(path("/test/hadoop/c"), paths[2].getPath());
+    ArrayList<String> list = new ArrayList<String>();
+    for (FileStatus fileState : paths) {
+      list.add(fileState.getPath().toString());
+    }
+    assertTrue(list.contains(path("/test/hadoop/a")));
+    assertTrue(list.contains(path("/test/hadoop/b")));
+    assertTrue(list.contains(path("/test/hadoop/c")));
 
     paths = fs.listStatus(path("/test/hadoop/a"));
     assertEquals(0, paths.length);


[38/50] hadoop git commit: MAPREDUCE-6384. Add the last reporting reducer info for too many fetch failure diagnostics. Contributed by Chang Li

Posted by vv...@apache.org.
MAPREDUCE-6384. Add the last reporting reducer info for too many fetch failure diagnostics. Contributed by Chang Li


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

Branch: refs/heads/YARN-2139
Commit: b6ba56457c6b01dae795c11d587c3fe3855ee707
Parents: 147e020
Author: Jason Lowe <jl...@apache.org>
Authored: Tue Jun 30 21:22:30 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Tue Jun 30 21:22:30 2015 +0000

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |   3 +
 .../event/JobTaskAttemptFetchFailureEvent.java  |   9 +-
 .../TaskAttemptTooManyFetchFailureEvent.java    |  50 +++++++
 .../mapreduce/v2/app/job/impl/JobImpl.java      |   7 +-
 .../v2/app/job/impl/TaskAttemptImpl.java        |  15 +-
 .../mapreduce/v2/app/TestFetchFailure.java      |  31 ++--
 .../v2/app/job/impl/TestTaskAttempt.java        | 143 ++++++++++---------
 7 files changed, 168 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6ba5645/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 785fce8..5a4d826 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -364,6 +364,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6408. Queue name and user name should be printed on the job page.
     (Siqi Li via gera)
 
+    MAPREDUCE-6384. Add the last reporting reducer info for too many fetch
+    failure diagnostics (Chang Li via jlowe)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6ba5645/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/JobTaskAttemptFetchFailureEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/JobTaskAttemptFetchFailureEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/JobTaskAttemptFetchFailureEvent.java
index 37e2034..787711c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/JobTaskAttemptFetchFailureEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/JobTaskAttemptFetchFailureEvent.java
@@ -28,13 +28,15 @@ public class JobTaskAttemptFetchFailureEvent extends JobEvent {
 
   private final TaskAttemptId reduce;
   private final List<TaskAttemptId> maps;
+  private final String hostname;
 
   public JobTaskAttemptFetchFailureEvent(TaskAttemptId reduce, 
-      List<TaskAttemptId> maps) {
-    super(reduce.getTaskId().getJobId(), 
+      List<TaskAttemptId> maps, String host) {
+    super(reduce.getTaskId().getJobId(),
         JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE);
     this.reduce = reduce;
     this.maps = maps;
+    this.hostname = host;
   }
 
   public List<TaskAttemptId> getMaps() {
@@ -45,4 +47,7 @@ public class JobTaskAttemptFetchFailureEvent extends JobEvent {
     return reduce;
   }
 
+  public String getHost() {
+    return hostname;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6ba5645/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskAttemptTooManyFetchFailureEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskAttemptTooManyFetchFailureEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskAttemptTooManyFetchFailureEvent.java
new file mode 100644
index 0000000..662e712
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskAttemptTooManyFetchFailureEvent.java
@@ -0,0 +1,50 @@
+/**
+* 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.mapreduce.v2.app.job.event;
+
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+
+/**
+ * TaskAttemptTooManyFetchFailureEvent is used for TA_TOO_MANY_FETCH_FAILURE.
+ */
+public class TaskAttemptTooManyFetchFailureEvent extends TaskAttemptEvent {
+  private TaskAttemptId reduceID;
+  private String  reduceHostname;
+
+  /**
+   * Create a new TaskAttemptTooManyFetchFailureEvent.
+   * @param attemptId the id of the mapper task attempt
+   * @param reduceId the id of the reporting reduce task attempt.
+   * @param reduceHost the hostname of the reporting reduce task attempt.
+   */
+  public TaskAttemptTooManyFetchFailureEvent(TaskAttemptId attemptId,
+      TaskAttemptId reduceId, String reduceHost) {
+      super(attemptId, TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURE);
+    this.reduceID = reduceId;
+    this.reduceHostname = reduceHost;
+  }
+
+  public TaskAttemptId getReduceId() {
+    return reduceID;
+  }
+
+  public String getReduceHost() {
+    return reduceHostname;
+  }  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6ba5645/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
index 6e9f13c..2c48019 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
@@ -103,9 +103,8 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskAttemptCompletedEvent
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskAttemptFetchFailureEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobUpdatedNodesEvent;
-import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
-import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptKillEvent;
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptTooManyFetchFailureEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskRecoverEvent;
@@ -1914,8 +1913,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
             && failureRate >= job.getMaxAllowedFetchFailuresFraction()) {
           LOG.info("Too many fetch-failures for output of task attempt: " + 
               mapId + " ... raising fetch failure to map");
-          job.eventHandler.handle(new TaskAttemptEvent(mapId, 
-              TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURE));
+          job.eventHandler.handle(new TaskAttemptTooManyFetchFailureEvent(mapId,
+              fetchfailureEvent.getReduce(), fetchfailureEvent.getHost()));
           job.fetchFailuresMapping.remove(mapId);
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6ba5645/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
index 3055a25..3fa42fe 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
@@ -95,6 +95,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptKillEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptRecoverEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptTooManyFetchFailureEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskTAttemptEvent;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher;
@@ -1916,12 +1917,17 @@ public abstract class TaskAttemptImpl implements
     @SuppressWarnings("unchecked")
     @Override
     public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) {
+      TaskAttemptTooManyFetchFailureEvent fetchFailureEvent =
+          (TaskAttemptTooManyFetchFailureEvent) event;
       // too many fetch failure can only happen for map tasks
       Preconditions
           .checkArgument(taskAttempt.getID().getTaskId().getTaskType() == TaskType.MAP);
       //add to diagnostic
-      taskAttempt.addDiagnosticInfo("Too Many fetch failures.Failing the attempt");
-      
+      taskAttempt.addDiagnosticInfo("Too many fetch failures."
+          + " Failing the attempt. Last failure reported by " +
+          fetchFailureEvent.getReduceId() +
+          " from host " + fetchFailureEvent.getReduceHost());
+
       if (taskAttempt.getLaunchTime() != 0) {
         taskAttempt.eventHandler
             .handle(createJobCounterUpdateEventTAFailed(taskAttempt, true));
@@ -2225,8 +2231,11 @@ public abstract class TaskAttemptImpl implements
       //this only will happen in reduce attempt type
       if (taskAttempt.reportedStatus.fetchFailedMaps != null && 
           taskAttempt.reportedStatus.fetchFailedMaps.size() > 0) {
+        String hostname = taskAttempt.container == null ? "UNKNOWN"
+            : taskAttempt.container.getNodeId().getHost();
         taskAttempt.eventHandler.handle(new JobTaskAttemptFetchFailureEvent(
-            taskAttempt.attemptId, taskAttempt.reportedStatus.fetchFailedMaps));
+            taskAttempt.attemptId, taskAttempt.reportedStatus.fetchFailedMaps,
+                hostname));
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6ba5645/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java
index 4e4e2e7..8d25079 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java
@@ -94,10 +94,10 @@ public class TestFetchFailure {
     app.waitForState(reduceAttempt, TaskAttemptState.RUNNING);
     
     //send 3 fetch failures from reduce to trigger map re execution
-    sendFetchFailure(app, reduceAttempt, mapAttempt1);
-    sendFetchFailure(app, reduceAttempt, mapAttempt1);
-    sendFetchFailure(app, reduceAttempt, mapAttempt1);
-    
+    sendFetchFailure(app, reduceAttempt, mapAttempt1, "host");
+    sendFetchFailure(app, reduceAttempt, mapAttempt1, "host");
+    sendFetchFailure(app, reduceAttempt, mapAttempt1, "host");
+
     //wait for map Task state move back to RUNNING
     app.waitForState(mapTask, TaskState.RUNNING);
     
@@ -215,9 +215,9 @@ public class TestFetchFailure {
     app.waitForState(reduceAttempt, TaskAttemptState.RUNNING);
 
     //send 3 fetch failures from reduce to trigger map re execution
-    sendFetchFailure(app, reduceAttempt, mapAttempt1);
-    sendFetchFailure(app, reduceAttempt, mapAttempt1);
-    sendFetchFailure(app, reduceAttempt, mapAttempt1);
+    sendFetchFailure(app, reduceAttempt, mapAttempt1, "host");
+    sendFetchFailure(app, reduceAttempt, mapAttempt1, "host");
+    sendFetchFailure(app, reduceAttempt, mapAttempt1, "host");
 
     //wait for map Task state move back to RUNNING
     app.waitForState(mapTask, TaskState.RUNNING);
@@ -324,8 +324,8 @@ public class TestFetchFailure {
     updateStatus(app, reduceAttempt3, Phase.SHUFFLE);
 
     //send 2 fetch failures from reduce to prepare for map re execution
-    sendFetchFailure(app, reduceAttempt, mapAttempt1);
-    sendFetchFailure(app, reduceAttempt, mapAttempt1);
+    sendFetchFailure(app, reduceAttempt, mapAttempt1, "host1");
+    sendFetchFailure(app, reduceAttempt2, mapAttempt1, "host2");
 
     //We should not re-launch the map task yet
     assertEquals(TaskState.SUCCEEDED, mapTask.getState());
@@ -333,7 +333,7 @@ public class TestFetchFailure {
     updateStatus(app, reduceAttempt3, Phase.REDUCE);
 
     //send 3rd fetch failures from reduce to trigger map re execution
-    sendFetchFailure(app, reduceAttempt, mapAttempt1);
+    sendFetchFailure(app, reduceAttempt3, mapAttempt1, "host3");
 
     //wait for map Task state move back to RUNNING
     app.waitForState(mapTask, TaskState.RUNNING);
@@ -342,6 +342,11 @@ public class TestFetchFailure {
     Assert.assertEquals("Map TaskAttempt state not correct",
         TaskAttemptState.FAILED, mapAttempt1.getState());
 
+    Assert.assertEquals(mapAttempt1.getDiagnostics().get(0),
+            "Too many fetch failures. Failing the attempt. "
+            + "Last failure reported by "
+            + reduceAttempt3.getID().toString() + " from host host3");
+
     Assert.assertEquals("Num attempts in Map Task not correct",
         2, mapTask.getAttempts().size());
     
@@ -410,7 +415,6 @@ public class TestFetchFailure {
     Assert.assertEquals("Unexpected map event", convertedEvents[2],
         mapEvents[0]);
   }
-  
 
   private void updateStatus(MRApp app, TaskAttempt attempt, Phase phase) {
     TaskAttemptStatusUpdateEvent.TaskAttemptStatus status = new TaskAttemptStatusUpdateEvent.TaskAttemptStatus();
@@ -430,11 +434,12 @@ public class TestFetchFailure {
   }
 
   private void sendFetchFailure(MRApp app, TaskAttempt reduceAttempt, 
-      TaskAttempt mapAttempt) {
+      TaskAttempt mapAttempt, String hostname) {
     app.getContext().getEventHandler().handle(
         new JobTaskAttemptFetchFailureEvent(
             reduceAttempt.getID(), 
-            Arrays.asList(new TaskAttemptId[] {mapAttempt.getID()})));
+            Arrays.asList(new TaskAttemptId[] {mapAttempt.getID()}),
+                hostname));
   }
   
   static class MRAppWithHistory extends MRApp {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6ba5645/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
index 79b88d8..a88a935 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
@@ -70,6 +70,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssignedEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerLaunchedEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptDiagnosticsUpdateEvent;
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptTooManyFetchFailureEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.rm.ContainerRequestEvent;
@@ -507,6 +508,9 @@ public class TestTaskAttempt{
     JobId jobId = MRBuilderUtils.newJobId(appId, 1);
     TaskId taskId = MRBuilderUtils.newTaskId(jobId, 1, TaskType.MAP);
     TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(taskId, 0);
+    TaskId reduceTaskId = MRBuilderUtils.newTaskId(jobId, 1, TaskType.REDUCE);
+    TaskAttemptId reduceTAId =
+        MRBuilderUtils.newTaskAttemptId(reduceTaskId, 0);
     Path jobFile = mock(Path.class);
 
     MockEventHandler eventHandler = new MockEventHandler();
@@ -554,8 +558,8 @@ public class TestTaskAttempt{
 
     assertEquals("Task attempt is not in succeeded state", taImpl.getState(),
         TaskAttemptState.SUCCEEDED);
-    taImpl.handle(new TaskAttemptEvent(attemptId,
-        TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURE));
+    taImpl.handle(new TaskAttemptTooManyFetchFailureEvent(attemptId,
+        reduceTAId, "Host"));
     assertEquals("Task attempt is not in FAILED state", taImpl.getState(),
         TaskAttemptState.FAILED);
     taImpl.handle(new TaskAttemptEvent(attemptId,
@@ -735,72 +739,75 @@ public class TestTaskAttempt{
     
   @Test
   public void testFetchFailureAttemptFinishTime() throws Exception{
-	ApplicationId appId = ApplicationId.newInstance(1, 2);
-	ApplicationAttemptId appAttemptId =
-	ApplicationAttemptId.newInstance(appId, 0);
-	JobId jobId = MRBuilderUtils.newJobId(appId, 1);
-	TaskId taskId = MRBuilderUtils.newTaskId(jobId, 1, TaskType.MAP);
-	TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(taskId, 0);
-	Path jobFile = mock(Path.class);
-
-	MockEventHandler eventHandler = new MockEventHandler();
-	TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-	when(taListener.getAddress()).thenReturn(
-		new InetSocketAddress("localhost", 0));
-
-	JobConf jobConf = new JobConf();
-	jobConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
-	jobConf.setBoolean("fs.file.impl.disable.cache", true);
-	jobConf.set(JobConf.MAPRED_MAP_TASK_ENV, "");
-	jobConf.set(MRJobConfig.APPLICATION_ATTEMPT_ID, "10");
-
-	TaskSplitMetaInfo splits = mock(TaskSplitMetaInfo.class);
-	when(splits.getLocations()).thenReturn(new String[] {"127.0.0.1"});
-
-	AppContext appCtx = mock(AppContext.class);
-	ClusterInfo clusterInfo = mock(ClusterInfo.class);
-	when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
-  setupTaskAttemptFinishingMonitor(eventHandler, jobConf, appCtx);
-
-	TaskAttemptImpl taImpl =
-	  new MapTaskAttemptImpl(taskId, 1, eventHandler, jobFile, 1,
-	  splits, jobConf, taListener,mock(Token.class), new Credentials(),
-	  new SystemClock(), appCtx);
-
-	NodeId nid = NodeId.newInstance("127.0.0.1", 0);
-	ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
-	Container container = mock(Container.class);
-	when(container.getId()).thenReturn(contId);
-	when(container.getNodeId()).thenReturn(nid);
-	when(container.getNodeHttpAddress()).thenReturn("localhost:0"); 
-	    
-	taImpl.handle(new TaskAttemptEvent(attemptId,
-	 	TaskAttemptEventType.TA_SCHEDULE));
-	taImpl.handle(new TaskAttemptContainerAssignedEvent(attemptId,
-	    container, mock(Map.class)));
-	taImpl.handle(new TaskAttemptContainerLaunchedEvent(attemptId, 0));
-	taImpl.handle(new TaskAttemptEvent(attemptId,
-	    TaskAttemptEventType.TA_DONE));
-	taImpl.handle(new TaskAttemptEvent(attemptId,
-	    TaskAttemptEventType.TA_CONTAINER_COMPLETED));
-	    
-	assertEquals("Task attempt is not in succeeded state", taImpl.getState(),
-		      TaskAttemptState.SUCCEEDED);
-	
-	assertTrue("Task Attempt finish time is not greater than 0", 
-			taImpl.getFinishTime() > 0);
-	
-	Long finishTime = taImpl.getFinishTime();
-	Thread.sleep(5);   
-	taImpl.handle(new TaskAttemptEvent(attemptId,
-	   TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURE));
-	
-	assertEquals("Task attempt is not in Too Many Fetch Failure state", 
-			taImpl.getState(), TaskAttemptState.FAILED);
-	
-	assertEquals("After TA_TOO_MANY_FETCH_FAILURE,"
-		+ " Task attempt finish time is not the same ",
-		finishTime, Long.valueOf(taImpl.getFinishTime()));  
+    ApplicationId appId = ApplicationId.newInstance(1, 2);
+    ApplicationAttemptId appAttemptId =
+    ApplicationAttemptId.newInstance(appId, 0);
+    JobId jobId = MRBuilderUtils.newJobId(appId, 1);
+    TaskId taskId = MRBuilderUtils.newTaskId(jobId, 1, TaskType.MAP);
+    TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(taskId, 0);
+    TaskId reducetaskId = MRBuilderUtils.newTaskId(jobId, 1, TaskType.REDUCE);
+    TaskAttemptId reduceTAId =
+        MRBuilderUtils.newTaskAttemptId(reducetaskId, 0);
+    Path jobFile = mock(Path.class);
+
+    MockEventHandler eventHandler = new MockEventHandler();
+    TaskAttemptListener taListener = mock(TaskAttemptListener.class);
+    when(taListener.getAddress()).thenReturn(
+        new InetSocketAddress("localhost", 0));
+
+    JobConf jobConf = new JobConf();
+    jobConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
+    jobConf.setBoolean("fs.file.impl.disable.cache", true);
+    jobConf.set(JobConf.MAPRED_MAP_TASK_ENV, "");
+    jobConf.set(MRJobConfig.APPLICATION_ATTEMPT_ID, "10");
+
+    TaskSplitMetaInfo splits = mock(TaskSplitMetaInfo.class);
+    when(splits.getLocations()).thenReturn(new String[] {"127.0.0.1"});
+
+    AppContext appCtx = mock(AppContext.class);
+    ClusterInfo clusterInfo = mock(ClusterInfo.class);
+    when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
+    setupTaskAttemptFinishingMonitor(eventHandler, jobConf, appCtx);
+
+    TaskAttemptImpl taImpl =
+      new MapTaskAttemptImpl(taskId, 1, eventHandler, jobFile, 1,
+      splits, jobConf, taListener,mock(Token.class), new Credentials(),
+      new SystemClock(), appCtx);
+
+    NodeId nid = NodeId.newInstance("127.0.0.1", 0);
+    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
+    Container container = mock(Container.class);
+    when(container.getId()).thenReturn(contId);
+    when(container.getNodeId()).thenReturn(nid);
+    when(container.getNodeHttpAddress()).thenReturn("localhost:0");
+
+    taImpl.handle(new TaskAttemptEvent(attemptId,
+        TaskAttemptEventType.TA_SCHEDULE));
+    taImpl.handle(new TaskAttemptContainerAssignedEvent(attemptId,
+        container, mock(Map.class)));
+    taImpl.handle(new TaskAttemptContainerLaunchedEvent(attemptId, 0));
+    taImpl.handle(new TaskAttemptEvent(attemptId,
+        TaskAttemptEventType.TA_DONE));
+    taImpl.handle(new TaskAttemptEvent(attemptId,
+        TaskAttemptEventType.TA_CONTAINER_COMPLETED));
+
+    assertEquals("Task attempt is not in succeeded state", taImpl.getState(),
+        TaskAttemptState.SUCCEEDED);
+
+    assertTrue("Task Attempt finish time is not greater than 0",
+        taImpl.getFinishTime() > 0);
+
+    Long finishTime = taImpl.getFinishTime();
+    Thread.sleep(5);
+    taImpl.handle(new TaskAttemptTooManyFetchFailureEvent(attemptId,
+        reduceTAId, "Host"));
+
+    assertEquals("Task attempt is not in Too Many Fetch Failure state",
+        taImpl.getState(), TaskAttemptState.FAILED);
+
+    assertEquals("After TA_TOO_MANY_FETCH_FAILURE,"
+        + " Task attempt finish time is not the same ",
+        finishTime, Long.valueOf(taImpl.getFinishTime()));
   }
   
   @Test


[37/50] hadoop git commit: HADOOP-12149. copy all of test-patch BINDIR prior to re-exec (aw)

Posted by vv...@apache.org.
HADOOP-12149. copy all of test-patch BINDIR prior to re-exec (aw)


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

Branch: refs/heads/YARN-2139
Commit: 147e020c7aef3ba42eddcef3be1b4ae7c7910371
Parents: 076948d
Author: Allen Wittenauer <aw...@apache.org>
Authored: Tue Jun 30 11:31:26 2015 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Tue Jun 30 11:31:26 2015 -0700

----------------------------------------------------------------------
 dev-support/test-patch.sh                       | 4 ++--
 hadoop-common-project/hadoop-common/CHANGES.txt | 2 ++
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/147e020c/dev-support/test-patch.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.sh b/dev-support/test-patch.sh
index cd91a5c..efcd614 100755
--- a/dev-support/test-patch.sh
+++ b/dev-support/test-patch.sh
@@ -1521,8 +1521,8 @@ function check_reexec
 
   cd "${CWD}"
   mkdir -p "${PATCH_DIR}/dev-support-test"
-  cp -pr "${BASEDIR}"/dev-support/test-patch* "${PATCH_DIR}/dev-support-test"
-  cp -pr "${BASEDIR}"/dev-support/smart-apply* "${PATCH_DIR}/dev-support-test"
+  (cd "${BINDIR}"; tar cpf - . ) \
+	| (cd  "${PATCH_DIR}/dev-support-test"; tar xpf - )
 
   big_console_header "exec'ing test-patch.sh now..."
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/147e020c/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index c010ff1..e332ea8 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -232,6 +232,8 @@ Trunk (Unreleased)
 
     HADOOP-11807. add a lint mode to releasedocmaker (ramtin via aw)
 
+    HADOOP-12149. copy all of test-patch BINDIR prior to re-exec (aw)
+
   BUG FIXES
 
     HADOOP-11473. test-patch says "-1 overall" even when all checks are +1


[13/50] hadoop git commit: YARN-3859. LeafQueue doesn't print user properly for application add. Contributed by Varun Saxena.

Posted by vv...@apache.org.
YARN-3859. LeafQueue doesn't print user properly for application add.
Contributed by Varun Saxena.


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

Branch: refs/heads/YARN-2139
Commit: b543d1a390a67e5e92fea67d3a2635058c29e9da
Parents: 79ed0f9
Author: Devaraj K <de...@apache.org>
Authored: Sun Jun 28 10:04:50 2015 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Sun Jun 28 10:04:50 2015 +0530

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b543d1a3/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index e2bf706..cb6deb1 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -562,6 +562,9 @@ Release 2.8.0 - UNRELEASED
     YARN-2871. TestRMRestart#testRMRestartGetApplicationList sometime fails in trunk.
     (zhihai xu via xgong)
 
+    YARN-3859. LeafQueue doesn't print user properly for application add.
+    (Varun Saxena via devaraj)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b543d1a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
index 8e39133..e5b44a6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
@@ -676,7 +676,8 @@ public class LeafQueue extends AbstractCSQueue {
     
     LOG.info("Application added -" +
         " appId: " + application.getApplicationId() +
-        " user: " + user + "," + " leaf-queue: " + getQueueName() +
+        " user: " + application.getUser() + "," +
+        " leaf-queue: " + getQueueName() +
         " #user-pending-applications: " + user.getPendingApplications() +
         " #user-active-applications: " + user.getActiveApplications() +
         " #queue-pending-applications: " + getNumPendingApplications() +


[45/50] hadoop git commit: HADOOP-12124. Add HTrace support for FsShell (cmccabe)

Posted by vv...@apache.org.
HADOOP-12124. Add HTrace support for FsShell (cmccabe)


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

Branch: refs/heads/YARN-2139
Commit: ad60807238c4f7779cb0685e7d39ca0c50e01b2f
Parents: 68e588c
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Tue Jun 30 16:46:25 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Tue Jun 30 16:46:25 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad608072/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 4d69a229..19ba620 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -671,6 +671,8 @@ Release 2.8.0 - UNRELEASED
     TestCryptoStreamsWithOpensslAesCtrCryptoCodec when OpenSSL is not
     installed. (wang)
 
+    HADOOP-12124. Add HTrace support for FsShell (cmccabe)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad608072/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
index db73f6d..24b6339 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
@@ -33,8 +33,13 @@ import org.apache.hadoop.fs.shell.Command;
 import org.apache.hadoop.fs.shell.CommandFactory;
 import org.apache.hadoop.fs.shell.FsCommand;
 import org.apache.hadoop.tools.TableListing;
+import org.apache.hadoop.tracing.TraceUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.htrace.Sampler;
+import org.apache.htrace.SamplerBuilder;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
 
 /** Provide command line access to a FileSystem. */
 @InterfaceAudience.Private
@@ -47,6 +52,7 @@ public class FsShell extends Configured implements Tool {
   private FileSystem fs;
   private Trash trash;
   protected CommandFactory commandFactory;
+  private Sampler traceSampler;
 
   private final String usagePrefix =
     "Usage: hadoop fs [generic options]";
@@ -272,7 +278,8 @@ public class FsShell extends Configured implements Tool {
   public int run(String argv[]) throws Exception {
     // initialize FsShell
     init();
-
+    traceSampler = new SamplerBuilder(TraceUtils.
+        wrapHadoopConf("dfs.shell.htrace.", getConf())).build();
     int exitCode = -1;
     if (argv.length < 1) {
       printUsage(System.err);
@@ -284,7 +291,12 @@ public class FsShell extends Configured implements Tool {
         if (instance == null) {
           throw new UnknownCommandException();
         }
-        exitCode = instance.run(Arrays.copyOfRange(argv, 1, argv.length));
+        TraceScope scope = Trace.startSpan(instance.getCommandName(), traceSampler);
+        try {
+          exitCode = instance.run(Arrays.copyOfRange(argv, 1, argv.length));
+        } finally {
+          scope.close();
+        }
       } catch (IllegalArgumentException e) {
         displayError(cmd, e.getLocalizedMessage());
         if (instance != null) {


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

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


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

Branch: refs/heads/YARN-2139
Commit: 79ed0f959ffc490414ca56a73e026500c24e7078
Parents: fe6c1bd
Author: Allen Wittenauer <aw...@apache.org>
Authored: Sat Jun 27 08:59:50 2015 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Sat Jun 27 08:59:50 2015 -0700

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79ed0f95/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 92e1bfa..18c73c0 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -230,6 +230,8 @@ Trunk (Unreleased)
     HADOOP-11142. Remove hdfs dfs reference from file system shell
     documentation (Kengo Seki via aw)
 
+    HADOOP-11807. add a lint mode to releasedocmaker (ramtin via aw)
+
   BUG FIXES
 
     HADOOP-11473. test-patch says "-1 overall" even when all checks are +1


[30/50] hadoop git commit: HDFS-8659. Block scanner INFO message is spamming logs. Contributed by Yongjun Zhang.

Posted by vv...@apache.org.
HDFS-8659. Block scanner INFO message is spamming logs. 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/43a1288f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/43a1288f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/43a1288f

Branch: refs/heads/YARN-2139
Commit: 43a1288fff5e323895fde66b9e3eb31a7412fcc2
Parents: 34ee0b9
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Mon Jun 29 14:35:39 2015 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Mon Jun 29 14:43:56 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                  | 2 ++
 .../org/apache/hadoop/hdfs/server/datanode/BlockScanner.java | 2 +-
 .../apache/hadoop/hdfs/server/datanode/VolumeScanner.java    | 8 ++++----
 3 files changed, 7 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/43a1288f/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 0c56f2b..eb006eb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -985,6 +985,8 @@ Release 2.7.2 - UNRELEASED
 
   IMPROVEMENTS
 
+  HDFS-8659. Block scanner INFO message is spamming logs. (Yongjun Zhang)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43a1288f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
index 9c4dd10..be6aa83 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
@@ -317,7 +317,7 @@ public class BlockScanner {
    */
   synchronized void markSuspectBlock(String storageId, ExtendedBlock block) {
     if (!isEnabled()) {
-      LOG.info("Not scanning suspicious block {} on {}, because the block " +
+      LOG.debug("Not scanning suspicious block {} on {}, because the block " +
           "scanner is disabled.", block, storageId);
       return;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43a1288f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java
index 615abe9..ff655c2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java
@@ -656,24 +656,24 @@ public class VolumeScanner extends Thread {
 
   public synchronized void markSuspectBlock(ExtendedBlock block) {
     if (stopping) {
-      LOG.info("{}: Not scheduling suspect block {} for " +
+      LOG.debug("{}: Not scheduling suspect block {} for " +
           "rescanning, because this volume scanner is stopping.", this, block);
       return;
     }
     Boolean recent = recentSuspectBlocks.getIfPresent(block);
     if (recent != null) {
-      LOG.info("{}: Not scheduling suspect block {} for " +
+      LOG.debug("{}: Not scheduling suspect block {} for " +
           "rescanning, because we rescanned it recently.", this, block);
       return;
     }
     if (suspectBlocks.contains(block)) {
-      LOG.info("{}: suspect block {} is already queued for " +
+      LOG.debug("{}: suspect block {} is already queued for " +
           "rescanning.", this, block);
       return;
     }
     suspectBlocks.add(block);
     recentSuspectBlocks.put(block, true);
-    LOG.info("{}: Scheduling suspect block {} for rescanning.", this, block);
+    LOG.debug("{}: Scheduling suspect block {} for rescanning.", this, block);
     notify(); // wake scanner thread.
   }
 


[08/50] hadoop git commit: HDFS-8623. Refactor NameNode handling of invalid, corrupt, and under-recovery blocks. Contributed by Zhe Zhang.

Posted by vv...@apache.org.
HDFS-8623. Refactor NameNode handling of invalid, corrupt, and under-recovery blocks. Contributed by Zhe Zhang.


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

Branch: refs/heads/YARN-2139
Commit: de480d6c8945bd8b5b00e8657b7a72ce8dd9b6b5
Parents: 1b764a0
Author: Jing Zhao <ji...@apache.org>
Authored: Fri Jun 26 10:49:01 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Fri Jun 26 10:49:01 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hdfs/server/blockmanagement/BlockInfo.java  |  12 +-
 .../blockmanagement/BlockInfoContiguous.java    |   2 +-
 .../BlockInfoUnderConstructionContiguous.java   |   2 +-
 .../server/blockmanagement/BlockManager.java    | 595 ++++++++++---------
 .../blockmanagement/DatanodeStorageInfo.java    |  15 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  18 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |   2 +-
 .../server/blockmanagement/TestBlockInfo.java   |   2 +-
 .../blockmanagement/TestBlockManager.java       |   4 +-
 .../server/blockmanagement/TestNodeCount.java   |   2 +-
 .../TestOverReplicatedBlocks.java               |   4 +-
 .../blockmanagement/TestReplicationPolicy.java  |   2 +-
 13 files changed, 361 insertions(+), 302 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 27e2e89..bb1b3ff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -679,6 +679,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8651. Make hadoop-hdfs-project Native code -Wall-clean (Alan Burlison
     via Colin P. McCabe)
 
+    HDFS-8623. Refactor NameNode handling of invalid, corrupt, and under-recovery
+    blocks. (Zhe Zhang via jing9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
index 4cc2791..5ad992b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
@@ -172,19 +172,23 @@ public abstract class  BlockInfo extends Block
   public abstract int numNodes();
 
   /**
-   * Add a {@link DatanodeStorageInfo} location for a block.
+   * Add a {@link DatanodeStorageInfo} location for a block
+   * @param storage The storage to add
+   * @param reportedBlock The block reported from the datanode. This is only
+   *                      used by erasure coded blocks, this block's id contains
+   *                      information indicating the index of the block in the
+   *                      corresponding block group.
    */
-  abstract boolean addStorage(DatanodeStorageInfo storage);
+  abstract boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock);
 
   /**
    * Remove {@link DatanodeStorageInfo} location for a block
    */
   abstract boolean removeStorage(DatanodeStorageInfo storage);
 
-
   /**
    * Replace the current BlockInfo with the new one in corresponding
-   * DatanodeStorageInfo's linked list
+   * DatanodeStorageInfo's linked list.
    */
   abstract void replaceBlock(BlockInfo newBlock);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
index b9abcd0..de64ad8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
@@ -45,7 +45,7 @@ public class BlockInfoContiguous extends BlockInfo {
   }
 
   @Override
-  boolean addStorage(DatanodeStorageInfo storage) {
+  boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock) {
     return ContiguousBlockStorageOp.addStorage(this, storage);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java
index c66675a..d3cb337 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java
@@ -69,7 +69,7 @@ public class BlockInfoUnderConstructionContiguous extends
   }
 
   @Override
-  boolean addStorage(DatanodeStorageInfo storage) {
+  boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock) {
     return ContiguousBlockStorageOp.addStorage(this, storage);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/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 368d3b0..5bd4980 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
@@ -198,8 +198,8 @@ public class BlockManager {
    * Maps a StorageID to the set of blocks that are "extra" for this
    * DataNode. We'll eventually remove these extras.
    */
-  public final Map<String, LightWeightLinkedSet<Block>> excessReplicateMap =
-    new TreeMap<String, LightWeightLinkedSet<Block>>();
+  public final Map<String, LightWeightLinkedSet<BlockInfo>> excessReplicateMap =
+    new TreeMap<>();
 
   /**
    * Store set of Blocks that need to be replicated 1 or more times.
@@ -502,8 +502,8 @@ public class BlockManager {
   /** Dump meta data to out. */
   public void metaSave(PrintWriter out) {
     assert namesystem.hasWriteLock();
-    final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
-    final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
+    final List<DatanodeDescriptor> live = new ArrayList<>();
+    final List<DatanodeDescriptor> dead = new ArrayList<>();
     datanodeManager.fetchDatanodes(live, dead, false);
     out.println("Live Datanodes: " + live.size());
     out.println("Dead Datanodes: " + dead.size());
@@ -542,8 +542,8 @@ public class BlockManager {
     List<DatanodeDescriptor> containingNodes =
                                       new ArrayList<DatanodeDescriptor>();
     List<DatanodeStorageInfo> containingLiveReplicasNodes =
-      new ArrayList<DatanodeStorageInfo>();
-    
+      new ArrayList<>();
+
     NumberReplicas numReplicas = new NumberReplicas();
     // source node returned is not used
     chooseSourceDatanode(block, containingNodes,
@@ -572,7 +572,7 @@ public class BlockManager {
     Collection<DatanodeDescriptor> corruptNodes = 
                                   corruptReplicas.getNodes(block);
     
-    for (DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
+    for (DatanodeStorageInfo storage : getStorages(block)) {
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
       String state = "";
       if (corruptNodes != null && corruptNodes.contains(node)) {
@@ -595,11 +595,23 @@ public class BlockManager {
     return maxReplicationStreams;
   }
 
+  public int getDefaultStorageNum(BlockInfo block) {
+    return defaultReplication;
+  }
+
+  public short getMinStorageNum(BlockInfo block) {
+    return minReplication;
+  }
+
   /**
-   * @return true if the block has minimum replicas
+   * @return true if the block has minimum stored copies
    */
-  public boolean checkMinReplication(BlockInfo block) {
-    return (countNodes(block).liveReplicas() >= minReplication);
+  public boolean hasMinStorage(BlockInfo block) {
+    return hasMinStorage(block, countNodes(block).liveReplicas());
+  }
+
+  public boolean hasMinStorage(BlockInfo block, int liveNum) {
+    return liveNum >= getMinStorageNum(block);
   }
 
   /**
@@ -614,8 +626,9 @@ public class BlockManager {
   private static boolean commitBlock(
       final BlockInfoUnderConstruction block, final Block commitBlock)
       throws IOException {
-    if (block.getBlockUCState() == BlockUCState.COMMITTED)
+    if (block.getBlockUCState() == BlockUCState.COMMITTED) {
       return false;
+    }
     assert block.getNumBytes() <= commitBlock.getNumBytes() :
       "commitBlock length is less than the stored one "
       + commitBlock.getNumBytes() + " vs. " + block.getNumBytes();
@@ -635,18 +648,22 @@ public class BlockManager {
    */
   public boolean commitOrCompleteLastBlock(BlockCollection bc,
       Block commitBlock) throws IOException {
-    if(commitBlock == null)
+    if (commitBlock == null) {
       return false; // not committing, this is a block allocation retry
+    }
     BlockInfo lastBlock = bc.getLastBlock();
-    if(lastBlock == null)
+    if (lastBlock == null) {
       return false; // no blocks in file yet
-    if(lastBlock.isComplete())
+    }
+    if (lastBlock.isComplete()) {
       return false; // already completed (e.g. by syncBlock)
-    
+    }
+
     final boolean b = commitBlock(
         (BlockInfoUnderConstruction) lastBlock, commitBlock);
-    if(countNodes(lastBlock).liveReplicas() >= minReplication)
+    if(hasMinStorage(lastBlock)) {
       completeBlock(bc, bc.numBlocks()-1, false);
+    }
     return b;
   }
 
@@ -659,20 +676,24 @@ public class BlockManager {
    */
   private BlockInfo completeBlock(final BlockCollection bc,
       final int blkIndex, boolean force) throws IOException {
-    if(blkIndex < 0)
+    if (blkIndex < 0) {
       return null;
+    }
     BlockInfo curBlock = bc.getBlocks()[blkIndex];
-    if(curBlock.isComplete())
+    if(curBlock.isComplete()) {
       return curBlock;
+    }
     BlockInfoUnderConstruction ucBlock =
         (BlockInfoUnderConstruction) curBlock;
     int numNodes = ucBlock.numNodes();
-    if (!force && numNodes < minReplication)
+    if (!force && !hasMinStorage(curBlock, numNodes)) {
       throw new IOException("Cannot complete block: " +
           "block does not satisfy minimal replication requirement.");
-    if(!force && ucBlock.getBlockUCState() != BlockUCState.COMMITTED)
+    }
+    if(!force && ucBlock.getBlockUCState() != BlockUCState.COMMITTED) {
       throw new IOException(
           "Cannot complete block: block has not been COMMITTED by the client");
+    }
     BlockInfo completeBlock = ucBlock.convertToCompleteBlock();
     // replace penultimate block in file
     bc.setBlock(blkIndex, completeBlock);
@@ -757,7 +778,7 @@ public class BlockManager {
     // count in safe-mode.
     namesystem.adjustSafeModeBlockTotals(
         // decrement safe if we had enough
-        targets.length >= minReplication ? -1 : 0,
+        hasMinStorage(oldBlock, targets.length) ? -1 : 0,
         // always decrement total blocks
         -1);
 
@@ -771,8 +792,8 @@ public class BlockManager {
    */
   private List<DatanodeStorageInfo> getValidLocations(Block block) {
     final List<DatanodeStorageInfo> locations
-        = new ArrayList<DatanodeStorageInfo>(blocksMap.numNodes(block));
-    for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
+        = new ArrayList<>(blocksMap.numNodes(block));
+    for(DatanodeStorageInfo storage : getStorages(block)) {
       // filter invalidate replicas
       if(!invalidateBlocks.contains(storage.getDatanodeDescriptor(), block)) {
         locations.add(storage);
@@ -785,7 +806,7 @@ public class BlockManager {
       final BlockInfo[] blocks,
       final long offset, final long length, final int nrBlocksToReturn,
       final AccessMode mode) throws IOException {
-    int curBlk = 0;
+    int curBlk;
     long curPos = 0, blkSize = 0;
     int nrBlocks = (blocks[0].getNumBytes() == 0) ? 0 : blocks.length;
     for (curBlk = 0; curBlk < nrBlocks; curBlk++) {
@@ -798,10 +819,10 @@ public class BlockManager {
     }
 
     if (nrBlocks > 0 && curBlk == nrBlocks)   // offset >= end of file
-      return Collections.<LocatedBlock>emptyList();
+      return Collections.emptyList();
 
     long endOff = offset + length;
-    List<LocatedBlock> results = new ArrayList<LocatedBlock>(blocks.length);
+    List<LocatedBlock> results = new ArrayList<>(blocks.length);
     do {
       results.add(createLocatedBlock(blocks[curBlk], curPos, mode));
       curPos += blocks[curBlk].getNumBytes();
@@ -814,7 +835,7 @@ public class BlockManager {
 
   private LocatedBlock createLocatedBlock(final BlockInfo[] blocks,
       final long endPos, final AccessMode mode) throws IOException {
-    int curBlk = 0;
+    int curBlk;
     long curPos = 0;
     int nrBlocks = (blocks[0].getNumBytes() == 0) ? 0 : blocks.length;
     for (curBlk = 0; curBlk < nrBlocks; curBlk++) {
@@ -838,8 +859,8 @@ public class BlockManager {
   }
 
   /** @return a LocatedBlock for the given block */
-  private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos
-      ) throws IOException {
+  private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos)
+      throws IOException {
     if (blk instanceof BlockInfoUnderConstruction) {
       if (blk.isComplete()) {
         throw new IOException(
@@ -849,7 +870,8 @@ public class BlockManager {
       final BlockInfoUnderConstruction uc =
           (BlockInfoUnderConstruction) blk;
       final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
-      final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
+      final ExtendedBlock eb =
+          new ExtendedBlock(namesystem.getBlockPoolId(), blk);
       return newLocatedBlock(eb, storages, pos, false);
     }
 
@@ -869,11 +891,12 @@ public class BlockManager {
     final DatanodeStorageInfo[] machines = new DatanodeStorageInfo[numMachines];
     int j = 0;
     if (numMachines > 0) {
-      for(DatanodeStorageInfo storage : blocksMap.getStorages(blk)) {
+      for(DatanodeStorageInfo storage : getStorages(blk)) {
         final DatanodeDescriptor d = storage.getDatanodeDescriptor();
         final boolean replicaCorrupt = corruptReplicas.isReplicaCorrupt(blk, d);
-        if (isCorrupt || (!replicaCorrupt))
+        if (isCorrupt || (!replicaCorrupt)) {
           machines[j++] = storage;
+        }
       }
     }
     assert j == machines.length :
@@ -1047,7 +1070,7 @@ public class BlockManager {
     for(int i=0; i<startBlock; i++) {
       iter.next();
     }
-    List<BlockWithLocations> results = new ArrayList<BlockWithLocations>();
+    List<BlockWithLocations> results = new ArrayList<>();
     long totalSize = 0;
     BlockInfo curBlock;
     while(totalSize<size && iter.hasNext()) {
@@ -1071,7 +1094,7 @@ public class BlockManager {
    
   /** Remove the blocks associated to the given datanode. */
   void removeBlocksAssociatedTo(final DatanodeDescriptor node) {
-    final Iterator<? extends Block> it = node.getBlockIterator();
+    final Iterator<BlockInfo> it = node.getBlockIterator();
     while(it.hasNext()) {
       removeStoredBlock(it.next(), node);
     }
@@ -1085,10 +1108,10 @@ public class BlockManager {
   /** Remove the blocks associated to the given DatanodeStorageInfo. */
   void removeBlocksAssociatedTo(final DatanodeStorageInfo storageInfo) {
     assert namesystem.hasWriteLock();
-    final Iterator<? extends Block> it = storageInfo.getBlockIterator();
+    final Iterator<BlockInfo> it = storageInfo.getBlockIterator();
     DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
     while(it.hasNext()) {
-      Block block = it.next();
+      BlockInfo block = it.next();
       removeStoredBlock(block, node);
       invalidateBlocks.remove(node, block);
     }
@@ -1110,18 +1133,20 @@ public class BlockManager {
    * Adds block to list of blocks which will be invalidated on all its
    * datanodes.
    */
-  private void addToInvalidates(Block b) {
+  private void addToInvalidates(BlockInfo storedBlock) {
     if (!namesystem.isPopulatingReplQueues()) {
       return;
     }
     StringBuilder datanodes = new StringBuilder();
-    for(DatanodeStorageInfo storage : blocksMap.getStorages(b, State.NORMAL)) {
+    for(DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock,
+        State.NORMAL)) {
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
-      invalidateBlocks.add(b, node, false);
+      invalidateBlocks.add(storedBlock, node, false);
       datanodes.append(node).append(" ");
     }
     if (datanodes.length() != 0) {
-      blockLog.info("BLOCK* addToInvalidates: {} {}", b, datanodes.toString());
+      blockLog.info("BLOCK* addToInvalidates: {} {}", storedBlock,
+          datanodes.toString());
     }
   }
 
@@ -1148,7 +1173,8 @@ public class BlockManager {
   public void findAndMarkBlockAsCorrupt(final ExtendedBlock blk,
       final DatanodeInfo dn, String storageID, String reason) throws IOException {
     assert namesystem.hasWriteLock();
-    final BlockInfo storedBlock = getStoredBlock(blk.getLocalBlock());
+    final Block reportedBlock = blk.getLocalBlock();
+    final BlockInfo storedBlock = getStoredBlock(reportedBlock);
     if (storedBlock == null) {
       // Check if the replica is in the blockMap, if not
       // ignore the request for now. This could happen when BlockScanner
@@ -1164,8 +1190,8 @@ public class BlockManager {
           + " as corrupt because datanode " + dn + " (" + dn.getDatanodeUuid()
           + ") does not exist");
     }
-    
-    markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock,
+
+    markBlockAsCorrupt(new BlockToMarkCorrupt(reportedBlock, storedBlock,
             blk.getGenerationStamp(), reason, Reason.CORRUPTION_REPORTED),
         storageID == null ? null : node.getStorageInfo(storageID),
         node);
@@ -1181,18 +1207,18 @@ public class BlockManager {
       DatanodeStorageInfo storageInfo,
       DatanodeDescriptor node) throws IOException {
 
-    if (b.corrupted.isDeleted()) {
+    if (b.stored.isDeleted()) {
       blockLog.info("BLOCK markBlockAsCorrupt: {} cannot be marked as" +
           " corrupt as it does not belong to any file", b);
       addToInvalidates(b.corrupted, node);
       return;
     } 
     short expectedReplicas =
-        b.corrupted.getBlockCollection().getPreferredBlockReplication();
+        getExpectedReplicaNum(b.stored.getBlockCollection(), b.stored);
 
     // Add replica to the data-node if it is not already there
     if (storageInfo != null) {
-      storageInfo.addBlock(b.stored);
+      storageInfo.addBlock(b.stored, b.corrupted);
     }
 
     // Add this replica to corruptReplicas Map
@@ -1202,8 +1228,8 @@ public class BlockManager {
     NumberReplicas numberOfReplicas = countNodes(b.stored);
     boolean hasEnoughLiveReplicas = numberOfReplicas.liveReplicas() >=
         expectedReplicas;
-    boolean minReplicationSatisfied =
-        numberOfReplicas.liveReplicas() >= minReplication;
+    boolean minReplicationSatisfied = hasMinStorage(b.stored,
+        numberOfReplicas.liveReplicas());
     boolean hasMoreCorruptReplicas = minReplicationSatisfied &&
         (numberOfReplicas.liveReplicas() + numberOfReplicas.corruptReplicas()) >
         expectedReplicas;
@@ -1346,7 +1372,7 @@ public class BlockManager {
     int additionalReplRequired;
 
     int scheduledWork = 0;
-    List<ReplicationWork> work = new LinkedList<ReplicationWork>();
+    List<ReplicationWork> work = new LinkedList<>();
 
     namesystem.writeLock();
     try {
@@ -1363,11 +1389,11 @@ public class BlockManager {
               continue;
             }
 
-            requiredReplication = bc.getPreferredBlockReplication();
+            requiredReplication = getExpectedReplicaNum(bc, block);
 
             // get a source data-node
-            containingNodes = new ArrayList<DatanodeDescriptor>();
-            List<DatanodeStorageInfo> liveReplicaNodes = new ArrayList<DatanodeStorageInfo>();
+            containingNodes = new ArrayList<>();
+            List<DatanodeStorageInfo> liveReplicaNodes = new ArrayList<>();
             NumberReplicas numReplicas = new NumberReplicas();
             srcNode = chooseSourceDatanode(
                 block, containingNodes, liveReplicaNodes, numReplicas,
@@ -1387,7 +1413,7 @@ public class BlockManager {
       
             if (numEffectiveReplicas >= requiredReplication) {
               if ( (pendingReplications.getNumReplicas(block) > 0) ||
-                   (blockHasEnoughRacks(block)) ) {
+                   (blockHasEnoughRacks(block, requiredReplication)) ) {
                 neededReplications.remove(block, priority); // remove from neededReplications
                 blockLog.info("BLOCK* Removing {} from neededReplications as" +
                         " it has enough replicas", block);
@@ -1411,7 +1437,7 @@ public class BlockManager {
       namesystem.writeUnlock();
     }
 
-    final Set<Node> excludedNodes = new HashSet<Node>();
+    final Set<Node> excludedNodes = new HashSet<>();
     for(ReplicationWork rw : work){
       // Exclude all of the containing nodes from being targets.
       // This list includes decommissioning or corrupt nodes.
@@ -1447,7 +1473,7 @@ public class BlockManager {
             rw.targets = null;
             continue;
           }
-          requiredReplication = bc.getPreferredBlockReplication();
+          requiredReplication = getExpectedReplicaNum(bc, block);
 
           // do not schedule more if enough replicas is already pending
           NumberReplicas numReplicas = countNodes(block);
@@ -1456,7 +1482,7 @@ public class BlockManager {
 
           if (numEffectiveReplicas >= requiredReplication) {
             if ( (pendingReplications.getNumReplicas(block) > 0) ||
-                 (blockHasEnoughRacks(block)) ) {
+                 (blockHasEnoughRacks(block, requiredReplication)) ) {
               neededReplications.remove(block, priority); // remove from neededReplications
               rw.targets = null;
               blockLog.info("BLOCK* Removing {} from neededReplications as" +
@@ -1466,7 +1492,7 @@ public class BlockManager {
           }
 
           if ( (numReplicas.liveReplicas() >= requiredReplication) &&
-               (!blockHasEnoughRacks(block)) ) {
+               (!blockHasEnoughRacks(block, requiredReplication)) ) {
             if (rw.srcNode.getNetworkLocation().equals(
                 targets[0].getDatanodeDescriptor().getNetworkLocation())) {
               //No use continuing, unless a new rack in this case
@@ -1581,7 +1607,7 @@ public class BlockManager {
   List<DatanodeDescriptor> getDatanodeDescriptors(List<String> nodes) {
     List<DatanodeDescriptor> datanodeDescriptors = null;
     if (nodes != null) {
-      datanodeDescriptors = new ArrayList<DatanodeDescriptor>(nodes.size());
+      datanodeDescriptors = new ArrayList<>(nodes.size());
       for (int i = 0; i < nodes.size(); i++) {
         DatanodeDescriptor node = datanodeManager.getDatanodeDescriptor(nodes.get(i));
         if (node != null) {
@@ -1637,9 +1663,9 @@ public class BlockManager {
     int excess = 0;
     
     Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(block);
-    for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
+    for(DatanodeStorageInfo storage : getStorages(block)) {
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
-      LightWeightLinkedSet<Block> excessBlocks =
+      LightWeightLinkedSet<BlockInfo> excessBlocks =
         excessReplicateMap.get(node.getDatanodeUuid());
       int countableReplica = storage.getState() == State.NORMAL ? 1 : 0; 
       if ((nodesCorrupt != null) && (nodesCorrupt.contains(node)))
@@ -1707,7 +1733,7 @@ public class BlockManager {
            * Use the blockinfo from the blocksmap to be certain we're working
            * with the most up-to-date block information (e.g. genstamp).
            */
-          BlockInfo bi = blocksMap.getStoredBlock(timedOutItems[i]);
+          BlockInfo bi = getStoredBlock(timedOutItems[i]);
           if (bi == null) {
             continue;
           }
@@ -1757,7 +1783,7 @@ public class BlockManager {
     final BlockInfoUnderConstruction storedBlock;
     final Block reportedBlock;
     final ReplicaState reportedState;
-    
+
     StatefulBlockInfo(BlockInfoUnderConstruction storedBlock,
         Block reportedBlock, ReplicaState reportedState) {
       this.storedBlock = storedBlock;
@@ -1765,14 +1791,34 @@ public class BlockManager {
       this.reportedState = reportedState;
     }
   }
-  
+
+  private static class BlockInfoToAdd {
+    private final BlockInfo stored;
+    private final Block reported;
+
+    BlockInfoToAdd(BlockInfo stored, Block reported) {
+      this.stored = stored;
+      this.reported = reported;
+    }
+
+    public BlockInfo getStored() {
+      return stored;
+    }
+
+    public Block getReported() {
+      return reported;
+    }
+  }
+
   /**
    * BlockToMarkCorrupt is used to build the "toCorrupt" list, which is a
    * list of blocks that should be considered corrupt due to a block report.
    */
   private static class BlockToMarkCorrupt {
-    /** The corrupted block in a datanode. */
-    final BlockInfo corrupted;
+    /** The corrupted block in a datanode. This is the one reported by the
+     * datanode.
+     */
+    final Block corrupted;
     /** The corresponding block stored in the BlockManager. */
     final BlockInfo stored;
     /** The reason to mark corrupt. */
@@ -1780,7 +1826,7 @@ public class BlockManager {
     /** The reason code to be stored */
     final Reason reasonCode;
 
-    BlockToMarkCorrupt(BlockInfo corrupted,
+    BlockToMarkCorrupt(Block corrupted,
         BlockInfo stored, String reason,
         Reason reasonCode) {
       Preconditions.checkNotNull(corrupted, "corrupted is null");
@@ -1792,15 +1838,9 @@ public class BlockManager {
       this.reasonCode = reasonCode;
     }
 
-    BlockToMarkCorrupt(BlockInfo stored, String reason,
-        Reason reasonCode) {
-      this(stored, stored, reason, reasonCode);
-    }
-
-    BlockToMarkCorrupt(BlockInfo stored, long gs, String reason,
-        Reason reasonCode) {
-      this(new BlockInfoContiguous(stored), stored,
-          reason, reasonCode);
+    BlockToMarkCorrupt(Block corrupted, BlockInfo stored, long gs,
+        String reason, Reason reasonCode) {
+      this(corrupted, stored, reason, reasonCode);
       //the corrupted block in datanode has a different generation stamp
       corrupted.setGenerationStamp(gs);
     }
@@ -1987,7 +2027,7 @@ public class BlockManager {
           break;
         }
 
-        BlockInfo bi = blocksMap.getStoredBlock(b);
+        BlockInfo bi = getStoredBlock(b);
         if (bi == null) {
           if (LOG.isDebugEnabled()) {
             LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
@@ -2019,7 +2059,7 @@ public class BlockManager {
           endPostponedMisReplicatedBlocksCount) + " blocks are removed.");
     }
   }
-  
+
   private Collection<Block> processReport(
       final DatanodeStorageInfo storageInfo,
       final BlockListAsLongs report) throws IOException {
@@ -2027,25 +2067,26 @@ public class BlockManager {
     // Modify the (block-->datanode) map, according to the difference
     // between the old and new block report.
     //
-    Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
-    Collection<Block> toRemove = new TreeSet<Block>();
-    Collection<Block> toInvalidate = new LinkedList<Block>();
-    Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
-    Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
+    Collection<BlockInfoToAdd> toAdd = new LinkedList<>();
+    Collection<BlockInfo> toRemove = new TreeSet<>();
+    Collection<Block> toInvalidate = new LinkedList<>();
+    Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<>();
+    Collection<StatefulBlockInfo> toUC = new LinkedList<>();
     reportDiff(storageInfo, report,
         toAdd, toRemove, toInvalidate, toCorrupt, toUC);
-   
+
     DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
     // Process the blocks on each queue
-    for (StatefulBlockInfo b : toUC) { 
+    for (StatefulBlockInfo b : toUC) {
       addStoredBlockUnderConstruction(b, storageInfo);
     }
-    for (Block b : toRemove) {
+    for (BlockInfo b : toRemove) {
       removeStoredBlock(b, node);
     }
     int numBlocksLogged = 0;
-    for (BlockInfo b : toAdd) {
-      addStoredBlock(b, storageInfo, null, numBlocksLogged < maxNumBlocksToLog);
+    for (BlockInfoToAdd b : toAdd) {
+      addStoredBlock(b.getStored(), b.getReported(), storageInfo, null,
+          numBlocksLogged < maxNumBlocksToLog);
       numBlocksLogged++;
     }
     if (numBlocksLogged > maxNumBlocksToLog) {
@@ -2066,17 +2107,17 @@ public class BlockManager {
    * Mark block replicas as corrupt except those on the storages in 
    * newStorages list.
    */
-  public void markBlockReplicasAsCorrupt(BlockInfo block,
-      long oldGenerationStamp, long oldNumBytes, 
+  public void markBlockReplicasAsCorrupt(Block oldBlock, BlockInfo block,
+      long oldGenerationStamp, long oldNumBytes,
       DatanodeStorageInfo[] newStorages) throws IOException {
     assert namesystem.hasWriteLock();
     BlockToMarkCorrupt b = null;
     if (block.getGenerationStamp() != oldGenerationStamp) {
-      b = new BlockToMarkCorrupt(block, oldGenerationStamp,
+      b = new BlockToMarkCorrupt(oldBlock, block, oldGenerationStamp,
           "genstamp does not match " + oldGenerationStamp
           + " : " + block.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
     } else if (block.getNumBytes() != oldNumBytes) {
-      b = new BlockToMarkCorrupt(block,
+      b = new BlockToMarkCorrupt(oldBlock, block,
           "length does not match " + oldNumBytes
           + " : " + block.getNumBytes(), Reason.SIZE_MISMATCH);
     } else {
@@ -2135,7 +2176,7 @@ public class BlockManager {
         continue;
       }
       
-      BlockInfo storedBlock = blocksMap.getStoredBlock(iblk);
+      BlockInfo storedBlock = getStoredBlock(iblk);
       // If block does not belong to any file, we are done.
       if (storedBlock == null) continue;
       
@@ -2173,24 +2214,26 @@ public class BlockManager {
       }      
       //add replica if appropriate
       if (reportedState == ReplicaState.FINALIZED) {
-        addStoredBlockImmediate(storedBlock, storageInfo);
+        addStoredBlockImmediate(storedBlock, iblk, storageInfo);
       }
     }
   }
 
-  private void reportDiff(DatanodeStorageInfo storageInfo, 
-      BlockListAsLongs newReport, 
-      Collection<BlockInfo> toAdd,              // add to DatanodeDescriptor
-      Collection<Block> toRemove,           // remove from DatanodeDescriptor
+  private void reportDiff(DatanodeStorageInfo storageInfo,
+      BlockListAsLongs newReport,
+      Collection<BlockInfoToAdd> toAdd,     // add to DatanodeDescriptor
+      Collection<BlockInfo> toRemove,       // remove from DatanodeDescriptor
       Collection<Block> toInvalidate,       // should be removed from DN
       Collection<BlockToMarkCorrupt> toCorrupt, // add to corrupt replicas list
       Collection<StatefulBlockInfo> toUC) { // add to under-construction list
 
-    // place a delimiter in the list which separates blocks 
+    // place a delimiter in the list which separates blocks
     // that have been reported from those that have not
-    BlockInfo delimiter = new BlockInfoContiguous(new Block(), (short) 1);
-    AddBlockResult result = storageInfo.addBlock(delimiter);
-    assert result == AddBlockResult.ADDED 
+    Block delimiterBlock = new Block();
+    BlockInfo delimiter = new BlockInfoContiguous(delimiterBlock,
+        (short) 1);
+    AddBlockResult result = storageInfo.addBlock(delimiter, delimiterBlock);
+    assert result == AddBlockResult.ADDED
         : "Delimiting block cannot be present in the node";
     int headIndex = 0; //currently the delimiter is in the head of the list
     int curIndex;
@@ -2207,7 +2250,8 @@ public class BlockManager {
       // move block to the head of the list
       if (storedBlock != null &&
           (curIndex = storedBlock.findStorageInfo(storageInfo)) >= 0) {
-        headIndex = storageInfo.moveBlockToHead(storedBlock, curIndex, headIndex);
+        headIndex =
+            storageInfo.moveBlockToHead(storedBlock, curIndex, headIndex);
       }
     }
 
@@ -2215,8 +2259,9 @@ public class BlockManager {
     // all of them are next to the delimiter
     Iterator<BlockInfo> it =
         storageInfo.new BlockIterator(delimiter.getNext(0));
-    while(it.hasNext())
+    while (it.hasNext()) {
       toRemove.add(it.next());
+    }
     storageInfo.removeBlock(delimiter);
   }
 
@@ -2253,12 +2298,12 @@ public class BlockManager {
    */
   private BlockInfo processReportedBlock(
       final DatanodeStorageInfo storageInfo,
-      final Block block, final ReplicaState reportedState, 
-      final Collection<BlockInfo> toAdd,
-      final Collection<Block> toInvalidate, 
+      final Block block, final ReplicaState reportedState,
+      final Collection<BlockInfoToAdd> toAdd,
+      final Collection<Block> toInvalidate,
       final Collection<BlockToMarkCorrupt> toCorrupt,
       final Collection<StatefulBlockInfo> toUC) {
-    
+
     DatanodeDescriptor dn = storageInfo.getDatanodeDescriptor();
 
     if(LOG.isDebugEnabled()) {
@@ -2266,16 +2311,16 @@ public class BlockManager {
           + " on " + dn + " size " + block.getNumBytes()
           + " replicaState = " + reportedState);
     }
-  
+
     if (shouldPostponeBlocksFromFuture &&
         namesystem.isGenStampInFuture(block)) {
       queueReportedBlock(storageInfo, block, reportedState,
           QUEUE_REASON_FUTURE_GENSTAMP);
       return null;
     }
-    
+
     // find block by blockId
-    BlockInfo storedBlock = blocksMap.getStoredBlock(block);
+    BlockInfo storedBlock = getStoredBlock(block);
     if(storedBlock == null) {
       // If blocksMap does not contain reported block id,
       // the replica should be removed from the data-node.
@@ -2283,7 +2328,7 @@ public class BlockManager {
       return null;
     }
     BlockUCState ucState = storedBlock.getBlockUCState();
-    
+
     // Block is on the NN
     if(LOG.isDebugEnabled()) {
       LOG.debug("In memory blockUCState = " + ucState);
@@ -2328,8 +2373,8 @@ public class BlockManager {
     // but now okay, it might need to be updated.
     if (reportedState == ReplicaState.FINALIZED
         && (storedBlock.findStorageInfo(storageInfo) == -1 ||
-            corruptReplicas.isReplicaCorrupt(storedBlock, dn))) {
-      toAdd.add(storedBlock);
+        corruptReplicas.isReplicaCorrupt(storedBlock, dn))) {
+      toAdd.add(new BlockInfoToAdd(storedBlock, block));
     }
     return storedBlock;
   }
@@ -2375,7 +2420,7 @@ public class BlockManager {
       if (rbi.getReportedState() == null) {
         // This is a DELETE_BLOCK request
         DatanodeStorageInfo storageInfo = rbi.getStorageInfo();
-        removeStoredBlock(rbi.getBlock(),
+        removeStoredBlock(getStoredBlock(rbi.getBlock()),
             storageInfo.getDatanodeDescriptor());
       } else {
         processAndHandleReportedBlock(rbi.getStorageInfo(),
@@ -2423,15 +2468,15 @@ public class BlockManager {
       case COMMITTED:
         if (storedBlock.getGenerationStamp() != reported.getGenerationStamp()) {
           final long reportedGS = reported.getGenerationStamp();
-          return new BlockToMarkCorrupt(storedBlock, reportedGS,
+          return new BlockToMarkCorrupt(new Block(reported), storedBlock, reportedGS,
               "block is " + ucState + " and reported genstamp " + reportedGS
-              + " does not match genstamp in block map "
-              + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
+                  + " does not match genstamp in block map "
+                  + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
         } else if (storedBlock.getNumBytes() != reported.getNumBytes()) {
-          return new BlockToMarkCorrupt(storedBlock,
+          return new BlockToMarkCorrupt(new Block(reported), storedBlock,
               "block is " + ucState + " and reported length " +
-              reported.getNumBytes() + " does not match " +
-              "length in block map " + storedBlock.getNumBytes(),
+                  reported.getNumBytes() + " does not match " +
+                  "length in block map " + storedBlock.getNumBytes(),
               Reason.SIZE_MISMATCH);
         } else {
           return null; // not corrupt
@@ -2439,11 +2484,12 @@ public class BlockManager {
       case UNDER_CONSTRUCTION:
         if (storedBlock.getGenerationStamp() > reported.getGenerationStamp()) {
           final long reportedGS = reported.getGenerationStamp();
-          return new BlockToMarkCorrupt(storedBlock, reportedGS, "block is "
-              + ucState + " and reported state " + reportedState
-              + ", But reported genstamp " + reportedGS
+          return new BlockToMarkCorrupt(new Block(reported), storedBlock,
+              reportedGS, "block is " + ucState + " and reported state "
+              + reportedState + ", But reported genstamp " + reportedGS
               + " does not match genstamp in block map "
-              + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
+              + storedBlock.getGenerationStamp(),
+              Reason.GENSTAMP_MISMATCH);
         }
         return null;
       default:
@@ -2453,12 +2499,15 @@ public class BlockManager {
     case RWR:
       if (!storedBlock.isComplete()) {
         return null; // not corrupt
-      } else if (storedBlock.getGenerationStamp() != reported.getGenerationStamp()) {
+      } else if (storedBlock.getGenerationStamp() !=
+          reported.getGenerationStamp()) {
         final long reportedGS = reported.getGenerationStamp();
-        return new BlockToMarkCorrupt(storedBlock, reportedGS,
-            "reported " + reportedState + " replica with genstamp " + reportedGS
-            + " does not match COMPLETE block's genstamp in block map "
-            + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
+        return new BlockToMarkCorrupt(
+            new Block(reported), storedBlock, reportedGS,
+            "reported " + reportedState +
+                " replica with genstamp " + reportedGS +
+                " does not match COMPLETE block's genstamp in block map " +
+                storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
       } else { // COMPLETE block, same genstamp
         if (reportedState == ReplicaState.RBW) {
           // If it's a RBW report for a COMPLETE block, it may just be that
@@ -2470,7 +2519,7 @@ public class BlockManager {
               "complete with the same genstamp");
           return null;
         } else {
-          return new BlockToMarkCorrupt(storedBlock,
+          return new BlockToMarkCorrupt(new Block(reported), storedBlock,
               "reported replica has invalid state " + reportedState,
               Reason.INVALID_STATE);
         }
@@ -2483,7 +2532,8 @@ public class BlockManager {
       " on " + dn + " size " + storedBlock.getNumBytes();
       // log here at WARN level since this is really a broken HDFS invariant
       LOG.warn(msg);
-      return new BlockToMarkCorrupt(storedBlock, msg, Reason.INVALID_STATE);
+      return new BlockToMarkCorrupt(new Block(reported), storedBlock, msg,
+          Reason.INVALID_STATE);
     }
   }
 
@@ -2516,7 +2566,7 @@ public class BlockManager {
 
     if (ucBlock.reportedState == ReplicaState.FINALIZED &&
         (block.findStorageInfo(storageInfo) < 0)) {
-      addStoredBlock(block, storageInfo, null, true);
+      addStoredBlock(block, ucBlock.reportedBlock, storageInfo, null, true);
     }
   } 
 
@@ -2531,23 +2581,23 @@ public class BlockManager {
    * 
    * @throws IOException
    */
-  private void addStoredBlockImmediate(BlockInfo storedBlock,
+  private void addStoredBlockImmediate(BlockInfo storedBlock, Block reported,
       DatanodeStorageInfo storageInfo)
-  throws IOException {
+      throws IOException {
     assert (storedBlock != null && namesystem.hasWriteLock());
-    if (!namesystem.isInStartupSafeMode() 
+    if (!namesystem.isInStartupSafeMode()
         || namesystem.isPopulatingReplQueues()) {
-      addStoredBlock(storedBlock, storageInfo, null, false);
+      addStoredBlock(storedBlock, reported, storageInfo, null, false);
       return;
     }
 
     // just add it
-    AddBlockResult result = storageInfo.addBlock(storedBlock);
+    AddBlockResult result = storageInfo.addBlock(storedBlock, reported);
 
     // Now check for completion of blocks and safe block count
     int numCurrentReplica = countLiveNodes(storedBlock);
     if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED
-        && numCurrentReplica >= minReplication) {
+        && hasMinStorage(storedBlock, numCurrentReplica)) {
       completeBlock(storedBlock.getBlockCollection(), storedBlock, false);
     } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
       // check whether safe replication is reached for the block
@@ -2561,19 +2611,20 @@ public class BlockManager {
   /**
    * Modify (block-->datanode) map. Remove block from set of
    * needed replications if this takes care of the problem.
-   * @return the block that is stored in blockMap.
+   * @return the block that is stored in blocksMap.
    */
   private Block addStoredBlock(final BlockInfo block,
-                               DatanodeStorageInfo storageInfo,
-                               DatanodeDescriptor delNodeHint,
-                               boolean logEveryBlock)
-  throws IOException {
+      final Block reportedBlock,
+      DatanodeStorageInfo storageInfo,
+      DatanodeDescriptor delNodeHint,
+      boolean logEveryBlock)
+      throws IOException {
     assert block != null && namesystem.hasWriteLock();
     BlockInfo storedBlock;
     DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
     if (block instanceof BlockInfoUnderConstruction) {
       //refresh our copy in case the block got completed in another thread
-      storedBlock = blocksMap.getStoredBlock(block);
+      storedBlock = getStoredBlock(block);
     } else {
       storedBlock = block;
     }
@@ -2587,10 +2638,9 @@ public class BlockManager {
       return block;
     }
     BlockCollection bc = storedBlock.getBlockCollection();
-    assert bc != null : "Block must belong to a file";
 
     // add block to the datanode
-    AddBlockResult result = storageInfo.addBlock(storedBlock);
+    AddBlockResult result = storageInfo.addBlock(storedBlock, reportedBlock);
 
     int curReplicaDelta;
     if (result == AddBlockResult.ADDED) {
@@ -2618,10 +2668,10 @@ public class BlockManager {
     NumberReplicas num = countNodes(storedBlock);
     int numLiveReplicas = num.liveReplicas();
     int numCurrentReplica = numLiveReplicas
-      + pendingReplications.getNumReplicas(storedBlock);
+        + pendingReplications.getNumReplicas(storedBlock);
 
     if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
-        numLiveReplicas >= minReplication) {
+        hasMinStorage(storedBlock, numLiveReplicas)) {
       storedBlock = completeBlock(bc, storedBlock, false);
     } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
       // check whether safe replication is reached for the block
@@ -2631,7 +2681,7 @@ public class BlockManager {
       // handles the safe block count maintenance.
       namesystem.incrementSafeBlockCount(numCurrentReplica);
     }
-    
+
     // if file is under construction, then done for now
     if (bc.isUnderConstruction()) {
       return storedBlock;
@@ -2643,7 +2693,7 @@ public class BlockManager {
     }
 
     // handle underReplication/overReplication
-    short fileReplication = bc.getPreferredBlockReplication();
+    short fileReplication = getExpectedReplicaNum(bc, storedBlock);
     if (!isNeededReplication(storedBlock, fileReplication, numCurrentReplica)) {
       neededReplications.remove(storedBlock, numCurrentReplica,
           num.decommissionedAndDecommissioning(), fileReplication);
@@ -2659,11 +2709,12 @@ public class BlockManager {
     int numCorruptNodes = num.corruptReplicas();
     if (numCorruptNodes != corruptReplicasCount) {
       LOG.warn("Inconsistent number of corrupt replicas for " +
-          storedBlock + "blockMap has " + numCorruptNodes + 
+          storedBlock + ". blockMap has " + numCorruptNodes +
           " but corrupt replicas map has " + corruptReplicasCount);
     }
-    if ((corruptReplicasCount > 0) && (numLiveReplicas >= fileReplication))
-      invalidateCorruptReplicas(storedBlock);
+    if ((corruptReplicasCount > 0) && (numLiveReplicas >= fileReplication)) {
+      invalidateCorruptReplicas(storedBlock, reportedBlock);
+    }
     return storedBlock;
   }
 
@@ -2695,7 +2746,7 @@ public class BlockManager {
    *
    * @param blk Block whose corrupt replicas need to be invalidated
    */
-  private void invalidateCorruptReplicas(BlockInfo blk) {
+  private void invalidateCorruptReplicas(BlockInfo blk, Block reported) {
     Collection<DatanodeDescriptor> nodes = corruptReplicas.getNodes(blk);
     boolean removedFromBlocksMap = true;
     if (nodes == null)
@@ -2705,8 +2756,8 @@ public class BlockManager {
     DatanodeDescriptor[] nodesCopy = nodes.toArray(new DatanodeDescriptor[0]);
     for (DatanodeDescriptor node : nodesCopy) {
       try {
-        if (!invalidateBlock(new BlockToMarkCorrupt(blk, null,
-              Reason.ANY), node)) {
+        if (!invalidateBlock(new BlockToMarkCorrupt(reported, blk, null,
+            Reason.ANY), node)) {
           removedFromBlocksMap = false;
         }
       } catch (IOException e) {
@@ -2874,7 +2925,7 @@ public class BlockManager {
     }
     // calculate current replication
     short expectedReplication =
-        block.getBlockCollection().getPreferredBlockReplication();
+        getExpectedReplicaNum(block.getBlockCollection(), block);
     NumberReplicas num = countNodes(block);
     int numCurrentReplica = num.liveReplicas();
     // add to under-replicated queue if need to be
@@ -2933,14 +2984,14 @@ public class BlockManager {
    * If there are any extras, call chooseExcessReplicates() to
    * mark them in the excessReplicateMap.
    */
-  private void processOverReplicatedBlock(final Block block,
+  private void processOverReplicatedBlock(final BlockInfo block,
       final short replication, final DatanodeDescriptor addedNode,
       DatanodeDescriptor delNodeHint) {
     assert namesystem.hasWriteLock();
     if (addedNode == delNodeHint) {
       delNodeHint = null;
     }
-    Collection<DatanodeStorageInfo> nonExcess = new ArrayList<DatanodeStorageInfo>();
+    Collection<DatanodeStorageInfo> nonExcess = new ArrayList<>();
     Collection<DatanodeDescriptor> corruptNodes = corruptReplicas
         .getNodes(block);
     for(DatanodeStorageInfo storage : blocksMap.getStorages(block, State.NORMAL)) {
@@ -2954,8 +3005,8 @@ public class BlockManager {
         postponeBlock(block);
         return;
       }
-      LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(cur
-          .getDatanodeUuid());
+      LightWeightLinkedSet<BlockInfo> excessBlocks = excessReplicateMap.get(
+          cur.getDatanodeUuid());
       if (excessBlocks == null || !excessBlocks.contains(block)) {
         if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
           // exclude corrupt replicas
@@ -2965,7 +3016,7 @@ public class BlockManager {
         }
       }
     }
-    chooseExcessReplicates(nonExcess, block, replication, 
+    chooseExcessReplicates(nonExcess, block, replication,
         addedNode, delNodeHint, blockplacement);
   }
 
@@ -2984,29 +3035,29 @@ public class BlockManager {
    * If no such a node is available,
    * then pick a node with least free space
    */
-  private void chooseExcessReplicates(final Collection<DatanodeStorageInfo> nonExcess, 
-                              Block b, short replication,
-                              DatanodeDescriptor addedNode,
-                              DatanodeDescriptor delNodeHint,
-                              BlockPlacementPolicy replicator) {
+  private void chooseExcessReplicates(
+      final Collection<DatanodeStorageInfo> nonExcess,
+      BlockInfo storedBlock, short replication,
+      DatanodeDescriptor addedNode,
+      DatanodeDescriptor delNodeHint,
+      BlockPlacementPolicy replicator) {
     assert namesystem.hasWriteLock();
     // first form a rack to datanodes map and
-    BlockCollection bc = getBlockCollection(b);
-    final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(bc.getStoragePolicyID());
+    BlockCollection bc = getBlockCollection(storedBlock);
+    final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(
+        bc.getStoragePolicyID());
     final List<StorageType> excessTypes = storagePolicy.chooseExcess(
         replication, DatanodeStorageInfo.toStorageTypes(nonExcess));
 
+    final Map<String, List<DatanodeStorageInfo>> rackMap = new HashMap<>();
+    final List<DatanodeStorageInfo> moreThanOne = new ArrayList<>();
+    final List<DatanodeStorageInfo> exactlyOne = new ArrayList<>();
 
-    final Map<String, List<DatanodeStorageInfo>> rackMap
-        = new HashMap<String, List<DatanodeStorageInfo>>();
-    final List<DatanodeStorageInfo> moreThanOne = new ArrayList<DatanodeStorageInfo>();
-    final List<DatanodeStorageInfo> exactlyOne = new ArrayList<DatanodeStorageInfo>();
-    
     // split nodes into two sets
     // moreThanOne contains nodes on rack with more than one replica
     // exactlyOne contains the remaining nodes
     replicator.splitNodesWithRack(nonExcess, rackMap, moreThanOne, exactlyOne);
-    
+
     // pick one node to delete that favors the delete hint
     // otherwise pick one with least space from priSet if it is not empty
     // otherwise one node with least space from remains
@@ -3021,7 +3072,7 @@ public class BlockManager {
           moreThanOne, excessTypes)) {
         cur = delNodeHintStorage;
       } else { // regular excessive replica removal
-        cur = replicator.chooseReplicaToDelete(bc, b, replication,
+        cur = replicator.chooseReplicaToDelete(bc, storedBlock, replication,
             moreThanOne, exactlyOne, excessTypes);
       }
       firstOne = false;
@@ -3030,24 +3081,29 @@ public class BlockManager {
       replicator.adjustSetsWithChosenReplica(rackMap, moreThanOne,
           exactlyOne, cur);
 
-      nonExcess.remove(cur);
-      addToExcessReplicate(cur.getDatanodeDescriptor(), b);
-
-      //
-      // The 'excessblocks' tracks blocks until we get confirmation
-      // that the datanode has deleted them; the only way we remove them
-      // is when we get a "removeBlock" message.  
-      //
-      // The 'invalidate' list is used to inform the datanode the block 
-      // should be deleted.  Items are removed from the invalidate list
-      // upon giving instructions to the namenode.
-      //
-      addToInvalidates(b, cur.getDatanodeDescriptor());
-      blockLog.info("BLOCK* chooseExcessReplicates: "
-                +"({}, {}) is added to invalidated blocks set", cur, b);
+      processChosenExcessReplica(nonExcess, cur, storedBlock);
     }
   }
 
+  private void processChosenExcessReplica(
+      final Collection<DatanodeStorageInfo> nonExcess,
+      final DatanodeStorageInfo chosen, BlockInfo storedBlock) {
+    nonExcess.remove(chosen);
+    addToExcessReplicate(chosen.getDatanodeDescriptor(), storedBlock);
+    //
+    // The 'excessblocks' tracks blocks until we get confirmation
+    // that the datanode has deleted them; the only way we remove them
+    // is when we get a "removeBlock" message.
+    //
+    // The 'invalidate' list is used to inform the datanode the block
+    // should be deleted.  Items are removed from the invalidate list
+    // upon giving instructions to the datanodes.
+    //
+    addToInvalidates(storedBlock, chosen.getDatanodeDescriptor());
+    blockLog.info("BLOCK* chooseExcessReplicates: "
+        +"({}, {}) is added to invalidated blocks set", chosen, storedBlock);
+  }
+
   /** Check if we can use delHint */
   static boolean useDelHint(boolean isFirst, DatanodeStorageInfo delHint,
       DatanodeStorageInfo added, List<DatanodeStorageInfo> moreThan1Racks,
@@ -3069,17 +3125,18 @@ public class BlockManager {
     }
   }
 
-  private void addToExcessReplicate(DatanodeInfo dn, Block block) {
+  private void addToExcessReplicate(DatanodeInfo dn, BlockInfo storedBlock) {
     assert namesystem.hasWriteLock();
-    LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(dn.getDatanodeUuid());
+    LightWeightLinkedSet<BlockInfo> excessBlocks = excessReplicateMap.get(
+        dn.getDatanodeUuid());
     if (excessBlocks == null) {
-      excessBlocks = new LightWeightLinkedSet<Block>();
+      excessBlocks = new LightWeightLinkedSet<>();
       excessReplicateMap.put(dn.getDatanodeUuid(), excessBlocks);
     }
-    if (excessBlocks.add(block)) {
+    if (excessBlocks.add(storedBlock)) {
       excessBlocksCount.incrementAndGet();
       blockLog.debug("BLOCK* addToExcessReplicate: ({}, {}) is added to"
-          + " excessReplicateMap", dn, block);
+          + " excessReplicateMap", dn, storedBlock);
     }
   }
 
@@ -3091,26 +3148,26 @@ public class BlockManager {
           QUEUE_REASON_FUTURE_GENSTAMP);
       return;
     }
-    removeStoredBlock(block, node);
+    removeStoredBlock(getStoredBlock(block), node);
   }
 
   /**
    * Modify (block-->datanode) map. Possibly generate replication tasks, if the
    * removed block is still valid.
    */
-  public void removeStoredBlock(Block block, DatanodeDescriptor node) {
-    blockLog.debug("BLOCK* removeStoredBlock: {} from {}", block, node);
+  public void removeStoredBlock(BlockInfo storedBlock,
+      DatanodeDescriptor node) {
+    blockLog.debug("BLOCK* removeStoredBlock: {} from {}", storedBlock, node);
     assert (namesystem.hasWriteLock());
     {
-      BlockInfo storedBlock = getStoredBlock(block);
       if (storedBlock == null || !blocksMap.removeNode(storedBlock, node)) {
         blockLog.debug("BLOCK* removeStoredBlock: {} has already been" +
-            " removed from node {}", block, node);
+            " removed from node {}", storedBlock, node);
         return;
       }
 
       CachedBlock cblock = namesystem.getCacheManager().getCachedBlocks()
-          .get(new CachedBlock(block.getBlockId(), (short) 0, false));
+          .get(new CachedBlock(storedBlock.getBlockId(), (short) 0, false));
       if (cblock != null) {
         boolean removed = false;
         removed |= node.getPendingCached().remove(cblock);
@@ -3118,7 +3175,7 @@ public class BlockManager {
         removed |= node.getPendingUncached().remove(cblock);
         if (removed) {
           blockLog.debug("BLOCK* removeStoredBlock: {} removed from caching "
-              + "related lists on node {}", block, node);
+              + "related lists on node {}", storedBlock, node);
         }
       }
 
@@ -3128,7 +3185,7 @@ public class BlockManager {
       // necessary. In that case, put block on a possibly-will-
       // be-replicated list.
       //
-      BlockCollection bc = blocksMap.getBlockCollection(block);
+      BlockCollection bc = storedBlock.getBlockCollection();
       if (bc != null) {
         namesystem.decrementSafeBlockCount(storedBlock);
         updateNeededReplications(storedBlock, -1, 0);
@@ -3138,13 +3195,13 @@ public class BlockManager {
       // We've removed a block from a node, so it's definitely no longer
       // in "excess" there.
       //
-      LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(node
-          .getDatanodeUuid());
+      LightWeightLinkedSet<BlockInfo> excessBlocks = excessReplicateMap.get(
+          node.getDatanodeUuid());
       if (excessBlocks != null) {
-        if (excessBlocks.remove(block)) {
+        if (excessBlocks.remove(storedBlock)) {
           excessBlocksCount.decrementAndGet();
           blockLog.debug("BLOCK* removeStoredBlock: {} is removed from " +
-              "excessBlocks", block);
+              "excessBlocks", storedBlock);
           if (excessBlocks.size() == 0) {
             excessReplicateMap.remove(node.getDatanodeUuid());
           }
@@ -3152,7 +3209,7 @@ public class BlockManager {
       }
 
       // Remove the replica from corruptReplicas
-      corruptReplicas.removeFromCorruptReplicasMap(block, node);
+      corruptReplicas.removeFromCorruptReplicasMap(storedBlock, node);
     }
   }
 
@@ -3160,7 +3217,7 @@ public class BlockManager {
    * Get all valid locations of the block & add the block to results
    * return the length of the added block; 0 if the block is not added
    */
-  private long addBlock(Block block, List<BlockWithLocations> results) {
+  private long addBlock(BlockInfo block, List<BlockWithLocations> results) {
     final List<DatanodeStorageInfo> locations = getValidLocations(block);
     if(locations.size() == 0) {
       return 0;
@@ -3212,31 +3269,32 @@ public class BlockManager {
     processAndHandleReportedBlock(storageInfo, block, ReplicaState.FINALIZED,
         delHintNode);
   }
-  
+
   private void processAndHandleReportedBlock(
       DatanodeStorageInfo storageInfo, Block block,
       ReplicaState reportedState, DatanodeDescriptor delHintNode)
       throws IOException {
     // blockReceived reports a finalized block
-    Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
-    Collection<Block> toInvalidate = new LinkedList<Block>();
-    Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
-    Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
+    Collection<BlockInfoToAdd> toAdd = new LinkedList<>();
+    Collection<Block> toInvalidate = new LinkedList<>();
+    Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<>();
+    Collection<StatefulBlockInfo> toUC = new LinkedList<>();
     final DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
 
-    processReportedBlock(storageInfo, block, reportedState,
-                              toAdd, toInvalidate, toCorrupt, toUC);
+    processReportedBlock(storageInfo, block, reportedState, toAdd, toInvalidate,
+        toCorrupt, toUC);
     // the block is only in one of the to-do lists
     // if it is in none then data-node already has it
     assert toUC.size() + toAdd.size() + toInvalidate.size() + toCorrupt.size() <= 1
-      : "The block should be only in one of the lists.";
+        : "The block should be only in one of the lists.";
 
-    for (StatefulBlockInfo b : toUC) { 
+    for (StatefulBlockInfo b : toUC) {
       addStoredBlockUnderConstruction(b, storageInfo);
     }
     long numBlocksLogged = 0;
-    for (BlockInfo b : toAdd) {
-      addStoredBlock(b, storageInfo, delHintNode, numBlocksLogged < maxNumBlocksToLog);
+    for (BlockInfoToAdd b : toAdd) {
+      addStoredBlock(b.getStored(), b.getReported(), storageInfo, delHintNode,
+          numBlocksLogged < maxNumBlocksToLog);
       numBlocksLogged++;
     }
     if (numBlocksLogged > maxNumBlocksToLog) {
@@ -3301,7 +3359,7 @@ public class BlockManager {
                                       ReplicaState.RBW, null);
         break;
       default:
-        String msg = 
+        String msg =
           "Unknown block status code reported by " + nodeID +
           ": " + rdbi;
         blockLog.warn(msg);
@@ -3337,8 +3395,8 @@ public class BlockManager {
       } else if (node.isDecommissioned()) {
         decommissioned++;
       } else {
-        LightWeightLinkedSet<Block> blocksExcess = excessReplicateMap.get(node
-            .getDatanodeUuid());
+        LightWeightLinkedSet<BlockInfo> blocksExcess =
+            excessReplicateMap.get(node.getDatanodeUuid());
         if (blocksExcess != null && blocksExcess.contains(b)) {
           excess++;
         } else {
@@ -3391,13 +3449,13 @@ public class BlockManager {
     int numOverReplicated = 0;
     while(it.hasNext()) {
       final BlockInfo block = it.next();
-      BlockCollection bc = blocksMap.getBlockCollection(block);
-      short expectedReplication = bc.getPreferredBlockReplication();
+      int expectedReplication = this.getReplication(block);
       NumberReplicas num = countNodes(block);
       int numCurrentReplica = num.liveReplicas();
       if (numCurrentReplica > expectedReplication) {
         // over-replicated block 
-        processOverReplicatedBlock(block, expectedReplication, null, null);
+        processOverReplicatedBlock(block, (short) expectedReplication, null,
+            null);
         numOverReplicated++;
       }
     }
@@ -3423,7 +3481,7 @@ public class BlockManager {
     if (pendingReplicationBlocksCount == 0 &&
         underReplicatedBlocksCount == 0) {
       LOG.info("Node {} is dead and there are no under-replicated" +
-          " blocks or blocks pending replication. Safe to decommission.", 
+          " blocks or blocks pending replication. Safe to decommission.",
           node);
       return true;
     }
@@ -3441,6 +3499,12 @@ public class BlockManager {
     return blocksMap.size();
   }
 
+
+  /** @return an iterator of the datanodes. */
+  public Iterable<DatanodeStorageInfo> getStorages(final Block block) {
+    return blocksMap.getStorages(block);
+  }
+
   public DatanodeStorageInfo[] getStorages(BlockInfo block) {
     final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[block.numNodes()];
     int i = 0;
@@ -3529,10 +3593,12 @@ public class BlockManager {
         final BlockInfoUnderConstruction uc =
             (BlockInfoUnderConstruction)b;
         final int numNodes = b.numNodes();
-        LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = "
-          + uc.getBlockUCState() + ", replication# = " + numNodes
-          + (numNodes < minReplication ? " < ": " >= ")
-          + " minimum = " + minReplication + ") in file " + src);
+        final int min = getMinStorageNum(b);
+        final BlockUCState state = b.getBlockUCState();
+        LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = " + state
+            + ", replication# = " + numNodes
+            + (numNodes < min ? " < " : " >= ")
+            + " minimum = " + min + ") in file " + src);
         return false;
       }
     }
@@ -3543,15 +3609,15 @@ public class BlockManager {
    * @return 0 if the block is not found;
    *         otherwise, return the replication factor of the block.
    */
-  private int getReplication(Block block) {
+  private int getReplication(BlockInfo block) {
     final BlockCollection bc = blocksMap.getBlockCollection(block);
-    return bc == null? 0: bc.getPreferredBlockReplication();
+    return bc == null? 0: getExpectedReplicaNum(bc, block);
   }
 
 
   /**
-   * Get blocks to invalidate for <i>nodeId</i>
-   * in {@link #invalidateBlocks}.
+   * Get blocks to invalidate for <i>nodeId</i>.
+   * in {@link #invalidateBlocks}.boolean blockHasEnoughRacks
    *
    * @return number of blocks scheduled for removal during this iteration.
    */
@@ -3589,22 +3655,20 @@ public class BlockManager {
     return toInvalidate.size();
   }
 
-  boolean blockHasEnoughRacks(Block b) {
+  boolean blockHasEnoughRacks(BlockInfo storedBlock, int expectedStorageNum) {
     if (!this.shouldCheckForEnoughRacks) {
       return true;
     }
-    boolean enoughRacks = false;;
-    Collection<DatanodeDescriptor> corruptNodes = 
-                                  corruptReplicas.getNodes(b);
-    int numExpectedReplicas = getReplication(b);
+    boolean enoughRacks = false;
+    Collection<DatanodeDescriptor> corruptNodes =
+        corruptReplicas.getNodes(storedBlock);
     String rackName = null;
-    for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
+    for(DatanodeStorageInfo storage : getStorages(storedBlock)) {
       final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
       if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
         if ((corruptNodes == null ) || !corruptNodes.contains(cur)) {
-          if (numExpectedReplicas == 1 ||
-              (numExpectedReplicas > 1 &&
-                  !datanodeManager.hasClusterEverBeenMultiRack())) {
+          if (expectedStorageNum == 1 || (expectedStorageNum > 1 &&
+              !datanodeManager.hasClusterEverBeenMultiRack())) {
             enoughRacks = true;
             break;
           }
@@ -3625,8 +3689,13 @@ public class BlockManager {
    * A block needs replication if the number of replicas is less than expected
    * or if it does not have enough racks.
    */
-  boolean isNeededReplication(Block b, int expected, int current) {
-    return current < expected || !blockHasEnoughRacks(b);
+  boolean isNeededReplication(BlockInfo storedBlock, int expected,
+      int current) {
+    return current < expected || !blockHasEnoughRacks(storedBlock, expected);
+  }
+
+  public short getExpectedReplicaNum(BlockCollection bc, BlockInfo block) {
+    return bc.getPreferredBlockReplication();
   }
   
   public long getMissingBlocksCount() {
@@ -3648,11 +3717,6 @@ public class BlockManager {
     return blocksMap.getBlockCollection(b);
   }
 
-  /** @return an iterator of the datanodes. */
-  public Iterable<DatanodeStorageInfo> getStorages(final Block block) {
-    return blocksMap.getStorages(block);
-  }
-
   public int numCorruptReplicas(Block block) {
     return corruptReplicas.numCorruptReplicas(block);
   }
@@ -3668,9 +3732,10 @@ public class BlockManager {
    * If a block is removed from blocksMap, remove it from excessReplicateMap.
    */
   private void removeFromExcessReplicateMap(Block block) {
-    for (DatanodeStorageInfo info : blocksMap.getStorages(block)) {
+    for (DatanodeStorageInfo info : getStorages(block)) {
       String uuid = info.getDatanodeDescriptor().getDatanodeUuid();
-      LightWeightLinkedSet<Block> excessReplicas = excessReplicateMap.get(uuid);
+      LightWeightLinkedSet<BlockInfo> excessReplicas =
+          excessReplicateMap.get(uuid);
       if (excessReplicas != null) {
         if (excessReplicas.remove(block)) {
           excessBlocksCount.decrementAndGet();
@@ -3685,26 +3750,6 @@ public class BlockManager {
   public int getCapacity() {
     return blocksMap.getCapacity();
   }
-  
-  /**
-   * Return a range of corrupt replica block ids. Up to numExpectedBlocks 
-   * blocks starting at the next block after startingBlockId are returned
-   * (fewer if numExpectedBlocks blocks are unavailable). If startingBlockId 
-   * is null, up to numExpectedBlocks blocks are returned from the beginning.
-   * If startingBlockId cannot be found, null is returned.
-   *
-   * @param numExpectedBlocks Number of block ids to return.
-   *  0 <= numExpectedBlocks <= 100
-   * @param startingBlockId Block id from which to start. If null, start at
-   *  beginning.
-   * @return Up to numExpectedBlocks blocks from startingBlockId if it exists
-   *
-   */
-  public long[] getCorruptReplicaBlockIds(int numExpectedBlocks,
-                                   Long startingBlockId) {
-    return corruptReplicas.getCorruptReplicaBlockIds(numExpectedBlocks,
-                                                     startingBlockId);
-  }
 
   /**
    * Return an iterator over the set of blocks for which there are no replicas.
@@ -3879,7 +3924,7 @@ public class BlockManager {
 
   /**
    * A simple result enum for the result of
-   * {@link BlockManager#processMisReplicatedBlock(BlockInfo)}.
+   * {@link BlockManager#processMisReplicatedBlock}.
    */
   enum MisReplicationResult {
     /** The block should be invalidated since it belongs to a deleted file. */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/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 57e81b4..65b83e1 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
@@ -24,6 +24,7 @@ import java.util.List;
 import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
@@ -226,7 +227,7 @@ public class DatanodeStorageInfo {
     return blockPoolUsed;
   }
 
-  public AddBlockResult addBlock(BlockInfo b) {
+  public AddBlockResult addBlock(BlockInfo b, Block reportedBlock) {
     // First check whether the block belongs to a different storage
     // on the same DN.
     AddBlockResult result = AddBlockResult.ADDED;
@@ -245,10 +246,18 @@ public class DatanodeStorageInfo {
     }
 
     // add to the head of the data-node list
-    b.addStorage(this);
+    b.addStorage(this, reportedBlock);
+    insertToList(b);
+    return result;
+  }
+
+  AddBlockResult addBlock(BlockInfo b) {
+    return addBlock(b, b);
+  }
+
+  public void insertToList(BlockInfo b) {
     blockList = b.listInsert(blockList, this);
     numBlocks++;
-    return result;
   }
 
   public boolean removeBlock(BlockInfo b) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/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 b073a89..d0f4e08 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
@@ -143,7 +143,6 @@ import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
@@ -3136,7 +3135,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (trackBlockCounts) {
         if (b.isComplete()) {
           numRemovedComplete++;
-          if (blockManager.checkMinReplication(b)) {
+          if (blockManager.hasMinStorage(b)) {
             numRemovedSafe++;
           }
         }
@@ -3368,7 +3367,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       curBlock = blocks[nrCompleteBlocks];
       if(!curBlock.isComplete())
         break;
-      assert blockManager.checkMinReplication(curBlock) :
+      assert blockManager.hasMinStorage(curBlock) :
               "A COMPLETE block is not minimally replicated in " + src;
     }
 
@@ -3404,7 +3403,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     // If penultimate block doesn't exist then its minReplication is met
     boolean penultimateBlockMinReplication = penultimateBlock == null ? true :
-        blockManager.checkMinReplication(penultimateBlock);
+        blockManager.hasMinStorage(penultimateBlock);
 
     switch(lastBlockState) {
     case COMPLETE:
@@ -3413,7 +3412,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     case COMMITTED:
       // Close file if committed blocks are minimally replicated
       if(penultimateBlockMinReplication &&
-          blockManager.checkMinReplication(lastBlock)) {
+          blockManager.hasMinStorage(lastBlock)) {
         finalizeINodeFileUnderConstruction(src, pendingFile,
             iip.getLatestSnapshotId());
         NameNode.stateChangeLog.warn("BLOCK*"
@@ -3705,9 +3704,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                 trimmedTargets.get(i).getStorageInfo(trimmedStorages.get(i));
             if (storageInfo != null) {
               if(copyTruncate) {
-                storageInfo.addBlock(truncatedBlock);
+                storageInfo.addBlock(truncatedBlock, truncatedBlock);
               } else {
-                storageInfo.addBlock(storedBlock);
+                storageInfo.addBlock(storedBlock, storedBlock);
               }
             }
           }
@@ -3723,8 +3722,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         } else {
           iFile.setLastBlock(storedBlock, trimmedStorageInfos);
           if (closeFile) {
-            blockManager.markBlockReplicasAsCorrupt(storedBlock,
-                oldGenerationStamp, oldNumBytes, trimmedStorageInfos);
+            blockManager.markBlockReplicasAsCorrupt(oldBlock.getLocalBlock(),
+                storedBlock, oldGenerationStamp, oldNumBytes,
+                trimmedStorageInfos);
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 7d4cd7e..ab179b4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -647,7 +647,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
                   .getStorageType()));
             }
             if (showReplicaDetails) {
-              LightWeightLinkedSet<Block> blocksExcess =
+              LightWeightLinkedSet<BlockInfo> blocksExcess =
                   bm.excessReplicateMap.get(dnDesc.getDatanodeUuid());
               Collection<DatanodeDescriptor> corruptReplicas =
                   bm.getCorruptReplicas(block.getLocalBlock());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
index 5126aa7..bae4f1d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
@@ -63,7 +63,7 @@ public class TestBlockInfo {
 
     final DatanodeStorageInfo storage = DFSTestUtil.createDatanodeStorageInfo("storageID", "127.0.0.1");
 
-    boolean added = blockInfo.addStorage(storage);
+    boolean added = blockInfo.addStorage(storage, blockInfo);
 
     Assert.assertTrue(added);
     Assert.assertEquals(storage, blockInfo.getStorageInfo(0));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/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 396dff3..9e31670 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
@@ -383,7 +383,7 @@ public class TestBlockManager {
     for (int i = 1; i < pipeline.length; i++) {
       DatanodeStorageInfo storage = pipeline[i];
       bm.addBlock(storage, blockInfo, null);
-      blockInfo.addStorage(storage);
+      blockInfo.addStorage(storage, blockInfo);
     }
   }
 
@@ -393,7 +393,7 @@ public class TestBlockManager {
 
     for (DatanodeDescriptor dn : nodes) {
       for (DatanodeStorageInfo storage : dn.getStorageInfos()) {
-        blockInfo.addStorage(storage);
+        blockInfo.addStorage(storage, blockInfo);
       }
     }
     return blockInfo;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
index 1c3f075..c33667d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
@@ -100,7 +100,7 @@ public class TestNodeCount {
       DatanodeDescriptor nonExcessDN = null;
       for(DatanodeStorageInfo storage : bm.blocksMap.getStorages(block.getLocalBlock())) {
         final DatanodeDescriptor dn = storage.getDatanodeDescriptor();
-        Collection<Block> blocks = bm.excessReplicateMap.get(dn.getDatanodeUuid());
+        Collection<BlockInfo> blocks = bm.excessReplicateMap.get(dn.getDatanodeUuid());
         if (blocks == null || !blocks.contains(block.getLocalBlock()) ) {
           nonExcessDN = dn;
           break;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
index 2d7bb44..83b3aa0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -42,7 +41,6 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
-import org.apache.hadoop.util.Time;
 import org.junit.Test;
 
 public class TestOverReplicatedBlocks {
@@ -185,7 +183,7 @@ public class TestOverReplicatedBlocks {
       // All replicas for deletion should be scheduled on lastDN.
       // And should not actually be deleted, because lastDN does not heartbeat.
       namesystem.readLock();
-      Collection<Block> dnBlocks = 
+      Collection<BlockInfo> dnBlocks =
         namesystem.getBlockManager().excessReplicateMap.get(lastDNid);
       assertEquals("Replicas on node " + lastDNid + " should have been deleted",
           SMALL_FILE_LENGTH / SMALL_BLOCK_SIZE, dnBlocks.size());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index 2812957..44f0e65 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
@@ -1250,7 +1250,7 @@ public class TestReplicationPolicy {
     when(storage.removeBlock(any(BlockInfo.class))).thenReturn(true);
     when(storage.addBlock(any(BlockInfo.class))).thenReturn
         (DatanodeStorageInfo.AddBlockResult.ADDED);
-    ucBlock.addStorage(storage);
+    ucBlock.addStorage(storage, ucBlock);
 
     when(mbc.setLastBlock((BlockInfo) any(), (DatanodeStorageInfo[]) any()))
     .thenReturn(ucBlock);


[17/50] hadoop git commit: YARN-3860. rmadmin -transitionToActive should check the state of non-target node. (Contributed by Masatake Iwasaki)

Posted by vv...@apache.org.
YARN-3860. rmadmin -transitionToActive should check the state of non-target node. (Contributed by Masatake Iwasaki)


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

Branch: refs/heads/YARN-2139
Commit: a95d39f9d08b3b215a1b33e77e9ab8a2ee59b3a9
Parents: aad6a7d
Author: Junping Du <ju...@apache.org>
Authored: Sun Jun 28 22:26:47 2015 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Sun Jun 28 22:26:47 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                           |  3 +++
 .../org/apache/hadoop/yarn/client/cli/RMAdminCLI.java     | 10 +++++++++-
 .../org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java |  4 ++++
 3 files changed, 16 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a95d39f9/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 714bfdc..3ebf799 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -565,6 +565,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3859. LeafQueue doesn't print user properly for application add.
     (Varun Saxena via devaraj)
 
+    YARN-3860. rmadmin -transitionToActive should check the state of non-target node.
+    (Masatake Iwasaki via junping_du)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a95d39f9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
index 932b62d..36d29d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
@@ -797,7 +797,15 @@ public class RMAdminCLI extends HAAdmin {
           "Could not connect to RM HA Admin for node " + rmId);
     }
   }
-  
+
+  /**
+   * returns the list of all resourcemanager ids for the given configuration.
+   */
+  @Override
+  protected Collection<String> getTargetIds(String targetNodeToActivate) {
+    return HAUtil.getRMHAIds(getConf());
+  }
+
   @Override
   protected String getUsageString() {
     return "Usage: rmadmin";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a95d39f9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java
index a95412c..6dc326a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java
@@ -27,6 +27,7 @@ import static org.mockito.Matchers.argThat;
 import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
@@ -116,6 +117,7 @@ public class TestRMAdminCLI {
 
     YarnConfiguration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.RM_HA_ENABLED, true);
+    conf.set(YarnConfiguration.RM_HA_IDS, "rm1,rm2");
     rmAdminCLIWithHAEnabled = new RMAdminCLI(conf) {
 
       @Override
@@ -259,6 +261,8 @@ public class TestRMAdminCLI {
     assertEquals(0, rmAdminCLIWithHAEnabled.run(args));
     verify(haadmin).transitionToActive(
         any(HAServiceProtocol.StateChangeRequestInfo.class));
+    // HAAdmin#isOtherTargetNodeActive should check state of non-target node.
+    verify(haadmin, times(1)).getServiceStatus();
   }
 
   @Test(timeout = 500)


[11/50] hadoop git commit: YARN-2871. TestRMRestart#testRMRestartGetApplicationList sometime fails in trunk. Contributed by zhihai xu

Posted by vv...@apache.org.
YARN-2871. TestRMRestart#testRMRestartGetApplicationList sometime fails
in trunk. Contributed by zhihai xu


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

Branch: refs/heads/YARN-2139
Commit: fe6c1bd73aee188ed58df4d33bbc2d2fe0779a97
Parents: aa07dea
Author: Xuan <xg...@apache.org>
Authored: Fri Jun 26 19:43:59 2015 -0700
Committer: Xuan <xg...@apache.org>
Committed: Fri Jun 26 19:43:59 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                              | 3 +++
 .../hadoop/yarn/server/resourcemanager/TestRMRestart.java    | 8 +++++---
 2 files changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe6c1bd7/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index d74e8a9..e2bf706 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -559,6 +559,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3745. SerializedException should also try to instantiate internal
     exception with the default constructor. (Lavkesh Lahngir via devaraj)
 
+    YARN-2871. TestRMRestart#testRMRestartGetApplicationList sometime fails in trunk.
+    (zhihai xu via xgong)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe6c1bd7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.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/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
index 628e9be..de17acd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager;
 import static org.mockito.Matchers.isA;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.timeout;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
@@ -976,9 +977,10 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     List<ApplicationReport> appList2 = response2.getApplicationList();
     Assert.assertTrue(3 == appList2.size());
 
-    // check application summary is logged for the completed apps after RM restart.
-    verify(rm2.getRMAppManager(), times(3)).logApplicationSummary(
-      isA(ApplicationId.class));
+    // check application summary is logged for the completed apps with timeout
+    // to make sure APP_COMPLETED events are processed, after RM restart.
+    verify(rm2.getRMAppManager(), timeout(1000).times(3)).
+        logApplicationSummary(isA(ApplicationId.class));
   }
 
   private MockAM launchAM(RMApp app, MockRM rm, MockNM nm)


[36/50] hadoop git commit: Revert "HADOOP-12009 Clarify FileSystem.listStatus() sorting order & fix FileSystemContractBaseTest:testListStatus. (J.Andreina via stevel)"

Posted by vv...@apache.org.
Revert "HADOOP-12009 Clarify FileSystem.listStatus() sorting order & fix  FileSystemContractBaseTest:testListStatus. (J.Andreina via stevel)"

This reverts commit 3dfa8161f9412bcb040f3c29c471344f25f24337.


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

Branch: refs/heads/YARN-2139
Commit: 076948d9a4053cc8be1927005c797273bae85e99
Parents: 7ba5bba
Author: Steve Loughran <st...@apache.org>
Authored: Tue Jun 30 11:54:12 2015 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Jun 30 11:54:12 2015 +0100

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt    |  4 ----
 .../main/java/org/apache/hadoop/fs/FileSystem.java | 17 +----------------
 .../src/site/markdown/filesystem/filesystem.md     |  4 ----
 .../hadoop/fs/FileSystemContractBaseTest.java      | 11 +++--------
 4 files changed, 4 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/076948d9/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 2cf9509..c010ff1 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -664,10 +664,6 @@ Release 2.8.0 - UNRELEASED
 
     HADOOP-11958. MetricsSystemImpl fails to show backtrace when an error
     occurs (Jason Lowe via jeagles)
-
-    HADOOP-12009 Clarify FileSystem.listStatus() sorting order & fix
-    FileSystemContractBaseTest:testListStatus. (J.Andreina via stevel)
-
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/076948d9/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
index c98074a..c73caf7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
@@ -1498,9 +1498,7 @@ public abstract class FileSystem extends Configured implements Closeable {
   /**
    * List the statuses of the files/directories in the given path if the path is
    * a directory.
-   * <p>
-   * Does not guarantee to return the List of files/directories status in a
-   * sorted order.
+   * 
    * @param f given path
    * @return the statuses of the files/directories in the given patch
    * @throws FileNotFoundException when the path does not exist;
@@ -1542,9 +1540,6 @@ public abstract class FileSystem extends Configured implements Closeable {
   /**
    * Filter files/directories in the given path using the user-supplied path
    * filter.
-   * <p>
-   * Does not guarantee to return the List of files/directories status in a
-   * sorted order.
    * 
    * @param f
    *          a path name
@@ -1565,9 +1560,6 @@ public abstract class FileSystem extends Configured implements Closeable {
   /**
    * Filter files/directories in the given list of paths using default
    * path filter.
-   * <p>
-   * Does not guarantee to return the List of files/directories status in a
-   * sorted order.
    * 
    * @param files
    *          a list of paths
@@ -1584,9 +1576,6 @@ public abstract class FileSystem extends Configured implements Closeable {
   /**
    * Filter files/directories in the given list of paths using user-supplied
    * path filter.
-   * <p>
-   * Does not guarantee to return the List of files/directories status in a
-   * sorted order.
    * 
    * @param files
    *          a list of paths
@@ -1747,8 +1736,6 @@ public abstract class FileSystem extends Configured implements Closeable {
    * while consuming the entries. Each file system implementation should
    * override this method and provide a more efficient implementation, if
    * possible. 
-   * Does not guarantee to return the iterator that traverses statuses
-   * of the files in a sorted order.
    *
    * @param p target path
    * @return remote iterator
@@ -1776,8 +1763,6 @@ public abstract class FileSystem extends Configured implements Closeable {
 
   /**
    * List the statuses and block locations of the files in the given path.
-   * Does not guarantee to return the iterator that traverses statuses
-   * of the files in a sorted order.
    * 
    * If the path is a directory, 
    *   if recursive is false, returns files in the directory;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/076948d9/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 f323374..84e3755 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
@@ -183,10 +183,6 @@ to the same path:
     forall fs in listStatus(Path) :
       fs == getFileStatus(fs.path)
 
-**Ordering of results**: there is no guarantee of ordering of the listed entries.
-While HDFS currently returns an alphanumerically sorted list, neither the Posix `readdir()`
-nor Java's `File.listFiles()` API calls define any ordering of returned values. Applications
-which require a uniform sort order on the results must perform the sorting themselves.
 
 ### Atomicity and Consistency
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/076948d9/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
index 495af00..2ca81e9 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.fs;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.util.ArrayList;
 
 import junit.framework.TestCase;
 
@@ -225,13 +224,9 @@ public abstract class FileSystemContractBaseTest extends TestCase {
 
     paths = fs.listStatus(path("/test/hadoop"));
     assertEquals(3, paths.length);
-    ArrayList<String> list = new ArrayList<String>();
-    for (FileStatus fileState : paths) {
-      list.add(fileState.getPath().toString());
-    }
-    assertTrue(list.contains(path("/test/hadoop/a")));
-    assertTrue(list.contains(path("/test/hadoop/b")));
-    assertTrue(list.contains(path("/test/hadoop/c")));
+    assertEquals(path("/test/hadoop/a"), paths[0].getPath());
+    assertEquals(path("/test/hadoop/b"), paths[1].getPath());
+    assertEquals(path("/test/hadoop/c"), paths[2].getPath());
 
     paths = fs.listStatus(path("/test/hadoop/a"));
     assertEquals(0, paths.length);


[41/50] hadoop git commit: HDFS-8635. Migrate HDFS native build to new CMake framework (Alan Burlison via Colin P. McCabe)

Posted by vv...@apache.org.
HDFS-8635. Migrate HDFS native build to new CMake framework (Alan Burlison via Colin P. McCabe)


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

Branch: refs/heads/YARN-2139
Commit: 29df73dc0572cf8e450568f1d0301e8d55d64b48
Parents: 9ee7b6e
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Tue Jun 30 15:54:38 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Tue Jun 30 15:54:38 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hadoop-hdfs/src/CMakeLists.txt              | 129 ++++++-------------
 .../src/contrib/libwebhdfs/CMakeLists.txt       |   8 +-
 .../libwebhdfs/resources/FindJansson.cmake      |   9 +-
 .../src/main/native/fuse-dfs/CMakeLists.txt     |  91 +++++++------
 5 files changed, 100 insertions(+), 140 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/29df73dc/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index b065f98..65f286d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -691,6 +691,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8493. Consolidate truncate() related implementation in a single class.
     (Rakesh R via wheat9)
 
+    HDFS-8635. Migrate HDFS native build to new CMake framework (Alan Burlison
+    via Colin P. McCabe)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/29df73dc/hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt
index 563727b..b4a3b40 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt
@@ -18,106 +18,59 @@
 
 cmake_minimum_required(VERSION 2.6 FATAL_ERROR)
 
-# Default to release builds
-set(CMAKE_BUILD_TYPE, Release)
+list(APPEND CMAKE_MODULE_PATH ${CMAKE_SOURCE_DIR}/../../../hadoop-common-project/hadoop-common)
+include(HadoopCommon)
 
-include(../../../hadoop-common-project/hadoop-common/src/JNIFlags.cmake NO_POLICY_SCOPE)
-
-# Compile a library with both shared and static variants
-function(add_dual_library LIBNAME)
-    add_library(${LIBNAME} SHARED ${ARGN})
-    add_library(${LIBNAME}_static STATIC ${ARGN})
-    # Linux builds traditionally ship a libhdfs.a (static linking) and libhdfs.so
-    # (dynamic linking).  On Windows, we cannot use the same base name for both
-    # static and dynamic, because Windows does not use distinct file extensions
-    # for a statically linked library vs. a DLL import library.  Both use the
-    # .lib extension.  On Windows, we'll build the static library as
-    # hdfs_static.lib.
-    if (NOT WIN32)
-        set_target_properties(${LIBNAME}_static PROPERTIES OUTPUT_NAME ${LIBNAME})
-    endif (NOT WIN32)
-endfunction(add_dual_library)
-
-# Link both a static and a dynamic target against some libraries
-function(target_link_dual_libraries LIBNAME)
-    target_link_libraries(${LIBNAME} ${ARGN})
-    target_link_libraries(${LIBNAME}_static ${ARGN})
-endfunction(target_link_dual_libraries)
-
-function(output_directory TGT DIR)
-    SET_TARGET_PROPERTIES(${TGT} PROPERTIES
-        RUNTIME_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-    SET_TARGET_PROPERTIES(${TGT} PROPERTIES
-        ARCHIVE_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-    SET_TARGET_PROPERTIES(${TGT} PROPERTIES
-        LIBRARY_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-endfunction(output_directory TGT DIR)
-
-function(dual_output_directory TGT DIR)
-    output_directory(${TGT} "${DIR}")
-    output_directory(${TGT}_static "${DIR}")
-endfunction(dual_output_directory TGT DIR)
+#
+# Main configuration
+#
 
-# Flatten a list into a string.
-function(FLATTEN_LIST INPUT SEPARATOR OUTPUT)
-  string (REPLACE ";" "${SEPARATOR}" _TMPS "${INPUT}")
-  set (${OUTPUT} "${_TMPS}" PARENT_SCOPE)
-endfunction()
+# The caller must specify where the generated headers have been placed.
+if(NOT GENERATED_JAVAH)
+    message(FATAL_ERROR "You must set the CMake variable GENERATED_JAVAH")
+endif()
 
 # Check to see if our compiler and linker support the __thread attribute.
-# On Linux and some other operating systems, this is a more efficient 
-# alternative to POSIX thread local storage. 
-INCLUDE(CheckCSourceCompiles)
-CHECK_C_SOURCE_COMPILES("int main(void) { static __thread int i = 0; return 0; }" HAVE_BETTER_TLS)
+# On Linux and some other operating systems, this is a more efficient
+# alternative to POSIX thread local storage.
+include(CheckCSourceCompiles)
+check_c_source_compiles("int main(void) { static __thread int i = 0; return 0; }" HAVE_BETTER_TLS)
 
 # Check to see if we have Intel SSE intrinsics.
-CHECK_C_SOURCE_COMPILES("#include <emmintrin.h>\nint main(void) { __m128d sum0 = _mm_set_pd(0.0,0.0); return 0; }" HAVE_INTEL_SSE_INTRINSICS)
+check_c_source_compileS("#include <emmintrin.h>\nint main(void) { __m128d sum0 = _mm_set_pd(0.0,0.0); return 0; }" HAVE_INTEL_SSE_INTRINSICS)
 
 # Check if we need to link dl library to get dlopen.
 # dlopen on Linux is in separate library but on FreeBSD its in libc
-INCLUDE(CheckLibraryExists)
-CHECK_LIBRARY_EXISTS(dl dlopen "" NEED_LINK_DL)
+include(CheckLibraryExists)
+check_library_exists(dl dlopen "" NEED_LINK_DL)
 
-find_package(JNI REQUIRED)
-if (NOT GENERATED_JAVAH)
-    # Must identify where the generated headers have been placed
-    MESSAGE(FATAL_ERROR "You must set the CMake variable GENERATED_JAVAH")
-endif (NOT GENERATED_JAVAH)
-
-if (WIN32)
+if(WIN32)
+    # Set the optimizer level.
     set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} /O2")
-
     # Set warning level 4.
     set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} /W4")
-
     # Skip "unreferenced formal parameter".
     set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} /wd4100")
-
     # Skip "conditional expression is constant".
     set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} /wd4127")
-
     # Skip deprecated POSIX function warnings.
     set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_CRT_NONSTDC_NO_DEPRECATE")
-
     # Skip CRT non-secure function warnings.  If we can convert usage of
     # strerror, getenv and ctime to their secure CRT equivalents, then we can
     # re-enable the CRT non-secure function warnings.
     set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_CRT_SECURE_NO_WARNINGS")
-
     # Omit unneeded headers.
     set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -DWIN32_LEAN_AND_MEAN")
-
     set(OS_DIR main/native/libhdfs/os/windows)
     set(OUT_DIR target/bin)
-else (WIN32)
-    set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -g -Wall -O2")
-    set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_REENTRANT -D_GNU_SOURCE")
-    set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_LARGEFILE_SOURCE -D_FILE_OFFSET_BITS=64")
+else()
     set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fvisibility=hidden")
     set(OS_DIR main/native/libhdfs/os/posix)
-    set(OS_LINK_LIBRARIES pthread)
     set(OUT_DIR target/usr/local/lib)
-endif (WIN32)
+endif()
+
+# Configure JNI.
+include(HadoopJNI)
 
 add_definitions(-DLIBHDFS_DLL_EXPORT)
 
@@ -132,9 +85,9 @@ include_directories(
 )
 
 set(_FUSE_DFS_VERSION 0.1.0)
-CONFIGURE_FILE(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h)
+configure_file(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h)
 
-add_dual_library(hdfs
+hadoop_add_dual_library(hdfs
     main/native/libhdfs/exception.c
     main/native/libhdfs/jni_helper.c
     main/native/libhdfs/hdfs.c
@@ -142,17 +95,17 @@ add_dual_library(hdfs
     ${OS_DIR}/mutexes.c
     ${OS_DIR}/thread_local_storage.c
 )
-if (NEED_LINK_DL)
+if(NEED_LINK_DL)
    set(LIB_DL dl)
-endif(NEED_LINK_DL)
+endif()
 
-target_link_dual_libraries(hdfs
+hadoop_target_link_dual_libraries(hdfs
     ${JAVA_JVM_LIBRARY}
     ${LIB_DL}
     ${OS_LINK_LIBRARIES}
 )
 
-dual_output_directory(hdfs ${OUT_DIR})
+hadoop_dual_output_directory(hdfs ${OUT_DIR})
 set(LIBHDFS_VERSION "0.0.0")
 set_target_properties(hdfs PROPERTIES
     SOVERSION ${LIBHDFS_VERSION})
@@ -232,25 +185,21 @@ target_link_libraries(test_htable
 
 # Skip vecsum on Windows.  This could be made to work in the future by
 # introducing an abstraction layer over the sys/mman.h functions.
-if (NOT WIN32)
-    add_executable(test_libhdfs_vecsum
-        main/native/libhdfs/test/vecsum.c
-    )
-    if (${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
+if(NOT WIN32)
+    add_executable(test_libhdfs_vecsum main/native/libhdfs/test/vecsum.c)
+    if(CMAKE_SYSTEM_NAME MATCHES "Darwin")
         target_link_libraries(test_libhdfs_vecsum
             hdfs
-            pthread
-        )
-    else (${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
+            pthread)
+    else()
         target_link_libraries(test_libhdfs_vecsum
             hdfs
             pthread
-            rt
-        )
-    endif (${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
-endif(NOT WIN32)
+            rt)
+    endif()
+endif()
 
-IF(REQUIRE_LIBWEBHDFS)
+if(REQUIRE_LIBWEBHDFS)
     add_subdirectory(contrib/libwebhdfs)
-ENDIF(REQUIRE_LIBWEBHDFS)
+endif()
 add_subdirectory(main/native/fuse-dfs)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/29df73dc/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/CMakeLists.txt
index 44f0c42..dc74feb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/CMakeLists.txt
@@ -19,12 +19,12 @@
 find_package(CURL REQUIRED)
 
 set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH}
-"${CMAKE_SOURCE_DIR}/contrib/libwebhdfs/resources/")
+    "${CMAKE_SOURCE_DIR}/contrib/libwebhdfs/resources/")
 
 find_package(Jansson REQUIRED)
 include_directories(${JANSSON_INCLUDE_DIR})
 
-add_dual_library(webhdfs
+hadoop_add_dual_library(webhdfs
     src/hdfs_web.c
     src/hdfs_http_client.c
     src/hdfs_http_query.c
@@ -35,13 +35,13 @@ add_dual_library(webhdfs
     ../../${OS_DIR}/thread_local_storage.c
     ../../main/native/libhdfs/common/htable.c
 )
-target_link_dual_libraries(webhdfs
+hadoop_target_link_dual_libraries(webhdfs
     ${JAVA_JVM_LIBRARY}
     ${CURL_LIBRARY}
     ${JANSSON_LIBRARY}
     pthread
 )
-dual_output_directory(webhdfs target)
+hadoop_dual_output_directory(webhdfs target)
 set(LIBWEBHDFS_VERSION "0.0.0")
 set_target_properties(webhdfs PROPERTIES
     SOVERSION ${LIBWEBHDFS_VERSION})

http://git-wip-us.apache.org/repos/asf/hadoop/blob/29df73dc/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/resources/FindJansson.cmake
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/resources/FindJansson.cmake b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/resources/FindJansson.cmake
index cb8c8db..b8c67ea 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/resources/FindJansson.cmake
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/resources/FindJansson.cmake
@@ -5,9 +5,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
-# 
+#
 # 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
@@ -24,7 +24,8 @@
 #  JANSSON_DEFINITIONS - Compiler switches required for using Jansson
 
 find_path(JANSSON_INCLUDE_DIR jansson.h
-          /usr/incluce
+          /usr/include
+          /usr/include/jansson
           /usr/local/include )
 
 find_library(JANSSON_LIBRARY NAMES jansson
@@ -39,4 +40,4 @@ include(FindPackageHandleStandardArgs)
 find_package_handle_standard_args(Jansson  DEFAULT_MSG
                                   JANSSON_LIBRARY JANSSON_INCLUDE_DIR)
 
-mark_as_advanced(JANSSON_INCLUDE_DIR JANSSON_LIBRARY )
\ No newline at end of file
+mark_as_advanced(JANSSON_INCLUDE_DIR JANSSON_LIBRARY )

http://git-wip-us.apache.org/repos/asf/hadoop/blob/29df73dc/hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/CMakeLists.txt
index 8a8ea5b..7a5e5a1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/CMakeLists.txt
@@ -18,57 +18,63 @@
 
 set(CMAKE_SKIP_RPATH TRUE)
 
+# Flatten a list into a string.
+function(flatten_list INPUT SEPARATOR OUTPUT)
+  string (REPLACE ";" "${SEPARATOR}" _TMPS "${INPUT}")
+  set (${OUTPUT} "${_TMPS}" PARENT_SCOPE)
+endfunction()
+
 # Find Linux FUSE
-IF (${CMAKE_SYSTEM_NAME} MATCHES "Linux")
+if(${CMAKE_SYSTEM_NAME} MATCHES "Linux")
     find_package(PkgConfig REQUIRED)
     pkg_check_modules(FUSE fuse)
-    IF(FUSE_FOUND)
-        FLATTEN_LIST("${FUSE_CFLAGS}" " " FUSE_CFLAGS)
-        FLATTEN_LIST("${FUSE_LDFLAGS}" " " FUSE_LDFLAGS)
+    if(FUSE_FOUND)
+        flatten_list("${FUSE_CFLAGS}" " " FUSE_CFLAGS)
+        flatten_list("${FUSE_LDFLAGS}" " " FUSE_LDFLAGS)
         set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${FUSE_CFLAGS}")
         set(CMAKE_LD_FLAGS "${CMAKE_LD_FLAGS} ${FUSE_LDFLAGS}")
-        MESSAGE(STATUS "Building Linux FUSE client.")
+        message(STATUS "Building Linux FUSE client.")
         include_directories(${FUSE_INCLUDE_DIRS})
-    ELSE(FUSE_FOUND)
-        MESSAGE(STATUS "Failed to find Linux FUSE libraries or include files.  Will not build FUSE client.")
-    ENDIF(FUSE_FOUND)
-ELSE (${CMAKE_SYSTEM_NAME} MATCHES "Linux")
-    MESSAGE(STATUS "Non-Linux system detected.  Will not build FUSE client.")
-ENDIF (${CMAKE_SYSTEM_NAME} MATCHES "Linux")
+    else()
+        message(STATUS "Failed to find Linux FUSE libraries or include files.  Will not build FUSE client.")
+    endif(FUSE_FOUND)
+else()
+    message(STATUS "Non-Linux system detected.  Will not build FUSE client.")
+endif()
 
-IF(FUSE_FOUND)
+if(FUSE_FOUND)
     add_library(posix_util
         ../util/posix_util.c
     )
 
     add_executable(fuse_dfs
         fuse_dfs.c
-        fuse_options.c 
-        fuse_connect.c 
-        fuse_impls_access.c 
-        fuse_impls_chmod.c  
-        fuse_impls_chown.c  
-        fuse_impls_create.c  
-        fuse_impls_flush.c 
-        fuse_impls_getattr.c  
-        fuse_impls_mkdir.c  
-        fuse_impls_mknod.c  
-        fuse_impls_open.c 
-        fuse_impls_read.c 
-        fuse_impls_readdir.c 
-        fuse_impls_release.c 
-        fuse_impls_rename.c 
-        fuse_impls_rmdir.c 
-        fuse_impls_statfs.c 
-        fuse_impls_symlink.c 
-        fuse_impls_truncate.c 
-        fuse_impls_unlink.c 
-        fuse_impls_utimens.c  
+        fuse_options.c
+        fuse_connect.c
+        fuse_impls_access.c
+        fuse_impls_chmod.c
+        fuse_impls_chown.c
+        fuse_impls_create.c
+        fuse_impls_flush.c
+        fuse_impls_getattr.c
+        fuse_impls_mkdir.c
+        fuse_impls_mknod.c
+        fuse_impls_open.c
+        fuse_impls_read.c
+        fuse_impls_readdir.c
+        fuse_impls_release.c
+        fuse_impls_rename.c
+        fuse_impls_rmdir.c
+        fuse_impls_statfs.c
+        fuse_impls_symlink.c
+        fuse_impls_truncate.c
+        fuse_impls_unlink.c
+        fuse_impls_utimens.c
         fuse_impls_write.c
-        fuse_init.c 
-        fuse_stat_struct.c 
-        fuse_trash.c 
-        fuse_users.c 
+        fuse_init.c
+        fuse_stat_struct.c
+        fuse_trash.c
+        fuse_users.c
     )
     target_link_libraries(fuse_dfs
         ${FUSE_LIBRARIES}
@@ -88,8 +94,9 @@ IF(FUSE_FOUND)
         posix_util
         pthread
     )
-ELSE(FUSE_FOUND)
-    IF(REQUIRE_FUSE)
-        MESSAGE(FATAL_ERROR "Required component fuse_dfs could not be built.")
-    ENDIF(REQUIRE_FUSE)
-ENDIF(FUSE_FOUND)
+    set(REQUIRE_FUSE ${REQUIRE_FUSE}) # Prevent unused variable warning
+else()
+    if(REQUIRE_FUSE)
+        message(FATAL_ERROR "Required component fuse_dfs could not be built.")
+    endif()
+endif()


[16/50] hadoop git commit: Adding release 2.7.2 to CHANGES.txt.

Posted by vv...@apache.org.
Adding release 2.7.2 to CHANGES.txt.


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

Branch: refs/heads/YARN-2139
Commit: aad6a7d5dba5858d6e9845f18c4baec16c91911d
Parents: c6793dd
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Sun Jun 28 16:29:12 2015 -0700
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Sun Jun 28 16:29:12 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt | 12 ++++++++++++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     | 12 ++++++++++++
 hadoop-mapreduce-project/CHANGES.txt            | 12 ++++++++++++
 hadoop-yarn-project/CHANGES.txt                 | 12 ++++++++++++
 4 files changed, 48 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aad6a7d5/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 132e4c0..219ef25 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -884,6 +884,18 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12076. Incomplete Cache Mechanism in CredentialProvider API.
     (Larry McCay via cnauroth)
 
+Release 2.7.2 - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aad6a7d5/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 94477fe..b89e10c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -968,6 +968,18 @@ Release 2.8.0 - UNRELEASED
     HDFS-8546. Prune cached replicas from DatanodeDescriptor state on replica
     invalidation. (wang)
 
+Release 2.7.2 - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aad6a7d5/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 4df3b44..785fce8 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -512,6 +512,18 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6413. TestLocalJobSubmission is failing with unknown host
     (zhihai xu via jlowe)
 
+Release 2.7.2 - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aad6a7d5/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index cb6deb1..714bfdc 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -565,6 +565,18 @@ Release 2.8.0 - UNRELEASED
     YARN-3859. LeafQueue doesn't print user properly for application add.
     (Varun Saxena via devaraj)
 
+Release 2.7.2 - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[07/50] hadoop git commit: HDFS-8651. Make hadoop-hdfs-project Native code -Wall-clean (Alan Burlison via Colin P. McCabe)

Posted by vv...@apache.org.
HDFS-8651. Make hadoop-hdfs-project Native code -Wall-clean (Alan Burlison via Colin P. McCabe)


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

Branch: refs/heads/YARN-2139
Commit: 1b764a01fd8010cf9660eb378977a1b2b81f330a
Parents: 40b2569
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Fri Jun 26 10:21:40 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Fri Jun 26 10:22:40 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                      | 3 +++
 .../hadoop-hdfs/src/main/native/fuse-dfs/fuse_impls_open.c       | 4 ----
 2 files changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b764a01/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 7b97f41..27e2e89 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -676,6 +676,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8546. Use try with resources in DataStorage and Storage. (wang)
 
+    HDFS-8651. Make hadoop-hdfs-project Native code -Wall-clean (Alan Burlison
+    via Colin P. McCabe)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b764a01/hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/fuse_impls_open.c
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/fuse_impls_open.c b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/fuse_impls_open.c
index 9ca2650..ca670ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/fuse_impls_open.c
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/fuse_impls_open.c
@@ -24,9 +24,6 @@
 #include <stdio.h>
 #include <stdlib.h>
 
-static int get_hdfs_open_flags_from_info(hdfsFS fs, const char *path,
-                  int flags, int *outflags, const hdfsFileInfo *info);
-
 /**
  * Given a set of FUSE flags, determine the libhdfs flags we need.
  *
@@ -47,7 +44,6 @@ static int get_hdfs_open_flags_from_info(hdfsFS fs, const char *path,
  */
 static int64_t get_hdfs_open_flags(hdfsFS fs, const char *path, int flags)
 {
-  int hasContent;
   int64_t ret;
   hdfsFileInfo *info;
 


[03/50] hadoop git commit: HDFS-8665. Fix replication check in DFSTestUtils#waitForReplication.

Posted by vv...@apache.org.
HDFS-8665. Fix replication check in DFSTestUtils#waitForReplication.


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

Branch: refs/heads/YARN-2139
Commit: ff0e5e572f5dcf7b49381cbe901360f6e171d423
Parents: aa5b15b
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Jun 25 17:29:24 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Jun 25 17:29:24 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                 | 2 ++
 .../src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java   | 9 ++++++++-
 2 files changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff0e5e57/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 01bb92d..dcedb9f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -672,6 +672,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-8640. Make reserved RBW space visible through JMX. (kanaka kumar
     avvaru via Arpit Agarwal)
 
+    HDFS-8665. Fix replication check in DFSTestUtils#waitForReplication. (wang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff0e5e57/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 96fb669..89ee674 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -535,7 +535,14 @@ public class DFSTestUtil {
       public Boolean get() {
         try {
           FileStatus stat = dfs.getFileStatus(file);
-          return replication == stat.getReplication();
+          BlockLocation[] locs = dfs.getFileBlockLocations(stat, 0, stat
+              .getLen());
+          for (BlockLocation loc : locs) {
+            if (replication != loc.getHosts().length) {
+              return false;
+            }
+          }
+          return true;
         } catch (IOException e) {
           LOG.info("getFileStatus on path " + file + " failed!", e);
           return false;


[35/50] hadoop git commit: HDFS-8627. NPE thrown if unable to fetch token from Namenode (Contributed by J.Andreina)

Posted by vv...@apache.org.
HDFS-8627. NPE thrown if unable to fetch token from Namenode (Contributed by J.Andreina)


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

Branch: refs/heads/YARN-2139
Commit: 7ba5bbac02b688f68a8d23671a1e869234b4cebe
Parents: 6d99017
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Jun 30 15:42:59 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Tue Jun 30 15:42:59 2015 +0530

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt      |  3 +++
 .../hdfs/tools/DelegationTokenFetcher.java       | 19 +++++++++++--------
 .../hdfs/tools/TestDelegationTokenFetcher.java   | 15 +++++++++++++++
 3 files changed, 29 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ba5bbac/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index ea67ea1..b065f98 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -338,6 +338,9 @@ Trunk (Unreleased)
 
     HDFS-8412. Fix the test failures in HTTPFS. (umamahesh)
 
+    HDFS-8627. NPE thrown if unable to fetch token from Namenode
+    (J.Andreina via vinayakumarb)
+
 Release 2.8.0 - UNRELEASED
 
   NEW FEATURES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ba5bbac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
index 6376243..803402d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
@@ -176,14 +176,17 @@ public class DelegationTokenFetcher {
                                   final String renewer, final Path tokenFile)
           throws IOException {
     Token<?> token = fs.getDelegationToken(renewer);
-
-    Credentials cred = new Credentials();
-    cred.addToken(token.getKind(), token);
-    cred.writeTokenStorageFile(tokenFile, conf);
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Fetched token " + fs.getUri() + " for " + token.getService()
-              + " into " + tokenFile);
+    if (null != token) {
+      Credentials cred = new Credentials();
+      cred.addToken(token.getKind(), token);
+      cred.writeTokenStorageFile(tokenFile, conf);
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Fetched token " + fs.getUri() + " for " +
+            token.getService() + " into " + tokenFile);
+      }
+    } else {
+      System.err.println("ERROR: Failed to fetch token from " + fs.getUri());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ba5bbac/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDelegationTokenFetcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDelegationTokenFetcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDelegationTokenFetcher.java
index b4cf1c0..ab3933b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDelegationTokenFetcher.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDelegationTokenFetcher.java
@@ -90,4 +90,19 @@ public class TestDelegationTokenFetcher {
     DelegationTokenFetcher.cancelTokens(conf, p);
     Assert.assertEquals(testToken, FakeRenewer.getLastCanceled());
   }
+
+  /**
+   * If token returned is null, saveDelegationToken should not
+   * throw nullPointerException
+   */
+  @Test
+  public void testReturnedTokenIsNull() throws Exception {
+    WebHdfsFileSystem fs = mock(WebHdfsFileSystem.class);
+    doReturn(null).when(fs).getDelegationToken(anyString());
+    Path p = new Path(f.getRoot().getAbsolutePath(), tokenFile);
+    DelegationTokenFetcher.saveDelegationToken(conf, fs, null, p);
+    // When Token returned is null, TokenFile should not exist
+    Assert.assertFalse(p.getFileSystem(conf).exists(p));
+
+  }
 }


[25/50] hadoop git commit: YARN-3695. ServerProxy (NMProxy, etc.) shouldn't retry forever for non network exception. Contributed by Raju Bairishetti

Posted by vv...@apache.org.
YARN-3695. ServerProxy (NMProxy, etc.) shouldn't retry forever for non network exception. Contributed by Raju Bairishetti


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

Branch: refs/heads/YARN-2139
Commit: 62e583c7dcbb30d95d8b32a4978fbdb3b98d67cc
Parents: fad291e
Author: Jian He <ji...@apache.org>
Authored: Mon Jun 29 13:37:32 2015 -0700
Committer: Jian He <ji...@apache.org>
Committed: Mon Jun 29 13:37:32 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../apache/hadoop/yarn/client/ServerProxy.java  | 21 ++---
 .../containermanager/TestNMProxy.java           | 81 +++++++++++++-------
 3 files changed, 70 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/62e583c7/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 3ebf799..2fdcc9d 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -568,6 +568,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3860. rmadmin -transitionToActive should check the state of non-target node.
     (Masatake Iwasaki via junping_du)
 
+    YARN-3695. ServerProxy (NMProxy, etc.) shouldn't retry forever for non
+    network exception. (Raju Bairishetti via jianhe)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62e583c7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ServerProxy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ServerProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ServerProxy.java
index e9bcf8d..de7fc7d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ServerProxy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ServerProxy.java
@@ -53,19 +53,22 @@ public class ServerProxy {
     long maxWaitTime = conf.getLong(maxWaitTimeStr, defMaxWaitTime);
     long retryIntervalMS =
         conf.getLong(connectRetryIntervalStr, defRetryInterval);
-    if (maxWaitTime == -1) {
-      // wait forever.
-      return RetryPolicies.RETRY_FOREVER;
-    }
 
-    Preconditions.checkArgument(maxWaitTime > 0, "Invalid Configuration. "
-        + maxWaitTimeStr + " should be a positive value.");
+    Preconditions.checkArgument((maxWaitTime == -1 || maxWaitTime > 0),
+        "Invalid Configuration. " + maxWaitTimeStr + " should be either"
+            + " positive value or -1.");
     Preconditions.checkArgument(retryIntervalMS > 0, "Invalid Configuration. "
         + connectRetryIntervalStr + "should be a positive value.");
 
-    RetryPolicy retryPolicy =
-        RetryPolicies.retryUpToMaximumTimeWithFixedSleep(maxWaitTime,
-          retryIntervalMS, TimeUnit.MILLISECONDS);
+    RetryPolicy retryPolicy = null;
+    if (maxWaitTime == -1) {
+      // wait forever.
+      retryPolicy = RetryPolicies.RETRY_FOREVER;
+    } else {
+      retryPolicy =
+          RetryPolicies.retryUpToMaximumTimeWithFixedSleep(maxWaitTime,
+              retryIntervalMS, TimeUnit.MILLISECONDS);
+    }
 
     Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap =
         new HashMap<Class<? extends Exception>, RetryPolicy>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62e583c7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestNMProxy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestNMProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestNMProxy.java
index 0b372be..102c9c6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestNMProxy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestNMProxy.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.io.retry.UnreliableInterface;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
@@ -58,8 +59,8 @@ public class TestNMProxy extends BaseContainerManagerTest {
 
   @Before
   public void setUp() throws Exception {
-    conf.setLong(YarnConfiguration.CLIENT_NM_CONNECT_MAX_WAIT_MS, 10000);
-    conf.setLong(YarnConfiguration.CLIENT_NM_CONNECT_RETRY_INTERVAL_MS, 100);
+    containerManager.start();
+    containerManager.setBlockNewContainerRequests(false);
   }
 
   @Override
@@ -77,7 +78,13 @@ public class TestNMProxy extends BaseContainerManagerTest {
             // This causes super to throw an NMNotYetReadyException
             containerManager.setBlockNewContainerRequests(true);
           } else {
-            throw new java.net.ConnectException("start container exception");
+            if (isRetryPolicyRetryForEver()) {
+              // Throw non network exception
+              throw new IOException(
+                  new UnreliableInterface.UnreliableException());
+            } else {
+              throw new java.net.ConnectException("start container exception");
+            }
           }
         } else {
           // This stops super from throwing an NMNotYetReadyException
@@ -86,6 +93,11 @@ public class TestNMProxy extends BaseContainerManagerTest {
         return super.startContainers(requests);
       }
 
+      private boolean isRetryPolicyRetryForEver() {
+        return conf.getLong(
+            YarnConfiguration.CLIENT_NM_CONNECT_MAX_WAIT_MS, 1000) == -1;
+      }
+
       @Override
       public StopContainersResponse stopContainers(
           StopContainersRequest requests) throws YarnException, IOException {
@@ -110,30 +122,13 @@ public class TestNMProxy extends BaseContainerManagerTest {
   }
 
   @Test(timeout = 20000)
-  public void testNMProxyRetry() throws Exception {
-    containerManager.start();
-    containerManager.setBlockNewContainerRequests(false);
-    StartContainersRequest allRequests =
-        Records.newRecord(StartContainersRequest.class);
-    ApplicationId appId = ApplicationId.newInstance(1, 1);
-    ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1);
+   public void testNMProxyRetry() throws Exception {
+     conf.setLong(YarnConfiguration.CLIENT_NM_CONNECT_MAX_WAIT_MS, 10000);
+     conf.setLong(YarnConfiguration.CLIENT_NM_CONNECT_RETRY_INTERVAL_MS, 100);
+     StartContainersRequest allRequests =
+         Records.newRecord(StartContainersRequest.class);
 
-    org.apache.hadoop.yarn.api.records.Token nmToken =
-        context.getNMTokenSecretManager().createNMToken(attemptId,
-          context.getNodeId(), user);
-    final InetSocketAddress address =
-        conf.getSocketAddr(YarnConfiguration.NM_BIND_HOST,
-          YarnConfiguration.NM_ADDRESS, YarnConfiguration.DEFAULT_NM_ADDRESS,
-          YarnConfiguration.DEFAULT_NM_PORT);
-    Token<NMTokenIdentifier> token =
-        ConverterUtils.convertFromYarn(nmToken,
-          SecurityUtil.buildTokenService(address));
-    UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user);
-    ugi.addToken(token);
-
-    ContainerManagementProtocol proxy =
-        NMProxy.createNMProxy(conf, ContainerManagementProtocol.class, ugi,
-          YarnRPC.create(conf), address);
+    ContainerManagementProtocol proxy = getNMProxy();
 
     retryCount = 0;
     shouldThrowNMNotYetReadyException = false;
@@ -156,4 +151,38 @@ public class TestNMProxy extends BaseContainerManagerTest {
     proxy.startContainers(allRequests);
     Assert.assertEquals(5, retryCount);
   }
+
+  @Test(timeout = 20000, expected = IOException.class)
+  public void testShouldNotRetryForeverForNonNetworkExceptionsOnNMConnections()
+      throws Exception {
+    conf.setLong(YarnConfiguration.CLIENT_NM_CONNECT_MAX_WAIT_MS, -1);
+    StartContainersRequest allRequests =
+        Records.newRecord(StartContainersRequest.class);
+
+    ContainerManagementProtocol proxy = getNMProxy();
+
+    shouldThrowNMNotYetReadyException = false;
+    retryCount = 0;
+    proxy.startContainers(allRequests);
+  }
+
+  private ContainerManagementProtocol getNMProxy() {
+    ApplicationId appId = ApplicationId.newInstance(1, 1);
+    ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1);
+
+    org.apache.hadoop.yarn.api.records.Token nmToken =
+        context.getNMTokenSecretManager().createNMToken(attemptId,
+            context.getNodeId(), user);
+    final InetSocketAddress address =
+        conf.getSocketAddr(YarnConfiguration.NM_BIND_HOST,
+            YarnConfiguration.NM_ADDRESS, YarnConfiguration.DEFAULT_NM_ADDRESS,
+            YarnConfiguration.DEFAULT_NM_PORT);
+    Token<NMTokenIdentifier> token =
+        ConverterUtils.convertFromYarn(nmToken,
+            SecurityUtil.buildTokenService(address));
+    UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user);
+    ugi.addToken(token);
+    return NMProxy.createNMProxy(conf, ContainerManagementProtocol.class, ugi,
+        YarnRPC.create(conf), address);
+  }
 }


[29/50] hadoop git commit: Merge branch 'trunk' of https://git-wip-us.apache.org/repos/asf/hadoop into trunk

Posted by vv...@apache.org.
Merge branch 'trunk' of https://git-wip-us.apache.org/repos/asf/hadoop into trunk


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

Branch: refs/heads/YARN-2139
Commit: 34ee0b9b4797093f5aeb0d55f32cf1b74b02e1c2
Parents: 8e1bdc1 4672315
Author: Ming Ma <mi...@apache.org>
Authored: Mon Jun 29 14:37:44 2015 -0700
Committer: Ming Ma <mi...@apache.org>
Committed: Mon Jun 29 14:37:44 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                         |  3 +++
 .../api/records/impl/pb/SerializedExceptionPBImpl.java  |  2 +-
 .../records/impl/pb/TestSerializedExceptionPBImpl.java  | 12 ++++++++++--
 3 files changed, 14 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[06/50] hadoop git commit: YARN-3850. NM fails to read files from full disks which can lead to container logs being lost and other issues. Contributed by Varun Saxena

Posted by vv...@apache.org.
YARN-3850. NM fails to read files from full disks which can lead to container logs being lost and other issues. Contributed by Varun Saxena


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

Branch: refs/heads/YARN-2139
Commit: 40b256949ad6f6e0dbdd248f2d257b05899f4332
Parents: 8ef07f7
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Jun 26 15:47:07 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Jun 26 15:47:07 2015 +0000

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../nodemanager/LocalDirsHandlerService.java    | 24 +++++++++
 .../launcher/RecoveredContainerLaunch.java      |  3 +-
 .../logaggregation/AppLogAggregatorImpl.java    |  4 +-
 .../nodemanager/webapp/ContainerLogsUtils.java  |  2 +-
 .../TestLogAggregationService.java              | 54 +++++++++++++++-----
 .../webapp/TestContainerLogsPage.java           | 22 +++++++-
 7 files changed, 93 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b25694/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 7dc02cd..d74e8a9 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -716,6 +716,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3832. Resource Localization fails on a cluster due to existing cache
     directories (Brahma Reddy Battula via jlowe)
 
+    YARN-3850. NM fails to read files from full disks which can lead to
+    container logs being lost and other issues (Varun Saxena via jlowe)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b25694/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.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/LocalDirsHandlerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java
index 57d4395..0a61035 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java
@@ -238,6 +238,18 @@ public class LocalDirsHandlerService extends AbstractService {
   }
 
   /**
+   * Function to get the local dirs which should be considered for reading
+   * existing files on disk. Contains the good local dirs and the local dirs
+   * that have reached the disk space limit
+   *
+   * @return the local dirs which should be considered for reading
+   */
+  public List<String> getLocalDirsForRead() {
+    return DirectoryCollection.concat(localDirs.getGoodDirs(),
+        localDirs.getFullDirs());
+  }
+
+  /**
    * Function to get the local dirs which should be considered when cleaning up
    * resources. Contains the good local dirs and the local dirs that have reached
    * the disk space limit
@@ -250,6 +262,18 @@ public class LocalDirsHandlerService extends AbstractService {
   }
 
   /**
+   * Function to get the log dirs which should be considered for reading
+   * existing files on disk. Contains the good log dirs and the log dirs that
+   * have reached the disk space limit
+   *
+   * @return the log dirs which should be considered for reading
+   */
+  public List<String> getLogDirsForRead() {
+    return DirectoryCollection.concat(logDirs.getGoodDirs(),
+        logDirs.getFullDirs());
+  }
+
+  /**
    * Function to get the log dirs which should be considered when cleaning up
    * resources. Contains the good log dirs and the log dirs that have reached
    * the disk space limit

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b25694/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/RecoveredContainerLaunch.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/RecoveredContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/RecoveredContainerLaunch.java
index fb10f22..d7b9ae2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/RecoveredContainerLaunch.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/RecoveredContainerLaunch.java
@@ -126,7 +126,8 @@ public class RecoveredContainerLaunch extends ContainerLaunch {
 
   private File locatePidFile(String appIdStr, String containerIdStr) {
     String pidSubpath= getPidFileSubpath(appIdStr, containerIdStr);
-    for (String dir : getContext().getLocalDirsHandler().getLocalDirs()) {
+    for (String dir : getContext().getLocalDirsHandler().
+        getLocalDirsForRead()) {
       File pidFile = new File(dir, pidSubpath);
       if (pidFile.exists()) {
         return pidFile;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b25694/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
index 81be813..4b95a03 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
@@ -595,10 +595,10 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
         boolean appFinished) {
       LOG.info("Uploading logs for container " + containerId
           + ". Current good log dirs are "
-          + StringUtils.join(",", dirsHandler.getLogDirs()));
+          + StringUtils.join(",", dirsHandler.getLogDirsForRead()));
       final LogKey logKey = new LogKey(containerId);
       final LogValue logValue =
-          new LogValue(dirsHandler.getLogDirs(), containerId,
+          new LogValue(dirsHandler.getLogDirsForRead(), containerId,
             userUgi.getShortUserName(), logAggregationContext,
             this.uploadedFileMeta, appFinished);
       try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b25694/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsUtils.java
index c588a89..319f49b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsUtils.java
@@ -74,7 +74,7 @@ public class ContainerLogsUtils {
   
   static List<File> getContainerLogDirs(ContainerId containerId,
       LocalDirsHandlerService dirsHandler) throws YarnException {
-    List<String> logDirs = dirsHandler.getLogDirs();
+    List<String> logDirs = dirsHandler.getLogDirsForRead();
     List<File> containerLogDirs = new ArrayList<File>(logDirs.size());
     for (String logDir : logDirs) {
       logDir = new File(logDir).toURI().getPath();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b25694/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
index eb0d055..fd97cef 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
@@ -177,22 +177,11 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
     dispatcher.close();
   }
 
-  @Test
-  public void testLocalFileDeletionAfterUpload() throws Exception {
-    this.delSrvc = new DeletionService(createContainerExecutor());
-    delSrvc = spy(delSrvc);
-    this.delSrvc.init(conf);
-    this.conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
-    this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
-        this.remoteRootLogDir.getAbsolutePath());
-    
-    LogAggregationService logAggregationService = spy(
-        new LogAggregationService(dispatcher, this.context, this.delSrvc,
-                                  super.dirsHandler));
+  private void verifyLocalFileDeletion(
+      LogAggregationService logAggregationService) throws Exception {
     logAggregationService.init(this.conf);
     logAggregationService.start();
 
-    
     ApplicationId application1 = BuilderUtils.newApplicationId(1234, 1);
 
     // AppLogDir should be created
@@ -252,10 +241,47 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
             ApplicationEventType.APPLICATION_LOG_HANDLING_FINISHED)
     };
 
-    checkEvents(appEventHandler, expectedEvents, true, "getType", "getApplicationID");
+    checkEvents(appEventHandler, expectedEvents, true, "getType",
+        "getApplicationID");
   }
 
   @Test
+  public void testLocalFileDeletionAfterUpload() throws Exception {
+    this.delSrvc = new DeletionService(createContainerExecutor());
+    delSrvc = spy(delSrvc);
+    this.delSrvc.init(conf);
+    this.conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
+    this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
+        this.remoteRootLogDir.getAbsolutePath());
+
+    LogAggregationService logAggregationService = spy(
+        new LogAggregationService(dispatcher, this.context, this.delSrvc,
+                                  super.dirsHandler));
+    verifyLocalFileDeletion(logAggregationService);
+  }
+
+  @Test
+  public void testLocalFileDeletionOnDiskFull() throws Exception {
+    this.delSrvc = new DeletionService(createContainerExecutor());
+    delSrvc = spy(delSrvc);
+    this.delSrvc.init(conf);
+    this.conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
+    this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
+        this.remoteRootLogDir.getAbsolutePath());
+    List<String> logDirs = super.dirsHandler.getLogDirs();
+    LocalDirsHandlerService dirsHandler = spy(super.dirsHandler);
+    // Simulate disk being full by returning no good log dirs but having a
+    // directory in full log dirs.
+    when(dirsHandler.getLogDirs()).thenReturn(new ArrayList<String>());
+    when(dirsHandler.getLogDirsForRead()).thenReturn(logDirs);
+    LogAggregationService logAggregationService = spy(
+        new LogAggregationService(dispatcher, this.context, this.delSrvc,
+            dirsHandler));
+    verifyLocalFileDeletion(logAggregationService);
+  }
+
+
+  @Test
   public void testNoContainerOnNode() throws Exception {
     this.conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
     this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b25694/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java
index 43100b3..e63f681 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.nodemanager.webapp;
 
 import static org.junit.Assume.assumeTrue;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.verify;
 
@@ -29,6 +30,7 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -122,6 +124,24 @@ public class TestContainerLogsPage {
     Assert.assertNull(nmContext.getContainers().get(container1));
     files = ContainerLogsUtils.getContainerLogDirs(container1, user, nmContext);
     Assert.assertTrue(!(files.get(0).toString().contains("file:")));
+
+    // Create a new context to check if correct container log dirs are fetched
+    // on full disk.
+    LocalDirsHandlerService dirsHandlerForFullDisk = spy(dirsHandler);
+    // good log dirs are empty and nm log dir is in the full log dir list.
+    when(dirsHandlerForFullDisk.getLogDirs()).
+        thenReturn(new ArrayList<String>());
+    when(dirsHandlerForFullDisk.getLogDirsForRead()).
+        thenReturn(Arrays.asList(new String[] {absLogDir.getAbsolutePath()}));
+    nmContext = new NodeManager.NMContext(null, null, dirsHandlerForFullDisk,
+        new ApplicationACLsManager(conf), new NMNullStateStoreService());
+    nmContext.getApplications().put(appId, app);
+    container.setState(ContainerState.RUNNING);
+    nmContext.getContainers().put(container1, container);
+    List<File> dirs =
+        ContainerLogsUtils.getContainerLogDirs(container1, user, nmContext);
+    File containerLogDir = new File(absLogDir, appId + "/" + container1);
+    Assert.assertTrue(dirs.contains(containerLogDir));
   }
 
   @Test(timeout = 10000)
@@ -231,7 +251,7 @@ public class TestContainerLogsPage {
     LocalDirsHandlerService localDirs = mock(LocalDirsHandlerService.class);
     List<String> logDirs = new ArrayList<String>();
     logDirs.add("F:/nmlogs");
-    when(localDirs.getLogDirs()).thenReturn(logDirs);
+    when(localDirs.getLogDirsForRead()).thenReturn(logDirs);
     
     ApplicationIdPBImpl appId = mock(ApplicationIdPBImpl.class);
     when(appId.toString()).thenReturn("app_id_1");


[47/50] hadoop git commit: HADOOP-12159. Move DistCpUtils#compareFs() to org.apache.hadoop.fs.FileUtil and fix for HA namespaces (rchiang via rkanter)

Posted by vv...@apache.org.
HADOOP-12159. Move DistCpUtils#compareFs() to org.apache.hadoop.fs.FileUtil and fix for HA namespaces (rchiang via rkanter)


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

Branch: refs/heads/YARN-2139
Commit: aaafa0b2ee64f6cfb7fdc717500e1c483b9df8cc
Parents: ad60807
Author: Robert Kanter <rk...@apache.org>
Authored: Tue Jun 30 16:42:59 2015 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Tue Jun 30 16:49:27 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../java/org/apache/hadoop/fs/FileUtil.java     |  42 +++++++
 .../java/org/apache/hadoop/fs/TestFileUtil.java | 112 ++++++++++++++++++-
 .../java/org/apache/hadoop/tools/DistCp.java    |   3 +-
 .../apache/hadoop/tools/util/DistCpUtils.java   |  39 -------
 5 files changed, 155 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aaafa0b2/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 19ba620..4e5f6bb 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -910,6 +910,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-10798. globStatus() should always return a sorted list of files
     (cmccabe)
 
+    HADOOP-12159. Move DistCpUtils#compareFs() to org.apache.hadoop.fs.FileUtil
+    and fix for HA namespaces (rchiang via rkanter)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aaafa0b2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
index 9b9e213..8abb4eb 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
@@ -19,6 +19,9 @@
 package org.apache.hadoop.fs;
 
 import java.io.*;
+import java.net.InetAddress;
+import java.net.URI;
+import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Enumeration;
@@ -1329,4 +1332,43 @@ public class FileUtil {
                         unexpandedWildcardClasspath.toString()};
     return jarCp;
   }
+
+  public static boolean compareFs(FileSystem srcFs, FileSystem destFs) {
+    if (srcFs==null || destFs==null) {
+      return false;
+    }
+    URI srcUri = srcFs.getUri();
+    URI dstUri = destFs.getUri();
+    if (srcUri.getScheme()==null) {
+      return false;
+    }
+    if (!srcUri.getScheme().equals(dstUri.getScheme())) {
+      return false;
+    }
+    String srcHost = srcUri.getHost();
+    String dstHost = dstUri.getHost();
+    if ((srcHost!=null) && (dstHost!=null)) {
+      if (srcHost.equals(dstHost)) {
+        return srcUri.getPort()==dstUri.getPort();
+      }
+      try {
+        srcHost = InetAddress.getByName(srcHost).getCanonicalHostName();
+        dstHost = InetAddress.getByName(dstHost).getCanonicalHostName();
+      } catch (UnknownHostException ue) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Could not compare file-systems. Unknown host: ", ue);
+        }
+        return false;
+      }
+      if (!srcHost.equals(dstHost)) {
+        return false;
+      }
+    } else if (srcHost==null && dstHost!=null) {
+      return false;
+    } else if (srcHost!=null) {
+      return false;
+    }
+    // check for ports
+    return srcUri.getPort()==dstUri.getPort();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aaafa0b2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java
index 41794b8..3418ade 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java
@@ -17,7 +17,8 @@
  */
 package org.apache.hadoop.fs;
 
-import org.junit.Before;
+import org.junit.*;
+
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileInputStream;
@@ -25,8 +26,11 @@ import java.io.FileOutputStream;
 import java.io.FileReader;
 import java.io.IOException;
 import java.io.OutputStream;
+import java.net.InetAddress;
 import java.net.URI;
 import java.io.PrintWriter;
+import java.net.URISyntaxException;
+import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -44,10 +48,12 @@ import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.tools.tar.TarEntry;
 import org.apache.tools.tar.TarOutputStream;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Test;
+
+import javax.print.attribute.URISyntax;
+
 import static org.junit.Assert.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 public class TestFileUtil {
   private static final Log LOG = LogFactory.getLog(TestFileUtil.class);
@@ -64,6 +70,25 @@ public class TestFileUtil {
   private final File dir2 = new File(del, DIR + "2");
   private final File partitioned = new File(TEST_DIR, "partitioned");
 
+  private InetAddress inet1;
+  private InetAddress inet2;
+  private InetAddress inet3;
+  private InetAddress inet4;
+  private InetAddress inet5;
+  private InetAddress inet6;
+  private URI uri1;
+  private URI uri2;
+  private URI uri3;
+  private URI uri4;
+  private URI uri5;
+  private URI uri6;
+  private FileSystem fs1;
+  private FileSystem fs2;
+  private FileSystem fs3;
+  private FileSystem fs4;
+  private FileSystem fs5;
+  private FileSystem fs6;
+
   /**
    * Creates multiple directories for testing.
    * 
@@ -80,6 +105,7 @@ public class TestFileUtil {
    *   file: part-r-00000, contents: "foo"
    *   file: part-r-00001, contents: "bar"
    */
+  @Ignore
   private void setupDirs() throws IOException {
     Assert.assertFalse(del.exists());
     Assert.assertFalse(tmp.exists());
@@ -1096,4 +1122,82 @@ public class TestFileUtil {
       }
     }
   }
+
+  @Ignore
+  public void setupCompareFs() {
+    // Set up Strings
+    String host1 = "1.2.3.4";
+    String host2 = "2.3.4.5";
+    int port1 = 7000;
+    int port2 = 7001;
+    String uris1 = "hdfs://" + host1 + ":" + Integer.toString(port1) + "/tmp/foo";
+    String uris2 = "hdfs://" + host1 + ":" + Integer.toString(port2) + "/tmp/foo";
+    String uris3 = "hdfs://" + host2 + ":" + Integer.toString(port2) + "/tmp/foo";
+    String uris4 = "hdfs://" + host2 + ":" + Integer.toString(port2) + "/tmp/foo";
+    String uris5 = "file:///" + host1 + ":" + Integer.toString(port1) + "/tmp/foo";
+    String uris6 = "hdfs:///" + host1 + "/tmp/foo";
+    // Set up URI objects
+    try {
+      uri1 = new URI(uris1);
+      uri2 = new URI(uris2);
+      uri3 = new URI(uris3);
+      uri4 = new URI(uris4);
+      uri5 = new URI(uris5);
+      uri6 = new URI(uris6);
+    } catch (URISyntaxException use) {
+    }
+    // Set up InetAddress
+    inet1 = mock(InetAddress.class);
+    when(inet1.getCanonicalHostName()).thenReturn(host1);
+    inet2 = mock(InetAddress.class);
+    when(inet2.getCanonicalHostName()).thenReturn(host1);
+    inet3 = mock(InetAddress.class);
+    when(inet3.getCanonicalHostName()).thenReturn(host2);
+    inet4 = mock(InetAddress.class);
+    when(inet4.getCanonicalHostName()).thenReturn(host2);
+    inet5 = mock(InetAddress.class);
+    when(inet5.getCanonicalHostName()).thenReturn(host1);
+    inet6 = mock(InetAddress.class);
+    when(inet6.getCanonicalHostName()).thenReturn(host1);
+
+    // Link of InetAddress to corresponding URI
+    try {
+      when(InetAddress.getByName(uris1)).thenReturn(inet1);
+      when(InetAddress.getByName(uris2)).thenReturn(inet2);
+      when(InetAddress.getByName(uris3)).thenReturn(inet3);
+      when(InetAddress.getByName(uris4)).thenReturn(inet4);
+      when(InetAddress.getByName(uris5)).thenReturn(inet5);
+    } catch (UnknownHostException ue) {
+    }
+
+    fs1 = mock(FileSystem.class);
+    when(fs1.getUri()).thenReturn(uri1);
+    fs2 = mock(FileSystem.class);
+    when(fs2.getUri()).thenReturn(uri2);
+    fs3 = mock(FileSystem.class);
+    when(fs3.getUri()).thenReturn(uri3);
+    fs4 = mock(FileSystem.class);
+    when(fs4.getUri()).thenReturn(uri4);
+    fs5 = mock(FileSystem.class);
+    when(fs5.getUri()).thenReturn(uri5);
+    fs6 = mock(FileSystem.class);
+    when(fs6.getUri()).thenReturn(uri6);
+  }
+
+  @Test
+  public void testCompareFsNull() throws Exception {
+    setupCompareFs();
+    assertEquals(FileUtil.compareFs(null,fs1),false);
+    assertEquals(FileUtil.compareFs(fs1,null),false);
+  }
+
+  @Test
+  public void testCompareFsDirectories() throws Exception {
+    setupCompareFs();
+    assertEquals(FileUtil.compareFs(fs1,fs1),true);
+    assertEquals(FileUtil.compareFs(fs1,fs2),false);
+    assertEquals(FileUtil.compareFs(fs1,fs5),false);
+    assertEquals(FileUtil.compareFs(fs3,fs4),true);
+    assertEquals(FileUtil.compareFs(fs1,fs6),false);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aaafa0b2/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 6921a1e..2385df3 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
@@ -28,6 +28,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Cluster;
@@ -347,7 +348,7 @@ public class DistCp extends Configured implements Tool {
       workDir = new Path(workDir, WIP_PREFIX + targetPath.getName()
                                 + rand.nextInt());
       FileSystem workFS = workDir.getFileSystem(configuration);
-      if (!DistCpUtils.compareFs(targetFS, workFS)) {
+      if (!FileUtil.compareFs(targetFS, workFS)) {
         throw new IllegalArgumentException("Work path " + workDir +
             " and target path " + targetPath + " are in different file system");
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aaafa0b2/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java
index 5ac5bf1..70d947e 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java
@@ -468,43 +468,4 @@ public class DistCpUtils {
     return (sourceChecksum == null || targetChecksum == null ||
             sourceChecksum.equals(targetChecksum));
   }
-
-  /* see if two file systems are the same or not
-   *
-   */
-  public static boolean compareFs(FileSystem srcFs, FileSystem destFs) {
-    URI srcUri = srcFs.getUri();
-    URI dstUri = destFs.getUri();
-    if (srcUri.getScheme() == null) {
-      return false;
-    }
-    if (!srcUri.getScheme().equals(dstUri.getScheme())) {
-      return false;
-    }
-    String srcHost = srcUri.getHost();
-    String dstHost = dstUri.getHost();
-    if ((srcHost != null) && (dstHost != null)) {
-      try {
-        srcHost = InetAddress.getByName(srcHost).getCanonicalHostName();
-        dstHost = InetAddress.getByName(dstHost).getCanonicalHostName();
-      } catch(UnknownHostException ue) {
-        if (LOG.isDebugEnabled())
-          LOG.debug("Could not compare file-systems. Unknown host: ", ue);
-        return false;
-      }
-      if (!srcHost.equals(dstHost)) {
-        return false;
-      }
-    }
-    else if (srcHost == null && dstHost != null) {
-      return false;
-    }
-    else if (srcHost != null) {
-      return false;
-    }
-
-    //check for ports
-
-    return srcUri.getPort() == dstUri.getPort();
-  }
 }


[39/50] hadoop git commit: YARN-3768. ArrayIndexOutOfBoundsException with empty environment variables. (Zhihai Xu via gera)

Posted by vv...@apache.org.
YARN-3768. ArrayIndexOutOfBoundsException with empty environment variables. (Zhihai Xu via gera)


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

Branch: refs/heads/YARN-2139
Commit: 6f2a41e37d0b36cdafcfff75125165f212c612a6
Parents: b6ba564
Author: Gera Shegalov <ge...@apache.org>
Authored: Tue Jun 30 14:44:48 2015 -0700
Committer: Gera Shegalov <ge...@apache.org>
Committed: Tue Jun 30 14:59:33 2015 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/util/Shell.java |  6 +-
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../java/org/apache/hadoop/yarn/util/Apps.java  | 19 ++++--
 .../org/apache/hadoop/yarn/util/TestApps.java   | 61 ++++++++++++++++++++
 4 files changed, 81 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f2a41e3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
index 45c1588..ed83e8d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
@@ -221,10 +221,12 @@ abstract public class Shell {
       new String[] { "kill", "-" + code, isSetsidAvailable ? "-" + pid : pid };
   }
 
+  public static final String ENV_NAME_REGEX = "[A-Za-z_][A-Za-z0-9_]*";
   /** Return a regular expression string that match environment variables */
   public static String getEnvironmentVariableRegex() {
-    return (WINDOWS) ? "%([A-Za-z_][A-Za-z0-9_]*?)%" :
-      "\\$([A-Za-z_][A-Za-z0-9_]*)";
+    return (WINDOWS)
+        ? "%(" + ENV_NAME_REGEX + "?)%"
+        : "\\$(" + ENV_NAME_REGEX + ")";
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f2a41e3/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 8461f69..94bc8fc 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -574,6 +574,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3770. SerializedException should also handle java.lang.Error on
     de-serialization. (Lavkesh Lahngir via jianhe)
 
+    YARN-3768. ArrayIndexOutOfBoundsException with empty environment variables.
+    (Zhihai Xu via gera)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f2a41e3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Apps.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Apps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Apps.java
index cf3940f..9235e7d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Apps.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Apps.java
@@ -44,6 +44,14 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 public class Apps {
   public static final String APP = "application";
   public static final String ID = "ID";
+  private static final Pattern VAR_SUBBER =
+      Pattern.compile(Shell.getEnvironmentVariableRegex());
+  private static final Pattern VARVAL_SPLITTER = Pattern.compile(
+        "(?<=^|,)"                            // preceded by ',' or line begin
+      + '(' + Shell.ENV_NAME_REGEX + ')'      // var group
+      + '='
+      + "([^,]*)"                             // val group
+      );
 
   public static ApplicationId toAppID(String aid) {
     Iterator<String> it = _split(aid).iterator();
@@ -73,11 +81,10 @@ public class Apps {
   public static void setEnvFromInputString(Map<String, String> env,
       String envString,  String classPathSeparator) {
     if (envString != null && envString.length() > 0) {
-      String childEnvs[] = envString.split(",");
-      Pattern p = Pattern.compile(Shell.getEnvironmentVariableRegex());
-      for (String cEnv : childEnvs) {
-        String[] parts = cEnv.split("="); // split on '='
-        Matcher m = p.matcher(parts[1]);
+      Matcher varValMatcher = VARVAL_SPLITTER.matcher(envString);
+      while (varValMatcher.find()) {
+        String envVar = varValMatcher.group(1);
+        Matcher m = VAR_SUBBER.matcher(varValMatcher.group(2));
         StringBuffer sb = new StringBuffer();
         while (m.find()) {
           String var = m.group(1);
@@ -93,7 +100,7 @@ public class Apps {
           m.appendReplacement(sb, Matcher.quoteReplacement(replace));
         }
         m.appendTail(sb);
-        addToEnvironment(env, parts[0], sb.toString(), classPathSeparator);
+        addToEnvironment(env, envVar, sb.toString(), classPathSeparator);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f2a41e3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestApps.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestApps.java
new file mode 100644
index 0000000..66d2d23
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestApps.java
@@ -0,0 +1,61 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.hadoop.yarn.util;
+
+import org.apache.hadoop.util.Shell;
+import org.junit.Test;
+
+import java.io.File;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestApps {
+  @Test
+  public void testSetEnvFromInputString() {
+    Map<String, String> environment = new HashMap<String, String>();
+    environment.put("JAVA_HOME", "/path/jdk");
+    String goodEnv = "a1=1,b_2=2,_c=3,d=4,e=,f_win=%JAVA_HOME%"
+        + ",g_nix=$JAVA_HOME";
+    Apps.setEnvFromInputString(environment, goodEnv, File.pathSeparator);
+    assertEquals("1", environment.get("a1"));
+    assertEquals("2", environment.get("b_2"));
+    assertEquals("3", environment.get("_c"));
+    assertEquals("4", environment.get("d"));
+    assertEquals("", environment.get("e"));
+    if (Shell.WINDOWS) {
+      assertEquals("$JAVA_HOME", environment.get("g_nix"));
+      assertEquals("/path/jdk", environment.get("f_win"));
+    } else {
+      assertEquals("/path/jdk", environment.get("g_nix"));
+      assertEquals("%JAVA_HOME%", environment.get("f_win"));
+    }
+    String badEnv = "1,,2=a=b,3=a=,4==,5==a,==,c-3=3,=";
+    environment.clear();
+    Apps.setEnvFromInputString(environment, badEnv, File.pathSeparator);
+    assertEquals(environment.size(), 0);
+
+    // Test "=" in the value part
+    environment.clear();
+    Apps.setEnvFromInputString(environment, "b1,e1==,e2=a1=a2,b2",
+        File.pathSeparator);
+    assertEquals("=", environment.get("e1"));
+    assertEquals("a1=a2", environment.get("e2"));
+  }
+}


[33/50] hadoop git commit: HDFS-8579. Update HDFS usage with missing options (Contributed by J.Andreina)

Posted by vv...@apache.org.
HDFS-8579. Update HDFS usage with missing options (Contributed by J.Andreina)


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

Branch: refs/heads/YARN-2139
Commit: 77588e1d32cc465e6b0699bb3564a4c7c6df16fa
Parents: d3797f9
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Jun 30 15:00:22 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Tue Jun 30 15:00:22 2015 +0530

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/77588e1d/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 3535f90..ea67ea1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -983,6 +983,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8687. Remove the duplicate usage message from Dfsck.java. (Brahma
     Reddy Battula via Arpit Agarwal)
 
+    HDFS-8579. Update HDFS usage with missing options
+    (J.Andreina via vinayakumarb)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77588e1d/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
index f464261..2e09cef 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
@@ -26,6 +26,7 @@ function hadoop_usage
   echo "                       Hadoop jar and the required libraries"
   echo "  crypto               configure HDFS encryption zones"
   echo "  datanode             run a DFS datanode"
+  echo "  debug                run a Debug Admin to execute HDFS debug commands"
   echo "  dfs                  run a filesystem command on the file system"
   echo "  dfsadmin             run a DFS admin client"
   echo "  fetchdt              fetch a delegation token from the NameNode"


[44/50] hadoop git commit: HADOOP-10798. globStatus() should always return a sorted list of files (cmccabe)

Posted by vv...@apache.org.
HADOOP-10798. globStatus() should always return a sorted list of files (cmccabe)


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

Branch: refs/heads/YARN-2139
Commit: 68e588cbee660d55dba518892d064bee3795a002
Parents: d0cc038
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Tue Jun 30 16:40:21 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Tue Jun 30 16:40:21 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +++
 .../main/java/org/apache/hadoop/fs/Globber.java | 11 ++++++++-
 .../org/apache/hadoop/fs/TestGlobPaths.java     | 25 ++++++++++++++++++++
 3 files changed, 38 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/68e588cb/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 2aab8b4..4d69a229 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -905,6 +905,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12154. FileSystem#getUsed() returns the file length only from root '/'
     (J.Andreina via vinayakumarb)
 
+    HADOOP-10798. globStatus() should always return a sorted list of files
+    (cmccabe)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68e588cb/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java
index 8a8137a..9cb810f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
 import org.apache.commons.logging.LogFactory;
@@ -285,6 +286,14 @@ class Globber {
         (flattenedPatterns.size() <= 1)) {
       return null;
     }
-    return results.toArray(new FileStatus[0]);
+    /*
+     * In general, the results list will already be sorted, since listStatus
+     * returns results in sorted order for many Hadoop filesystems.  However,
+     * not all Hadoop filesystems have this property.  So we sort here in order
+     * to get consistent results.  See HADOOP-10798 for details.
+     */
+    FileStatus ret[] = results.toArray(new FileStatus[0]);
+    Arrays.sort(ret);
+    return ret;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68e588cb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestGlobPaths.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestGlobPaths.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestGlobPaths.java
index 50e2e5b..afd8fb2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestGlobPaths.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestGlobPaths.java
@@ -21,9 +21,11 @@ import static org.junit.Assert.*;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
 import java.util.UUID;
 import java.util.regex.Pattern;
 
+import com.google.common.collect.Ordering;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -1284,4 +1286,27 @@ public class TestGlobPaths {
   public void testNonTerminalGlobsOnFC() throws Exception {
     testOnFileContext(new TestNonTerminalGlobs(true));
   }
+
+  @Test
+  public void testLocalFilesystem() throws Exception {
+    Configuration conf = new Configuration();
+    FileSystem fs = FileSystem.getLocal(conf);
+    String localTmp = System.getProperty("java.io.tmpdir");
+    Path base = new Path(new Path(localTmp), UUID.randomUUID().toString());
+    Assert.assertTrue(fs.mkdirs(base));
+    Assert.assertTrue(fs.mkdirs(new Path(base, "e")));
+    Assert.assertTrue(fs.mkdirs(new Path(base, "c")));
+    Assert.assertTrue(fs.mkdirs(new Path(base, "a")));
+    Assert.assertTrue(fs.mkdirs(new Path(base, "d")));
+    Assert.assertTrue(fs.mkdirs(new Path(base, "b")));
+    fs.deleteOnExit(base);
+    FileStatus[] status = fs.globStatus(new Path(base, "*"));
+    ArrayList list = new ArrayList();
+    for (FileStatus f: status) {
+        list.add(f.getPath().toString());
+    }
+    boolean sorted = Ordering.natural().isOrdered(list);
+    Assert.assertTrue(sorted);
+  }
 }
+


[19/50] hadoop git commit: HADOOP-12119. hadoop fs -expunge does not work for federated namespace (Contributed by J.Andreina)

Posted by vv...@apache.org.
HADOOP-12119. hadoop fs -expunge does not work for federated namespace (Contributed by J.Andreina)


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

Branch: refs/heads/YARN-2139
Commit: c815344e2e68d78f6587b65bc2db25e151aa4364
Parents: 88ceb38
Author: Vinayakumar B <vi...@apache.org>
Authored: Mon Jun 29 15:58:54 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Mon Jun 29 15:58:54 2015 +0530

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt    |  3 +++
 .../java/org/apache/hadoop/fs/shell/Delete.java    | 17 ++++++++++++++---
 .../test/java/org/apache/hadoop/fs/TestTrash.java  | 14 ++++++++++++--
 3 files changed, 29 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c815344e/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 219ef25..0a964a3 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -884,6 +884,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12076. Incomplete Cache Mechanism in CredentialProvider API.
     (Larry McCay via cnauroth)
 
+    HADOOP-12119. hadoop fs -expunge does not work for federated namespace
+    (J.Andreina via vinayakumarb)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c815344e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Delete.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Delete.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Delete.java
index f882817..40d9478 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Delete.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Delete.java
@@ -25,6 +25,7 @@ import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.PathIsDirectoryException;
 import org.apache.hadoop.fs.PathIsNotDirectoryException;
@@ -195,9 +196,19 @@ class Delete {
     @Override
     protected void processArguments(LinkedList<PathData> args)
     throws IOException {
-      Trash trash = new Trash(getConf());
-      trash.expunge();
-      trash.checkpoint();    
+      FileSystem[] childFileSystems =
+          FileSystem.get(getConf()).getChildFileSystems();
+      if (null != childFileSystems) {
+        for (FileSystem fs : childFileSystems) {
+          Trash trash = new Trash(fs, getConf());
+          trash.expunge();
+          trash.checkpoint();
+        }
+      } else {
+        Trash trash = new Trash(getConf());
+        trash.expunge();
+        trash.checkpoint();
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c815344e/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 a675e30..9a91733 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
@@ -594,8 +594,18 @@ public class TestTrash extends TestCase {
     TestLFS() {
       this(new Path(TEST_DIR, "user/test"));
     }
-    TestLFS(Path home) {
-      super();
+    TestLFS(final Path home) {
+      super(new RawLocalFileSystem() {
+        @Override
+        protected Path getInitialWorkingDirectory() {
+          return makeQualified(home);
+        }
+
+        @Override
+        public Path getHomeDirectory() {
+          return makeQualified(home);
+        }
+      });
       this.home = home;
     }
     @Override


[10/50] hadoop git commit: HADOOP-12036. Consolidate all of the cmake extensions in one directory (alanburlison via cmccabe)

Posted by vv...@apache.org.
HADOOP-12036. Consolidate all of the cmake extensions in one directory (alanburlison via cmccabe)


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

Branch: refs/heads/YARN-2139
Commit: aa07dea3577158b92a17651d10da20df73f54561
Parents: 60b858b
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Fri Jun 26 12:32:31 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Fri Jun 26 12:32:31 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../hadoop-common/HadoopCommon.cmake            | 207 +++++++++++
 .../hadoop-common/HadoopJNI.cmake               |  97 +++++
 .../hadoop-common/src/CMakeLists.txt            | 366 ++++++++-----------
 4 files changed, 457 insertions(+), 216 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa07dea3/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 5901794..92e1bfa 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -679,6 +679,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11885. hadoop-dist dist-layout-stitching.sh does not work with dash.
     (wang)
 
+    HADOOP-12036. Consolidate all of the cmake extensions in one directory
+    (alanburlison via cmccabe)
+
   BUG FIXES
 
     HADOOP-11802: DomainSocketWatcher thread terminates sometimes after there

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa07dea3/hadoop-common-project/hadoop-common/HadoopCommon.cmake
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/HadoopCommon.cmake b/hadoop-common-project/hadoop-common/HadoopCommon.cmake
new file mode 100644
index 0000000..5a83f3d
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/HadoopCommon.cmake
@@ -0,0 +1,207 @@
+#
+# 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.
+#
+
+#
+# Common CMake utilities and configuration, shared by all Native components.
+#
+
+#
+# Platform-specific prerequisite checks.
+#
+
+if(CMAKE_SYSTEM_NAME STREQUAL "SunOS")
+    # Only 64-bit Java is supported.
+    if(NOT JVM_ARCH_DATA_MODEL EQUAL 64)
+        message(FATAL_ERROR "Unrecognised JVM_ARCH_DATA_MODEL '${JVM_ARCH_DATA_MODEL}'. "
+          "A 64-bit JVM must be used on Solaris, make sure that one is installed and, "
+          "if necessary, the MAVEN_OPTS environment variable includes '-d64'")
+    endif()
+
+    # Only gcc is suported for now.
+    if(NOT(CMAKE_COMPILER_IS_GNUCC AND CMAKE_COMPILER_IS_GNUCXX))
+        message(FATAL_ERROR "Only gcc is supported on Solaris")
+    endif()
+endif()
+
+#
+# Helper functions and macros.
+#
+
+# Add flags to all the CMake compiler variables
+macro(hadoop_add_compiler_flags FLAGS)
+    set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${FLAGS}")
+    set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${FLAGS}")
+endmacro()
+
+# Add flags to all the CMake linker variables
+macro(hadoop_add_linker_flags FLAGS)
+    set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${FLAGS}")
+    set(CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} ${FLAGS}")
+    set(CMAKE_STATIC_LINKER_FLAGS "${CMAKE_STATIC_LINKER_FLAGS} ${FLAGS}")
+endmacro()
+
+# Compile a library with both shared and static variants.
+function(hadoop_add_dual_library LIBNAME)
+    add_library(${LIBNAME} SHARED ${ARGN})
+    add_library(${LIBNAME}_static STATIC ${ARGN})
+    set_target_properties(${LIBNAME}_static PROPERTIES OUTPUT_NAME ${LIBNAME})
+endfunction()
+
+# Link both a static and a dynamic target against some libraries.
+function(hadoop_target_link_dual_libraries LIBNAME)
+    target_link_libraries(${LIBNAME} ${ARGN})
+    target_link_libraries(${LIBNAME}_static ${ARGN})
+endfunction()
+
+# Set all the output directories to the same place.
+function(hadoop_output_directory TGT DIR)
+    set_target_properties(${TGT} PROPERTIES RUNTIME_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
+    set_target_properties(${TGT} PROPERTIES ARCHIVE_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
+    set_target_properties(${TGT} PROPERTIES LIBRARY_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
+endfunction()
+
+# Set the target directories for dynamic and static builds.
+function(hadoop_dual_output_directory TGT DIR)
+    hadoop_output_directory(${TGT} "${DIR}")
+    hadoop_output_directory(${TGT}_static "${DIR}")
+endfunction()
+
+# Alter the behavior of find_package and find_library so that we find only
+# shared libraries with a given version suffix.  You should save
+# CMAKE_FIND_LIBRARY_SUFFIXES before calling this function and restore it
+# afterwards.  On Windows this function is a no-op.  Windows does not encode
+# version number information information into library path names.
+macro(hadoop_set_find_shared_library_version LVERS)
+    if(${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
+        # Mac OS uses .dylib
+        set(CMAKE_FIND_LIBRARY_SUFFIXES ".${LVERS}.dylib")
+    elseif(${CMAKE_SYSTEM_NAME} MATCHES "FreeBSD")
+        # FreeBSD has always .so installed.
+        set(CMAKE_FIND_LIBRARY_SUFFIXES ".so")
+    elseif(${CMAKE_SYSTEM_NAME} MATCHES "Windows")
+        # Windows doesn't support finding shared libraries by version.
+    else()
+        # Most UNIX variants use .so
+        set(CMAKE_FIND_LIBRARY_SUFFIXES ".so.${LVERS}")
+    endif()
+endmacro()
+
+# Alter the behavior of find_package and find_library so that we find only
+# shared libraries without any version suffix.  You should save
+# CMAKE_FIND_LIBRARY_SUFFIXES before calling this function and restore it
+# afterwards. On Windows this function is a no-op.  Windows does not encode
+# version number information information into library path names.
+macro(hadoop_set_find_shared_library_without_version)
+    if(${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
+        # Mac OS uses .dylib
+        set(CMAKE_FIND_LIBRARY_SUFFIXES ".dylib")
+    elseif(${CMAKE_SYSTEM_NAME} MATCHES "Windows")
+        # No effect
+    else()
+        # Most UNIX variants use .so
+        set(CMAKE_FIND_LIBRARY_SUFFIXES ".so")
+    endif()
+endmacro()
+
+#
+# Configuration.
+#
+
+# Initialise the shared gcc/g++ flags if they aren't already defined.
+if(NOT DEFINED GCC_SHARED_FLAGS)
+    set(GCC_SHARED_FLAGS "-g -O2 -Wall -pthread -D_FILE_OFFSET_BITS=64")
+endif()
+
+# Add in support other compilers here, if necessary,
+# the assumption is that GCC or a GCC-compatible compiler is being used.
+
+# Set the shared GCC-compatible compiler and linker flags.
+hadoop_add_compiler_flags("${GCC_SHARED_FLAGS}")
+hadoop_add_linker_flags("${LINKER_SHARED_FLAGS}")
+
+#
+# Linux-specific configuration.
+#
+if(CMAKE_SYSTEM_NAME STREQUAL "Linux")
+    # Make GNU extensions available.
+    hadoop_add_compiler_flags("-D_GNU_SOURCE")
+
+    # If JVM_ARCH_DATA_MODEL is 32, compile all binaries as 32-bit.
+    if(JVM_ARCH_DATA_MODEL EQUAL 32)
+        # Force 32-bit code generation on amd64/x86_64, ppc64, sparc64
+        if(CMAKE_COMPILER_IS_GNUCC AND CMAKE_SYSTEM_PROCESSOR MATCHES ".*64")
+            hadoop_add_compiler_flags("-m32")
+            hadoop_add_linker_flags("-m32")
+        endif()
+        # Set CMAKE_SYSTEM_PROCESSOR to ensure that find_package(JNI) will use 32-bit libraries
+        if(CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "amd64")
+            set(CMAKE_SYSTEM_PROCESSOR "i686")
+        endif()
+    endif()
+
+    # Determine float ABI of JVM on ARM.
+    if(CMAKE_SYSTEM_PROCESSOR MATCHES "^arm")
+        find_program(READELF readelf)
+        if(READELF MATCHES "NOTFOUND")
+            message(WARNING "readelf not found; JVM float ABI detection disabled")
+        else(READELF MATCHES "NOTFOUND")
+            execute_process(
+                COMMAND ${READELF} -A ${JAVA_JVM_LIBRARY}
+                OUTPUT_VARIABLE JVM_ELF_ARCH
+                ERROR_QUIET)
+            if(NOT JVM_ELF_ARCH MATCHES "Tag_ABI_VFP_args: VFP registers")
+                # Test compilation with -mfloat-abi=softfp using an arbitrary libc function
+                # (typically fails with "fatal error: bits/predefs.h: No such file or directory"
+                # if soft-float dev libraries are not installed)
+                message("Soft-float JVM detected")
+                include(CMakePushCheckState)
+                cmake_push_check_state()
+                set(CMAKE_REQUIRED_FLAGS "${CMAKE_REQUIRED_FLAGS} -mfloat-abi=softfp")
+                include(CheckSymbolExists)
+                check_symbol_exists(exit stdlib.h SOFTFP_AVAILABLE)
+                if(NOT SOFTFP_AVAILABLE)
+                    message(FATAL_ERROR "Soft-float dev libraries required (e.g. 'apt-get install libc6-dev-armel' on Debian/Ubuntu)")
+                endif()
+                cmake_pop_check_state()
+                hadoop_add_compiler_flags("-mfloat-abi=softfp")
+            endif()
+        endif()
+    endif()
+
+#
+# Solaris-specific configuration.
+#
+elseif(CMAKE_SYSTEM_NAME STREQUAL "SunOS")
+    # Solaris flags. 64-bit compilation is mandatory, and is checked earlier.
+    hadoop_add_compiler_flags("-m64 -D__EXTENSIONS__ -D_POSIX_PTHREAD_SEMANTICS -D_XOPEN_SOURCE=500")
+    hadoop_add_linker_flags("-m64")
+
+    # CMAKE_SYSTEM_PROCESSOR is set to the output of 'uname -p', which on Solaris is
+    # the 'lowest' ISA supported, i.e. 'i386' or 'sparc'. However in order for the
+    # standard CMake modules to look in the right places it needs to reflect the required
+    # compilation mode, i.e. 64 bit. We therefore force it to either 'amd64' or 'sparcv9'.
+    if(CMAKE_SYSTEM_PROCESSOR STREQUAL "i386")
+        set(CMAKE_SYSTEM_PROCESSOR "amd64")
+        set(CMAKE_LIBRARY_ARCHITECTURE "amd64")
+    elseif(CMAKE_SYSTEM_PROCESSOR STREQUAL "sparc")
+        set(CMAKE_SYSTEM_PROCESSOR STREQUAL "sparcv9")
+        set(CMAKE_LIBRARY_ARCHITECTURE "sparcv9")
+    else()
+        message(FATAL_ERROR "Unrecognised CMAKE_SYSTEM_PROCESSOR ${CMAKE_SYSTEM_PROCESSOR}")
+    endif()
+endif()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa07dea3/hadoop-common-project/hadoop-common/HadoopJNI.cmake
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/HadoopJNI.cmake b/hadoop-common-project/hadoop-common/HadoopJNI.cmake
new file mode 100644
index 0000000..78d7ffd
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/HadoopJNI.cmake
@@ -0,0 +1,97 @@
+#
+# 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.
+#
+
+#
+# Common JNI detection for CMake, shared by all Native components.
+#
+
+# Check the JVM_ARCH_DATA_MODEL variable as been set to 32 or 64 by maven.
+if(NOT DEFINED JVM_ARCH_DATA_MODEL)
+    message(FATAL_ERROR "JVM_ARCH_DATA_MODEL is not defined")
+elseif(NOT (JVM_ARCH_DATA_MODEL EQUAL 32 OR JVM_ARCH_DATA_MODEL EQUAL 64))
+    message(FATAL_ERROR "JVM_ARCH_DATA_MODEL is not 32 or 64")
+endif()
+
+#
+# Linux-specific JNI configuration.
+#
+if(CMAKE_SYSTEM_NAME STREQUAL "Linux")
+    # Locate JNI_INCLUDE_DIRS and JNI_LIBRARIES.
+    # Since we were invoked from Maven, we know that the JAVA_HOME environment
+    # variable is valid.  So we ignore system paths here and just use JAVA_HOME.
+    file(TO_CMAKE_PATH "$ENV{JAVA_HOME}" _java_home)
+    if(CMAKE_SYSTEM_PROCESSOR MATCHES "^i.86$")
+        set(_java_libarch "i386")
+    elseif(CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "amd64")
+        set(_java_libarch "amd64")
+    elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "^arm")
+        set(_java_libarch "arm")
+    elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "^(powerpc|ppc)64le")
+        if(EXISTS "${_java_home}/jre/lib/ppc64le")
+            set(_java_libarch "ppc64le")
+        else()
+            set(_java_libarch "ppc64")
+        endif()
+    else()
+        set(_java_libarch ${CMAKE_SYSTEM_PROCESSOR})
+    endif()
+    set(_JDK_DIRS "${_java_home}/jre/lib/${_java_libarch}/*"
+                  "${_java_home}/jre/lib/${_java_libarch}"
+                  "${_java_home}/jre/lib/*"
+                  "${_java_home}/jre/lib"
+                  "${_java_home}/lib/*"
+                  "${_java_home}/lib"
+                  "${_java_home}/include/*"
+                  "${_java_home}/include"
+                  "${_java_home}"
+    )
+    find_path(JAVA_INCLUDE_PATH
+        NAMES jni.h
+        PATHS ${_JDK_DIRS}
+        NO_DEFAULT_PATH)
+    #In IBM java, it's jniport.h instead of jni_md.h
+    find_path(JAVA_INCLUDE_PATH2
+        NAMES jni_md.h jniport.h
+        PATHS ${_JDK_DIRS}
+        NO_DEFAULT_PATH)
+    set(JNI_INCLUDE_DIRS ${JAVA_INCLUDE_PATH} ${JAVA_INCLUDE_PATH2})
+    find_library(JAVA_JVM_LIBRARY
+        NAMES jvm JavaVM
+        PATHS ${_JDK_DIRS}
+        NO_DEFAULT_PATH)
+    set(JNI_LIBRARIES ${JAVA_JVM_LIBRARY})
+    unset(_java_libarch)
+    unset(_java_home)
+
+    message("JAVA_HOME=${JAVA_HOME}, JAVA_JVM_LIBRARY=${JAVA_JVM_LIBRARY}")
+    message("JAVA_INCLUDE_PATH=${JAVA_INCLUDE_PATH}, JAVA_INCLUDE_PATH2=${JAVA_INCLUDE_PATH2}")
+    if(JAVA_JVM_LIBRARY AND JAVA_INCLUDE_PATH AND JAVA_INCLUDE_PATH2)
+        message("Located all JNI components successfully.")
+    else()
+        message(FATAL_ERROR "Failed to find a viable JVM installation under JAVA_HOME.")
+    endif()
+
+    # Use the standard FindJNI module to locate the JNI components.
+    find_package(JNI REQUIRED)
+
+#
+# Otherwise, use the standard FindJNI module to locate the JNI components.
+#
+else()
+    find_package(JNI REQUIRED)
+endif()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa07dea3/hadoop-common-project/hadoop-common/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/CMakeLists.txt b/hadoop-common-project/hadoop-common/src/CMakeLists.txt
index 7d68fd7..c93bfe7 100644
--- a/hadoop-common-project/hadoop-common/src/CMakeLists.txt
+++ b/hadoop-common-project/hadoop-common/src/CMakeLists.txt
@@ -16,209 +16,149 @@
 # limitations under the License.
 #
 
-cmake_minimum_required(VERSION 2.6 FATAL_ERROR)
-
-# Default to release builds
-set(CMAKE_BUILD_TYPE, Release)
-
-include(JNIFlags.cmake NO_POLICY_SCOPE)
-
-# Compile a library with both shared and static variants
-function(add_dual_library LIBNAME)
-    add_library(${LIBNAME} SHARED ${ARGN})
-    add_library(${LIBNAME}_static STATIC ${ARGN})
-    set_target_properties(${LIBNAME}_static PROPERTIES OUTPUT_NAME ${LIBNAME})
-endfunction(add_dual_library)
+#
+# CMake configuration.
+#
 
-# Link both a static and a dynamic target against some libraries
-function(target_link_dual_libraries LIBNAME)
-    target_link_libraries(${LIBNAME} ${ARGN})
-    target_link_libraries(${LIBNAME}_static ${ARGN})
-endfunction(target_link_dual_libraries)
+cmake_minimum_required(VERSION 2.6 FATAL_ERROR)
 
-function(output_directory TGT DIR)
-    SET_TARGET_PROPERTIES(${TGT} PROPERTIES
-        RUNTIME_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-    SET_TARGET_PROPERTIES(${TGT} PROPERTIES
-        ARCHIVE_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-    SET_TARGET_PROPERTIES(${TGT} PROPERTIES
-        LIBRARY_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-endfunction(output_directory TGT DIR)
+list(APPEND CMAKE_MODULE_PATH ${CMAKE_SOURCE_DIR}/..)
+include(HadoopCommon)
 
-function(dual_output_directory TGT DIR)
-    output_directory(${TGT} "${DIR}")
-    output_directory(${TGT}_static "${DIR}")
-endfunction(dual_output_directory TGT DIR)
+# Source and test locations.
+set(SRC main/native/src/org/apache/hadoop)
+set(TST main/native/src/test/org/apache/hadoop)
 
 #
-# This macro alters the behavior of find_package and find_library.
-# It does this by setting the CMAKE_FIND_LIBRARY_SUFFIXES global variable. 
-# You should save that variable before calling this function and restore it
-# after you have accomplished your goal.
+# Main configuration.
 #
-# The behavior is altered in two ways:
-# 1. We always find shared libraries, never static;
-# 2. We find shared libraries with the given version number.
-#
-# On Windows this function is a no-op.  Windows does not encode
-# version number information information into library path names.
-#
-macro(set_find_shared_library_version LVERS)
-    IF(${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
-        # Mac OS uses .dylib
-        SET(CMAKE_FIND_LIBRARY_SUFFIXES ".${LVERS}.dylib")
-    ELSEIF(${CMAKE_SYSTEM_NAME} MATCHES "FreeBSD")
-        # FreeBSD has always .so installed.
-        SET(CMAKE_FIND_LIBRARY_SUFFIXES ".so")
-    ELSEIF(${CMAKE_SYSTEM_NAME} MATCHES "Windows")
-        # Windows doesn't support finding shared libraries by version.
-    ELSE()
-        # Most UNIX variants use .so
-        SET(CMAKE_FIND_LIBRARY_SUFFIXES ".so.${LVERS}")
-    ENDIF()
-endmacro(set_find_shared_library_version LVERS)
 
-#
-# Alter the behavior of find_package and find_library so that we find only
-# shared libraries without any version suffix.  You should save
-# CMAKE_FIND_LIBRARY_SUFFIXES before calling this function and restore it
-# afterwards.
-#
-macro(set_find_shared_library_without_version)
-    IF(${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
-        # Mac OS uses .dylib
-        SET(CMAKE_FIND_LIBRARY_SUFFIXES ".dylib")
-    ELSEIF(${CMAKE_SYSTEM_NAME} MATCHES "Windows")
-        # No effect
-    ELSE()
-        # Most UNIX variants use .so
-        SET(CMAKE_FIND_LIBRARY_SUFFIXES ".so")
-    ENDIF()
-endmacro(set_find_shared_library_without_version)
+# The caller must specify where the generated headers have been placed.
+if(NOT GENERATED_JAVAH)
+    message(FATAL_ERROR "You must set the CMake variable GENERATED_JAVAH")
+endif()
 
-if (NOT GENERATED_JAVAH)
-    # Must identify where the generated headers have been placed
-    MESSAGE(FATAL_ERROR "You must set the cmake variable GENERATED_JAVAH")
-endif (NOT GENERATED_JAVAH)
-find_package(JNI REQUIRED)
+# Configure JNI.
+include(HadoopJNI)
 
-SET(STORED_CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES})
-set_find_shared_library_version("1")
+# Require zlib.
+set(STORED_CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES})
+hadoop_set_find_shared_library_version("1")
 find_package(ZLIB REQUIRED)
-SET(CMAKE_FIND_LIBRARY_SUFFIXES ${STORED_CMAKE_FIND_LIBRARY_SUFFIXES})
-
-set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -g -Wall -O2")
-set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_REENTRANT -D_GNU_SOURCE")
-set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_LARGEFILE_SOURCE -D_FILE_OFFSET_BITS=64")
-set(D main/native/src/org/apache/hadoop)
-set(T main/native/src/test/org/apache/hadoop)
-
-GET_FILENAME_COMPONENT(HADOOP_ZLIB_LIBRARY ${ZLIB_LIBRARIES} NAME)
+set(CMAKE_FIND_LIBRARY_SUFFIXES ${STORED_CMAKE_FIND_LIBRARY_SUFFIXES})
+get_filename_component(HADOOP_ZLIB_LIBRARY ${ZLIB_LIBRARIES} NAME)
 
-SET(STORED_CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES})
-set_find_shared_library_version("1")
+# Look for bzip2.
+set(STORED_CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES})
+hadoop_set_find_shared_library_version("1")
 find_package(BZip2 QUIET)
-if (BZIP2_INCLUDE_DIR AND BZIP2_LIBRARIES)
-    GET_FILENAME_COMPONENT(HADOOP_BZIP2_LIBRARY ${BZIP2_LIBRARIES} NAME)
+if(BZIP2_INCLUDE_DIR AND BZIP2_LIBRARIES)
+    get_filename_component(HADOOP_BZIP2_LIBRARY ${BZIP2_LIBRARIES} NAME)
     set(BZIP2_SOURCE_FILES
-          "${D}/io/compress/bzip2/Bzip2Compressor.c"
-          "${D}/io/compress/bzip2/Bzip2Decompressor.c")
-else (BZIP2_INCLUDE_DIR AND BZIP2_LIBRARIES)
+          "${SRC}/io/compress/bzip2/Bzip2Compressor.c"
+          "${SRC}/io/compress/bzip2/Bzip2Decompressor.c")
+    set(REQUIRE_BZIP2 ${REQUIRE_BZIP2}) # Stop warning about unused variable.
+else()
     set(BZIP2_SOURCE_FILES "")
     set(BZIP2_INCLUDE_DIR "")
-    IF(REQUIRE_BZIP2)
-        MESSAGE(FATAL_ERROR "Required bzip2 library and/or header files could not be found.")
-    ENDIF(REQUIRE_BZIP2)
-endif (BZIP2_INCLUDE_DIR AND BZIP2_LIBRARIES)
-SET(CMAKE_FIND_LIBRARY_SUFFIXES ${STORED_CMAKE_FIND_LIBRARY_SUFFIXES})
-
-INCLUDE(CheckFunctionExists)
-INCLUDE(CheckCSourceCompiles)
-INCLUDE(CheckLibraryExists)
-CHECK_FUNCTION_EXISTS(sync_file_range HAVE_SYNC_FILE_RANGE)
-CHECK_FUNCTION_EXISTS(posix_fadvise HAVE_POSIX_FADVISE)
-CHECK_LIBRARY_EXISTS(dl dlopen "" NEED_LINK_DL)
-
-SET(STORED_CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES})
-set_find_shared_library_version("1")
-find_library(SNAPPY_LIBRARY 
+    if(REQUIRE_BZIP2)
+        message(FATAL_ERROR "Required bzip2 library and/or header files could not be found.")
+    endif()
+endif()
+set(CMAKE_FIND_LIBRARY_SUFFIXES ${STORED_CMAKE_FIND_LIBRARY_SUFFIXES})
+
+# Require snappy.
+set(STORED_CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES})
+hadoop_set_find_shared_library_version("1")
+find_library(SNAPPY_LIBRARY
     NAMES snappy
     PATHS ${CUSTOM_SNAPPY_PREFIX} ${CUSTOM_SNAPPY_PREFIX}/lib
           ${CUSTOM_SNAPPY_PREFIX}/lib64 ${CUSTOM_SNAPPY_LIB})
-SET(CMAKE_FIND_LIBRARY_SUFFIXES ${STORED_CMAKE_FIND_LIBRARY_SUFFIXES})
-find_path(SNAPPY_INCLUDE_DIR 
+set(CMAKE_FIND_LIBRARY_SUFFIXES ${STORED_CMAKE_FIND_LIBRARY_SUFFIXES})
+find_path(SNAPPY_INCLUDE_DIR
     NAMES snappy.h
     PATHS ${CUSTOM_SNAPPY_PREFIX} ${CUSTOM_SNAPPY_PREFIX}/include
           ${CUSTOM_SNAPPY_INCLUDE})
-if (SNAPPY_LIBRARY AND SNAPPY_INCLUDE_DIR)
-    GET_FILENAME_COMPONENT(HADOOP_SNAPPY_LIBRARY ${SNAPPY_LIBRARY} NAME)
+if(SNAPPY_LIBRARY AND SNAPPY_INCLUDE_DIR)
+    get_filename_component(HADOOP_SNAPPY_LIBRARY ${SNAPPY_LIBRARY} NAME)
     set(SNAPPY_SOURCE_FILES
-        "${D}/io/compress/snappy/SnappyCompressor.c"
-        "${D}/io/compress/snappy/SnappyDecompressor.c")
-else (SNAPPY_LIBRARY AND SNAPPY_INCLUDE_DIR)
+        "${SRC}/io/compress/snappy/SnappyCompressor.c"
+        "${SRC}/io/compress/snappy/SnappyDecompressor.c")
+    set(REQUIRE_SNAPPY ${REQUIRE_SNAPPY}) # Stop warning about unused variable.
+    message(STATUS "Found Snappy: ${SNAPPY_LIBRARY}")
+else()
     set(SNAPPY_INCLUDE_DIR "")
     set(SNAPPY_SOURCE_FILES "")
-    IF(REQUIRE_SNAPPY)
-        MESSAGE(FATAL_ERROR "Required snappy library could not be found.  SNAPPY_LIBRARY=${SNAPPY_LIBRARY}, SNAPPY_INCLUDE_DIR=${SNAPPY_INCLUDE_DIR}, CUSTOM_SNAPPY_INCLUDE_DIR=${CUSTOM_SNAPPY_INCLUDE_DIR}, CUSTOM_SNAPPY_PREFIX=${CUSTOM_SNAPPY_PREFIX}, CUSTOM_SNAPPY_INCLUDE=${CUSTOM_SNAPPY_INCLUDE}")
-    ENDIF(REQUIRE_SNAPPY)
-endif (SNAPPY_LIBRARY AND SNAPPY_INCLUDE_DIR)
-
-IF (CMAKE_SYSTEM_PROCESSOR MATCHES "^i.86$" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "amd64")
-  set(BULK_CRC_ARCH_SOURCE_FIlE "${D}/util/bulk_crc32_x86.c")
-ELSEIF (CMAKE_SYSTEM_PROCESSOR STREQUAL "aarch64")
-  set(BULK_CRC_ARCH_SOURCE_FIlE "${D}/util/bulk_crc32_aarch64.c")
-ELSE()
-  MESSAGE("No HW CRC acceleration for ${CMAKE_SYSTEM_PROCESSOR}, falling back to SW")
-ENDIF()
-
-# Find the no-suffix version of libcrypto.
-# See HADOOP-11216 for details.
-SET(STORED_CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES})
-set_find_shared_library_without_version()
-SET(OPENSSL_NAME "crypto")
-IF(${CMAKE_SYSTEM_NAME} MATCHES "Windows")
+    if(REQUIRE_SNAPPY)
+        message(FATAL_ERROR "Required snappy library could not be found.  SNAPPY_LIBRARY=${SNAPPY_LIBRARY}, SNAPPY_INCLUDE_DIR=${SNAPPY_INCLUDE_DIR}, CUSTOM_SNAPPY_INCLUDE_DIR=${CUSTOM_SNAPPY_INCLUDE_DIR}, CUSTOM_SNAPPY_PREFIX=${CUSTOM_SNAPPY_PREFIX}, CUSTOM_SNAPPY_INCLUDE=${CUSTOM_SNAPPY_INCLUDE}")
+    endif()
+endif()
+
+# Build hardware CRC32 acceleration, if supported on the platform.
+if(CMAKE_SYSTEM_PROCESSOR MATCHES "^i.86$" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "amd64")
+  set(BULK_CRC_ARCH_SOURCE_FIlE "${SRC}/util/bulk_crc32_x86.c")
+elseif(CMAKE_SYSTEM_PROCESSOR STREQUAL "aarch64")
+  set(BULK_CRC_ARCH_SOURCE_FIlE "${SRC}/util/bulk_crc32_aarch64.c")
+else()
+  message("No HW CRC acceleration for ${CMAKE_SYSTEM_PROCESSOR}, falling back to SW")
+endif()
+
+# Find the no-suffix version of libcrypto/openssl. See HADOOP-11216 for details.
+set(STORED_CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES})
+hadoop_set_find_shared_library_without_version()
+set(OPENSSL_NAME "crypto")
+if(${CMAKE_SYSTEM_NAME} MATCHES "Windows")
     SET(OPENSSL_NAME "eay32")
-ENDIF()
-MESSAGE("CUSTOM_OPENSSL_PREFIX = ${CUSTOM_OPENSSL_PREFIX}")
+endif()
+message("CUSTOM_OPENSSL_PREFIX = ${CUSTOM_OPENSSL_PREFIX}")
 find_library(OPENSSL_LIBRARY
     NAMES ${OPENSSL_NAME}
     PATHS ${CUSTOM_OPENSSL_PREFIX} ${CUSTOM_OPENSSL_PREFIX}/lib
           ${CUSTOM_OPENSSL_PREFIX}/lib64 ${CUSTOM_OPENSSL_LIB} NO_DEFAULT_PATH)
 find_library(OPENSSL_LIBRARY NAMES ${OPENSSL_NAME})
-find_path(OPENSSL_INCLUDE_DIR 
+find_path(OPENSSL_INCLUDE_DIR
     NAMES openssl/evp.h
     PATHS ${CUSTOM_OPENSSL_PREFIX} ${CUSTOM_OPENSSL_PREFIX}/include
           ${CUSTOM_OPENSSL_INCLUDE} NO_DEFAULT_PATH)
 find_path(OPENSSL_INCLUDE_DIR NAMES openssl/evp.h)
-SET(CMAKE_FIND_LIBRARY_SUFFIXES ${STORED_CMAKE_FIND_LIBRARY_SUFFIXES})
-SET(USABLE_OPENSSL 0)
-if (OPENSSL_LIBRARY AND OPENSSL_INCLUDE_DIR)
-    INCLUDE(CheckCSourceCompiles)
-    SET(OLD_CMAKE_REQUIRED_INCLUDES ${CMAKE_REQUIRED_INCLUDES})
-    SET(CMAKE_REQUIRED_INCLUDES ${OPENSSL_INCLUDE_DIR})
-    CHECK_C_SOURCE_COMPILES("#include \"${OPENSSL_INCLUDE_DIR}/openssl/evp.h\"\nint main(int argc, char **argv) { return !EVP_aes_256_ctr; }" HAS_NEW_ENOUGH_OPENSSL)
-    SET(CMAKE_REQUIRED_INCLUDES ${OLD_CMAKE_REQUIRED_INCLUDES})
+set(CMAKE_FIND_LIBRARY_SUFFIXES ${STORED_CMAKE_FIND_LIBRARY_SUFFIXES})
+set(USABLE_OPENSSL 0)
+if(OPENSSL_LIBRARY AND OPENSSL_INCLUDE_DIR)
+    include(CheckCSourceCompiles)
+    set(OLD_CMAKE_REQUIRED_INCLUDES ${CMAKE_REQUIRED_INCLUDES})
+    set(CMAKE_REQUIRED_INCLUDES ${OPENSSL_INCLUDE_DIR})
+    check_c_source_compiles("#include \"${OPENSSL_INCLUDE_DIR}/openssl/evp.h\"\nint main(int argc, char **argv) { return !EVP_aes_256_ctr; }" HAS_NEW_ENOUGH_OPENSSL)
+    set(CMAKE_REQUIRED_INCLUDES ${OLD_CMAKE_REQUIRED_INCLUDES})
     if(NOT HAS_NEW_ENOUGH_OPENSSL)
-        MESSAGE("The OpenSSL library installed at ${OPENSSL_LIBRARY} is too old.  You need a version at least new enough to have EVP_aes_256_ctr.")
-    else(NOT HAS_NEW_ENOUGH_OPENSSL)
+        message("The OpenSSL library installed at ${OPENSSL_LIBRARY} is too old.  You need a version at least new enough to have EVP_aes_256_ctr.")
+    else()
         SET(USABLE_OPENSSL 1)
-    endif(NOT HAS_NEW_ENOUGH_OPENSSL)
-endif (OPENSSL_LIBRARY AND OPENSSL_INCLUDE_DIR)
-if (USABLE_OPENSSL)
-    GET_FILENAME_COMPONENT(HADOOP_OPENSSL_LIBRARY ${OPENSSL_LIBRARY} NAME)
-    SET(OPENSSL_SOURCE_FILES
-        "${D}/crypto/OpensslCipher.c"
-        "${D}/crypto/random/OpensslSecureRandom.c")
-else (USABLE_OPENSSL)
-    MESSAGE("Cannot find a usable OpenSSL library.  OPENSSL_LIBRARY=${OPENSSL_LIBRARY}, OPENSSL_INCLUDE_DIR=${OPENSSL_INCLUDE_DIR}, CUSTOM_OPENSSL_LIB=${CUSTOM_OPENSSL_LIB}, CUSTOM_OPENSSL_PREFIX=${CUSTOM_OPENSSL_PREFIX}, CUSTOM_OPENSSL_INCLUDE=${CUSTOM_OPENSSL_INCLUDE}")
-    IF(REQUIRE_OPENSSL)
-        MESSAGE(FATAL_ERROR "Terminating build because require.openssl was specified.")
-    ENDIF(REQUIRE_OPENSSL)
-    SET(OPENSSL_LIBRARY "")
-    SET(OPENSSL_INCLUDE_DIR "")
-    SET(OPENSSL_SOURCE_FILES "")
-endif (USABLE_OPENSSL)
-
+    endif()
+endif()
+if(USABLE_OPENSSL)
+    get_filename_component(HADOOP_OPENSSL_LIBRARY ${OPENSSL_LIBRARY} NAME)
+    set(OPENSSL_SOURCE_FILES
+        "${SRC}/crypto/OpensslCipher.c"
+        "${SRC}/crypto/random/OpensslSecureRandom.c")
+    set(REQUIRE_OPENSSL ${REQUIRE_OPENSSL}) # Stop warning about unused variable.
+else()
+    message("Cannot find a usable OpenSSL library. OPENSSL_LIBRARY=${OPENSSL_LIBRARY}, OPENSSL_INCLUDE_DIR=${OPENSSL_INCLUDE_DIR}, CUSTOM_OPENSSL_LIB=${CUSTOM_OPENSSL_LIB}, CUSTOM_OPENSSL_PREFIX=${CUSTOM_OPENSSL_PREFIX}, CUSTOM_OPENSSL_INCLUDE=${CUSTOM_OPENSSL_INCLUDE}")
+    if(REQUIRE_OPENSSL)
+        message(FATAL_ERROR "Terminating build because require.openssl was specified.")
+    endif()
+    set(OPENSSL_LIBRARY "")
+    set(OPENSSL_INCLUDE_DIR "")
+    set(OPENSSL_SOURCE_FILES "")
+endif()
+
+# Check for platform-specific functions and libraries.
+include(CheckFunctionExists)
+include(CheckLibraryExists)
+check_function_exists(sync_file_range HAVE_SYNC_FILE_RANGE)
+check_function_exists(posix_fadvise HAVE_POSIX_FADVISE)
+check_library_exists(dl dlopen "" NEED_LINK_DL)
+
+# Configure the build.
 include_directories(
     ${GENERATED_JAVAH}
     main/native/src
@@ -230,66 +170,60 @@ include_directories(
     ${BZIP2_INCLUDE_DIR}
     ${SNAPPY_INCLUDE_DIR}
     ${OPENSSL_INCLUDE_DIR}
-    ${D}/util
-)
-CONFIGURE_FILE(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h)
-
-add_executable(test_bulk_crc32
-    ${D}/util/bulk_crc32.c
-    ${BULK_CRC_ARCH_SOURCE_FIlE}
-    ${T}/util/test_bulk_crc32.c
+    ${SRC}/util
 )
+configure_file(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h)
 
-SET(CMAKE_BUILD_WITH_INSTALL_RPATH TRUE)
-add_dual_library(hadoop
+set(CMAKE_BUILD_WITH_INSTALL_RPATH TRUE)
+hadoop_add_dual_library(hadoop
     main/native/src/exception.c
-    ${D}/io/compress/lz4/Lz4Compressor.c
-    ${D}/io/compress/lz4/Lz4Decompressor.c
-    ${D}/io/compress/lz4/lz4.c
-    ${D}/io/compress/lz4/lz4hc.c
+    ${SRC}/io/compress/lz4/Lz4Compressor.c
+    ${SRC}/io/compress/lz4/Lz4Decompressor.c
+    ${SRC}/io/compress/lz4/lz4.c
+    ${SRC}/io/compress/lz4/lz4hc.c
     ${SNAPPY_SOURCE_FILES}
     ${OPENSSL_SOURCE_FILES}
-    ${D}/io/compress/zlib/ZlibCompressor.c
-    ${D}/io/compress/zlib/ZlibDecompressor.c
+    ${SRC}/io/compress/zlib/ZlibCompressor.c
+    ${SRC}/io/compress/zlib/ZlibDecompressor.c
     ${BZIP2_SOURCE_FILES}
-    ${D}/io/nativeio/NativeIO.c
-    ${D}/io/nativeio/errno_enum.c
-    ${D}/io/nativeio/file_descriptor.c
-    ${D}/io/nativeio/SharedFileDescriptorFactory.c
-    ${D}/net/unix/DomainSocket.c
-    ${D}/net/unix/DomainSocketWatcher.c
-    ${D}/security/JniBasedUnixGroupsMapping.c
-    ${D}/security/JniBasedUnixGroupsNetgroupMapping.c
-    ${D}/security/hadoop_group_info.c
-    ${D}/security/hadoop_user_info.c
-    ${D}/util/NativeCodeLoader.c
-    ${D}/util/NativeCrc32.c
-    ${D}/util/bulk_crc32.c
+    ${SRC}/io/nativeio/NativeIO.c
+    ${SRC}/io/nativeio/errno_enum.c
+    ${SRC}/io/nativeio/file_descriptor.c
+    ${SRC}/io/nativeio/SharedFileDescriptorFactory.c
+    ${SRC}/net/unix/DomainSocket.c
+    ${SRC}/net/unix/DomainSocketWatcher.c
+    ${SRC}/security/JniBasedUnixGroupsMapping.c
+    ${SRC}/security/JniBasedUnixGroupsNetgroupMapping.c
+    ${SRC}/security/hadoop_group_info.c
+    ${SRC}/security/hadoop_user_info.c
+    ${SRC}/util/NativeCodeLoader.c
+    ${SRC}/util/NativeCrc32.c
+    ${SRC}/util/bulk_crc32.c
     ${BULK_CRC_ARCH_SOURCE_FIlE}
 )
-if (NEED_LINK_DL)
+if(NEED_LINK_DL)
    set(LIB_DL dl)
-endif (NEED_LINK_DL)
+endif()
 
-IF (${CMAKE_SYSTEM_NAME} MATCHES "Linux")
-    #
-    # By embedding '$ORIGIN' into the RPATH of libhadoop.so,
-    # dlopen will look in the directory containing libhadoop.so.
-    # However, $ORIGIN is not supported by all operating systems.
-    #
+hadoop_target_link_dual_libraries(hadoop ${LIB_DL} ${JAVA_JVM_LIBRARY})
+set(LIBHADOOP_VERSION "1.0.0")
+set_target_properties(hadoop PROPERTIES SOVERSION ${LIBHADOOP_VERSION})
+hadoop_dual_output_directory(hadoop target/usr/local/lib)
+
+# By embedding '$ORIGIN' into the RPATH of libhadoop.so, dlopen will look in
+# the directory containing libhadoop.so. However, $ORIGIN is not supported by
+# all operating systems.
+if(${CMAKE_SYSTEM_NAME} MATCHES "Linux|SunOS")
     set(RPATH "\$ORIGIN/")
-    if (EXTRA_LIBHADOOP_RPATH)
+    if(EXTRA_LIBHADOOP_RPATH)
         set(RPATH "${RPATH}:${EXTRA_LIBHADOOP_RPATH}/")
-    endif(EXTRA_LIBHADOOP_RPATH)
-    SET_TARGET_PROPERTIES(hadoop 
-        PROPERTIES INSTALL_RPATH "${RPATH}")
-ENDIF()
+    endif()
+    set_target_properties(hadoop PROPERTIES INSTALL_RPATH "${RPATH}")
+endif()
 
-target_link_dual_libraries(hadoop
-    ${LIB_DL}
-    ${JAVA_JVM_LIBRARY}
+# Build the CRC32 test executable.
+add_executable(test_bulk_crc32
+    ${SRC}/util/bulk_crc32.c
+    ${BULK_CRC_ARCH_SOURCE_FIlE}
+    ${TST}/util/test_bulk_crc32.c
 )
-SET(LIBHADOOP_VERSION "1.0.0")
-SET_TARGET_PROPERTIES(hadoop PROPERTIES
-    SOVERSION ${LIBHADOOP_VERSION})
-dual_output_directory(hadoop target/usr/local/lib)


[26/50] hadoop git commit: HADOOP-12089. StorageException complaining " no lease ID" when updating FolderLastModifiedTime in WASB. Contributed by Duo Xu.

Posted by vv...@apache.org.
HADOOP-12089. StorageException complaining " no lease ID" when updating FolderLastModifiedTime in WASB. Contributed by Duo Xu.


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

Branch: refs/heads/YARN-2139
Commit: 460e98f7b3ec84f3c5afcb2aad4f4e7031d16e3a
Parents: 62e583c
Author: cnauroth <cn...@apache.org>
Authored: Mon Jun 29 13:48:02 2015 -0700
Committer: cnauroth <cn...@apache.org>
Committed: Mon Jun 29 13:48:02 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                 | 3 +++
 .../java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java  | 5 ++---
 2 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/460e98f7/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 9008ead..a9b44e3 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -893,6 +893,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12119. hadoop fs -expunge does not work for federated namespace
     (J.Andreina via vinayakumarb)
 
+    HADOOP-12089. StorageException complaining " no lease ID" when updating
+    FolderLastModifiedTime in WASB. (Duo Xu via cnauroth)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/460e98f7/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
index 623645a..a567b33 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
@@ -1500,7 +1500,7 @@ public class NativeAzureFileSystem extends FileSystem {
               createPermissionStatus(FsPermission.getDefault()));
         } else {
           if (!skipParentFolderLastModifidedTimeUpdate) {
-            store.updateFolderLastModifiedTime(parentKey, null);
+            updateParentFolderLastModifiedTime(key);
           }
         }
       }
@@ -1561,9 +1561,8 @@ public class NativeAzureFileSystem extends FileSystem {
       // Update parent directory last modified time
       Path parent = absolutePath.getParent();
       if (parent != null && parent.getParent() != null) { // not root
-        String parentKey = pathToKey(parent);
         if (!skipParentFolderLastModifidedTimeUpdate) {
-          store.updateFolderLastModifiedTime(parentKey, null);
+          updateParentFolderLastModifiedTime(key);
         }
       }
       instrumentation.directoryDeleted();


[15/50] hadoop git commit: HDFS-8681. BlockScanner is incorrectly disabled by default. (Contributed by Arpit Agarwal)

Posted by vv...@apache.org.
HDFS-8681. BlockScanner is incorrectly disabled by default. (Contributed by Arpit Agarwal)


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

Branch: refs/heads/YARN-2139
Commit: c6793dd8cc69ea994eb23c3e1349efe4b9feca9a
Parents: 3dfa816
Author: Arpit Agarwal <ar...@apache.org>
Authored: Sun Jun 28 14:51:17 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Sun Jun 28 14:51:36 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  2 +-
 .../hdfs/server/datanode/BlockScanner.java      | 36 ++++++++++++++++----
 .../src/main/resources/hdfs-default.xml         |  9 +++--
 .../fsdataset/impl/TestFsDatasetImpl.java       |  1 +
 5 files changed, 41 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6793dd8/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e287ea4..94477fe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1113,6 +1113,9 @@ Release 2.7.1 - UNRELEASED
     HDFS08656. Preserve compatibility of ClientProtocol#rollingUpgrade after
     finalization. (wang)
 
+    HDFS-8681. BlockScanner is incorrectly disabled by default.
+    (Arpit Agarwal)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6793dd8/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 ebd668f..0e569f0 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
@@ -378,7 +378,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_DATANODE_MAX_RECEIVER_THREADS_KEY = "dfs.datanode.max.transfer.threads";
   public static final int     DFS_DATANODE_MAX_RECEIVER_THREADS_DEFAULT = 4096;
   public static final String  DFS_DATANODE_SCAN_PERIOD_HOURS_KEY = "dfs.datanode.scan.period.hours";
-  public static final int     DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT = 0;
+  public static final int     DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT = 21 * 24;  // 3 weeks.
   public static final String  DFS_BLOCK_SCANNER_VOLUME_BYTES_PER_SECOND = "dfs.block.scanner.volume.bytes.per.second";
   public static final long    DFS_BLOCK_SCANNER_VOLUME_BYTES_PER_SECOND_DEFAULT = 1048576L;
   public static final String  DFS_DATANODE_TRANSFERTO_ALLOWED_KEY = "dfs.datanode.transferTo.allowed";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6793dd8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
index b0248c5..9c4dd10 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
@@ -115,6 +115,34 @@ public class BlockScanner {
       }
     }
 
+    /**
+     * Determine the configured block scanner interval.
+     *
+     * For compatibility with prior releases of HDFS, if the
+     * configured value is zero then the scan period is
+     * set to 3 weeks.
+     *
+     * If the configured value is less than zero then the scanner
+     * is disabled.
+     *
+     * @param conf Configuration object.
+     * @return block scan period in milliseconds.
+     */
+    private static long getConfiguredScanPeriodMs(Configuration conf) {
+      long tempScanPeriodMs = getUnitTestLong(
+          conf, INTERNAL_DFS_DATANODE_SCAN_PERIOD_MS,
+              TimeUnit.MILLISECONDS.convert(conf.getLong(
+                  DFS_DATANODE_SCAN_PERIOD_HOURS_KEY,
+                  DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT), TimeUnit.HOURS));
+
+      if (tempScanPeriodMs == 0) {
+        tempScanPeriodMs = TimeUnit.MILLISECONDS.convert(
+            DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT, TimeUnit.HOURS);
+      }
+
+      return tempScanPeriodMs;
+    }
+
     @SuppressWarnings("unchecked")
     Conf(Configuration conf) {
       this.targetBytesPerSec = Math.max(0L, conf.getLong(
@@ -123,11 +151,7 @@ public class BlockScanner {
       this.maxStalenessMs = Math.max(0L, getUnitTestLong(conf,
           INTERNAL_DFS_BLOCK_SCANNER_MAX_STALENESS_MS,
           INTERNAL_DFS_BLOCK_SCANNER_MAX_STALENESS_MS_DEFAULT));
-      this.scanPeriodMs = Math.max(0L,
-          getUnitTestLong(conf, INTERNAL_DFS_DATANODE_SCAN_PERIOD_MS,
-              TimeUnit.MILLISECONDS.convert(conf.getLong(
-                  DFS_DATANODE_SCAN_PERIOD_HOURS_KEY,
-                  DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT), TimeUnit.HOURS)));
+      this.scanPeriodMs = getConfiguredScanPeriodMs(conf);
       this.cursorSaveMs = Math.max(0L, getUnitTestLong(conf,
           INTERNAL_DFS_BLOCK_SCANNER_CURSOR_SAVE_INTERVAL_MS,
           INTERNAL_DFS_BLOCK_SCANNER_CURSOR_SAVE_INTERVAL_MS_DEFAULT));
@@ -159,7 +183,7 @@ public class BlockScanner {
    * no threads will start.
    */
   public boolean isEnabled() {
-    return (conf.scanPeriodMs) > 0 && (conf.targetBytesPerSec > 0);
+    return (conf.scanPeriodMs > 0) && (conf.targetBytesPerSec > 0);
   }
 
  /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6793dd8/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 76161a5..8cb7d5f 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
@@ -1071,11 +1071,14 @@
 
 <property>
   <name>dfs.datanode.scan.period.hours</name>
-  <value>0</value>
+  <value>504</value>
   <description>
-        If this is 0 or negative, the DataNode's block scanner will be
-        disabled.  If this is positive, the DataNode will not scan any
+        If this is positive, the DataNode will not scan any
         individual block more than once in the specified scan period.
+        If this is negative, the block scanner is disabled.
+        If this is set to zero, then the default value of 504 hours
+        or 3 weeks is used. Prior versions of HDFS incorrectly documented
+        that setting this key to zero will disable the block scanner.
   </description>
 </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6793dd8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
index 59c7ade..d03fa2c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
@@ -274,6 +274,7 @@ public class TestFsDatasetImpl {
   public void testChangeVolumeWithRunningCheckDirs() throws IOException {
     RoundRobinVolumeChoosingPolicy<FsVolumeImpl> blockChooser =
         new RoundRobinVolumeChoosingPolicy<>();
+    conf.setLong(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1);
     final BlockScanner blockScanner = new BlockScanner(datanode, conf);
     final FsVolumeList volumeList = new FsVolumeList(
         Collections.<VolumeFailureInfo>emptyList(), blockScanner, blockChooser);


[48/50] hadoop git commit: HADOOP-12116. Fix unrecommended syntax usages in hadoop/hdfs/yarn script for cygwin in branch-2. Contributed by Li Lu.

Posted by vv...@apache.org.
HADOOP-12116. Fix unrecommended syntax usages in hadoop/hdfs/yarn script for cygwin in branch-2. Contributed by Li Lu.


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

Branch: refs/heads/YARN-2139
Commit: b8e792cba257fdb0ca266ecb2f60f3f10c3a0c3b
Parents: 3a72bfd
Author: cnauroth <cn...@apache.org>
Authored: Tue Jun 30 16:54:30 2015 -0700
Committer: cnauroth <cn...@apache.org>
Committed: Tue Jun 30 16:54:30 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8e792cb/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 4e5f6bb..74d428f 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -913,6 +913,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12159. Move DistCpUtils#compareFs() to org.apache.hadoop.fs.FileUtil
     and fix for HA namespaces (rchiang via rkanter)
 
+    HADOOP-12116. Fix unrecommended syntax usages in hadoop/hdfs/yarn script for
+    cygwin in branch-2. (Li Lu via cnauroth)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[27/50] hadoop git commit: YARN-3770. SerializedException should also handle java.lang.Error on de-serialization. Contributed by Lavkesh Lahngir

Posted by vv...@apache.org.
YARN-3770. SerializedException should also handle java.lang.Error on de-serialization. Contributed by Lavkesh Lahngir


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

Branch: refs/heads/YARN-2139
Commit: 4672315e2d6abe1cee0210cf7d3e8ab114ba933c
Parents: 460e98f
Author: Jian He <ji...@apache.org>
Authored: Mon Jun 29 14:31:32 2015 -0700
Committer: Jian He <ji...@apache.org>
Committed: Mon Jun 29 14:31:49 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                         |  3 +++
 .../api/records/impl/pb/SerializedExceptionPBImpl.java  |  2 +-
 .../records/impl/pb/TestSerializedExceptionPBImpl.java  | 12 ++++++++++--
 3 files changed, 14 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4672315e/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 2fdcc9d..8461f69 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -571,6 +571,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3695. ServerProxy (NMProxy, etc.) shouldn't retry forever for non
     network exception. (Raju Bairishetti via jianhe)
 
+    YARN-3770. SerializedException should also handle java.lang.Error on
+    de-serialization. (Lavkesh Lahngir via jianhe)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4672315e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SerializedExceptionPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SerializedExceptionPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SerializedExceptionPBImpl.java
index fd9e170..2ec232e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SerializedExceptionPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SerializedExceptionPBImpl.java
@@ -101,7 +101,7 @@ public class SerializedExceptionPBImpl extends SerializedException {
     } else if (RuntimeException.class.isAssignableFrom(realClass)) {
       classType = RuntimeException.class;
     } else {
-      classType = Exception.class;
+      classType = Throwable.class;
     }
     return instantiateException(realClass.asSubclass(classType), getMessage(),
       cause == null ? null : cause.deSerialize());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4672315e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/impl/pb/TestSerializedExceptionPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/impl/pb/TestSerializedExceptionPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/impl/pb/TestSerializedExceptionPBImpl.java
index ac7e40e..ecfa63e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/impl/pb/TestSerializedExceptionPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/impl/pb/TestSerializedExceptionPBImpl.java
@@ -20,10 +20,9 @@ package org.apache.hadoop.yarn.api.records.impl.pb;
 
 import java.nio.channels.ClosedChannelException;
 
-import org.junit.Assert;
-import org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProto;
+import org.junit.Assert;
 import org.junit.Test;
 
 public class TestSerializedExceptionPBImpl {
@@ -79,4 +78,13 @@ public class TestSerializedExceptionPBImpl {
     SerializedExceptionPBImpl pb3 = new SerializedExceptionPBImpl();
     Assert.assertEquals(defaultProto.getTrace(), pb3.getRemoteTrace());
   }
+
+  @Test
+  public void testThrowableDeserialization() {
+    // java.lang.Error should also be serializable
+    Error ex = new Error();
+    SerializedExceptionPBImpl pb = new SerializedExceptionPBImpl();
+    pb.init(ex);
+    Assert.assertEquals(ex.getClass(), pb.deSerialize().getClass());
+  }
 }


[28/50] hadoop git commit: HADOOP-12107. long running apps may have a huge number of StatisticsData instances under FileSystem (Sangjin Lee via Ming Ma)

Posted by vv...@apache.org.
HADOOP-12107. long running apps may have a huge number of StatisticsData instances under FileSystem (Sangjin Lee via Ming Ma)


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

Branch: refs/heads/YARN-2139
Commit: 8e1bdc17d9134e01115ae7c929503d8ac0325207
Parents: 460e98f
Author: Ming Ma <mi...@apache.org>
Authored: Mon Jun 29 14:37:38 2015 -0700
Committer: Ming Ma <mi...@apache.org>
Committed: Mon Jun 29 14:37:38 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../java/org/apache/hadoop/fs/FileSystem.java   | 140 +++++++++++++------
 .../apache/hadoop/fs/FCStatisticsBaseTest.java  |  56 +++++++-
 3 files changed, 155 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8e1bdc17/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index a9b44e3..50192ae 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -490,6 +490,9 @@ Trunk (Unreleased)
     HADOOP-11347. RawLocalFileSystem#mkdir and create should honor umask (Varun
     Saxena via Colin P. McCabe)
 
+    HADOOP-12107. long running apps may have a huge number of StatisticsData
+    instances under FileSystem (Sangjin Lee via Ming Ma)
+
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8e1bdc17/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
index 3f9e3bd..1d7bc87 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
@@ -20,7 +20,8 @@ package org.apache.hadoop.fs;
 import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.lang.ref.WeakReference;
+import java.lang.ref.PhantomReference;
+import java.lang.ref.ReferenceQueue;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.security.PrivilegedExceptionAction;
@@ -32,7 +33,6 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.IdentityHashMap;
 import java.util.Iterator;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
@@ -2920,16 +2920,6 @@ public abstract class FileSystem extends Configured implements Closeable {
       volatile int readOps;
       volatile int largeReadOps;
       volatile int writeOps;
-      /**
-       * Stores a weak reference to the thread owning this StatisticsData.
-       * This allows us to remove StatisticsData objects that pertain to
-       * threads that no longer exist.
-       */
-      final WeakReference<Thread> owner;
-
-      StatisticsData(WeakReference<Thread> owner) {
-        this.owner = owner;
-      }
 
       /**
        * Add another StatisticsData object to this one.
@@ -3000,17 +2990,37 @@ public abstract class FileSystem extends Configured implements Closeable {
      * Thread-local data.
      */
     private final ThreadLocal<StatisticsData> threadData;
-    
+
     /**
-     * List of all thread-local data areas.  Protected by the Statistics lock.
+     * Set of all thread-local data areas.  Protected by the Statistics lock.
+     * The references to the statistics data are kept using phantom references
+     * to the associated threads. Proper clean-up is performed by the cleaner
+     * thread when the threads are garbage collected.
      */
-    private LinkedList<StatisticsData> allData;
+    private final Set<StatisticsDataReference> allData;
+
+    /**
+     * Global reference queue and a cleaner thread that manage statistics data
+     * references from all filesystem instances.
+     */
+    private static final ReferenceQueue<Thread> STATS_DATA_REF_QUEUE;
+    private static final Thread STATS_DATA_CLEANER;
+
+    static {
+      STATS_DATA_REF_QUEUE = new ReferenceQueue<Thread>();
+      // start a single daemon cleaner thread
+      STATS_DATA_CLEANER = new Thread(new StatisticsDataReferenceCleaner());
+      STATS_DATA_CLEANER.
+          setName(StatisticsDataReferenceCleaner.class.getName());
+      STATS_DATA_CLEANER.setDaemon(true);
+      STATS_DATA_CLEANER.start();
+    }
 
     public Statistics(String scheme) {
       this.scheme = scheme;
-      this.rootData = new StatisticsData(null);
+      this.rootData = new StatisticsData();
       this.threadData = new ThreadLocal<StatisticsData>();
-      this.allData = null;
+      this.allData = new HashSet<StatisticsDataReference>();
     }
 
     /**
@@ -3020,7 +3030,7 @@ public abstract class FileSystem extends Configured implements Closeable {
      */
     public Statistics(Statistics other) {
       this.scheme = other.scheme;
-      this.rootData = new StatisticsData(null);
+      this.rootData = new StatisticsData();
       other.visitAll(new StatisticsAggregator<Void>() {
         @Override
         public void accept(StatisticsData data) {
@@ -3032,6 +3042,63 @@ public abstract class FileSystem extends Configured implements Closeable {
         }
       });
       this.threadData = new ThreadLocal<StatisticsData>();
+      this.allData = new HashSet<StatisticsDataReference>();
+    }
+
+    /**
+     * A phantom reference to a thread that also includes the data associated
+     * with that thread. On the thread being garbage collected, it is enqueued
+     * to the reference queue for clean-up.
+     */
+    private class StatisticsDataReference extends PhantomReference<Thread> {
+      private final StatisticsData data;
+
+      public StatisticsDataReference(StatisticsData data, Thread thread) {
+        super(thread, STATS_DATA_REF_QUEUE);
+        this.data = data;
+      }
+
+      public StatisticsData getData() {
+        return data;
+      }
+
+      /**
+       * Performs clean-up action when the associated thread is garbage
+       * collected.
+       */
+      public void cleanUp() {
+        // use the statistics lock for safety
+        synchronized (Statistics.this) {
+          /*
+           * If the thread that created this thread-local data no longer exists,
+           * remove the StatisticsData from our list and fold the values into
+           * rootData.
+           */
+          rootData.add(data);
+          allData.remove(this);
+        }
+      }
+    }
+
+    /**
+     * Background action to act on references being removed.
+     */
+    private static class StatisticsDataReferenceCleaner implements Runnable {
+      @Override
+      public void run() {
+        while (true) {
+          try {
+            StatisticsDataReference ref =
+                (StatisticsDataReference)STATS_DATA_REF_QUEUE.remove();
+            ref.cleanUp();
+          } catch (Throwable th) {
+            // the cleaner thread should continue to run even if there are
+            // exceptions, including InterruptedException
+            LOG.warn("exception in the cleaner thread but it will continue to "
+                + "run", th);
+          }
+        }
+      }
     }
 
     /**
@@ -3040,14 +3107,12 @@ public abstract class FileSystem extends Configured implements Closeable {
     public StatisticsData getThreadStatistics() {
       StatisticsData data = threadData.get();
       if (data == null) {
-        data = new StatisticsData(
-            new WeakReference<Thread>(Thread.currentThread()));
+        data = new StatisticsData();
         threadData.set(data);
+        StatisticsDataReference ref =
+            new StatisticsDataReference(data, Thread.currentThread());
         synchronized(this) {
-          if (allData == null) {
-            allData = new LinkedList<StatisticsData>();
-          }
-          allData.add(data);
+          allData.add(ref);
         }
       }
       return data;
@@ -3105,21 +3170,9 @@ public abstract class FileSystem extends Configured implements Closeable {
      */
     private synchronized <T> T visitAll(StatisticsAggregator<T> visitor) {
       visitor.accept(rootData);
-      if (allData != null) {
-        for (Iterator<StatisticsData> iter = allData.iterator();
-            iter.hasNext(); ) {
-          StatisticsData data = iter.next();
-          visitor.accept(data);
-          if (data.owner.get() == null) {
-            /*
-             * If the thread that created this thread-local data no
-             * longer exists, remove the StatisticsData from our list
-             * and fold the values into rootData.
-             */
-            rootData.add(data);
-            iter.remove();
-          }
-        }
+      for (StatisticsDataReference ref: allData) {
+        StatisticsData data = ref.getData();
+        visitor.accept(data);
       }
       return visitor.aggregate();
     }
@@ -3226,7 +3279,7 @@ public abstract class FileSystem extends Configured implements Closeable {
     @Override
     public String toString() {
       return visitAll(new StatisticsAggregator<String>() {
-        private StatisticsData total = new StatisticsData(null);
+        private StatisticsData total = new StatisticsData();
 
         @Override
         public void accept(StatisticsData data) {
@@ -3259,7 +3312,7 @@ public abstract class FileSystem extends Configured implements Closeable {
      */
     public void reset() {
       visitAll(new StatisticsAggregator<Void>() {
-        private StatisticsData total = new StatisticsData(null);
+        private StatisticsData total = new StatisticsData();
 
         @Override
         public void accept(StatisticsData data) {
@@ -3281,6 +3334,11 @@ public abstract class FileSystem extends Configured implements Closeable {
     public String getScheme() {
       return scheme;
     }
+
+    @VisibleForTesting
+    synchronized int getAllThreadLocalDataSize() {
+      return allData.size();
+    }
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8e1bdc17/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FCStatisticsBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FCStatisticsBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FCStatisticsBaseTest.java
index 90337a6..3e33362 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FCStatisticsBaseTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FCStatisticsBaseTest.java
@@ -18,26 +18,34 @@
 
 package org.apache.hadoop.fs;
 
+import static org.apache.hadoop.fs.FileContextTestHelper.createFile;
+
 import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.fs.FileSystem.Statistics;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.google.common.base.Supplier;
 import com.google.common.util.concurrent.Uninterruptibles;
 
-import static org.apache.hadoop.fs.FileContextTestHelper.*;
-
 /**
  * <p>
  *   Base class to test {@link FileContext} Statistics.
  * </p>
  */
 public abstract class FCStatisticsBaseTest {
-  
   static protected int blockSize = 512;
   static protected int numBlocks = 1;
   
@@ -102,6 +110,48 @@ public abstract class FCStatisticsBaseTest {
     fc.delete(filePath, true);
   }
 
+  @Test(timeout=60000)
+  public void testStatisticsThreadLocalDataCleanUp() throws Exception {
+    final Statistics stats = new Statistics("test");
+    // create a small thread pool to test the statistics
+    final int size = 2;
+    ExecutorService es = Executors.newFixedThreadPool(size);
+    List<Callable<Boolean>> tasks = new ArrayList<Callable<Boolean>>(size);
+    for (int i = 0; i < size; i++) {
+      tasks.add(new Callable<Boolean>() {
+        public Boolean call() {
+          // this populates the data set in statistics
+          stats.incrementReadOps(1);
+          return true;
+        }
+      });
+    }
+    // run the threads
+    es.invokeAll(tasks);
+    // assert that the data size is exactly the number of threads
+    final AtomicInteger allDataSize = new AtomicInteger(0);
+    allDataSize.set(stats.getAllThreadLocalDataSize());
+    Assert.assertEquals(size, allDataSize.get());
+    Assert.assertEquals(size, stats.getReadOps());
+    // force the GC to collect the threads by shutting down the thread pool
+    es.shutdownNow();
+    es.awaitTermination(1, TimeUnit.MINUTES);
+    es = null;
+    System.gc();
+
+    // wait for up to 10 seconds
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+          @Override
+          public Boolean get() {
+            int size = stats.getAllThreadLocalDataSize();
+            allDataSize.set(size);
+            return size == 0;
+          }
+        }, 1000, 10*1000);
+    Assert.assertEquals(0, allDataSize.get());
+    Assert.assertEquals(size, stats.getReadOps());
+  }
+
   /**
    * Bytes read may be different for different file systems. This method should
    * throw assertion error if bytes read are incorrect.


[05/50] hadoop git commit: HADOOP-11203. Allow ditscp to accept bandwitdh in fraction MegaBytes. Contributed by Raju Bairishetti

Posted by vv...@apache.org.
HADOOP-11203. Allow ditscp to accept bandwitdh in fraction MegaBytes. Contributed by Raju Bairishetti


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

Branch: refs/heads/YARN-2139
Commit: 8ef07f767f0421b006b0fc77e5daf36c7b06abf1
Parents: 1403b84
Author: Amareshwari Sriramadasu <am...@apache.org>
Authored: Fri Jun 26 09:52:06 2015 +0530
Committer: Amareshwari Sriramadasu <am...@apache.org>
Committed: Fri Jun 26 09:52:06 2015 +0530

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt    |  3 +++
 .../org/apache/hadoop/tools/DistCpConstants.java   |  2 +-
 .../apache/hadoop/tools/DistCpOptionSwitch.java    |  5 +++--
 .../org/apache/hadoop/tools/DistCpOptions.java     |  6 +++---
 .../org/apache/hadoop/tools/OptionsParser.java     |  2 +-
 .../org/apache/hadoop/tools/mapred/CopyMapper.java | 17 ++++++++++++++++-
 .../tools/mapred/RetriableFileCopyCommand.java     |  2 +-
 .../hadoop/tools/util/ThrottledInputStream.java    |  6 +++---
 .../org/apache/hadoop/tools/TestOptionsParser.java | 16 +++++++++-------
 9 files changed, 40 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ef07f76/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index b2975dc..5901794 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -53,6 +53,9 @@ Trunk (Unreleased)
 
   IMPROVEMENTS
 
+    HADOOP-11203. Allow ditscp to accept bandwitdh in fraction MegaBytes
+    (Raju Bairishetti via amareshwari)
+
     HADOOP-8017. Configure hadoop-main pom to get rid of M2E plugin execution
     not covered (Eric Charles via bobby)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ef07f76/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 21dca62..93d6a62 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
@@ -30,7 +30,7 @@ public class DistCpConstants {
   public static final int DEFAULT_MAPS = 20;
 
   /* Default bandwidth if none specified */
-  public static final int DEFAULT_BANDWIDTH_MB = 100;
+  public static final float DEFAULT_BANDWIDTH_MB = 100;
 
   /* Default strategy for copying. Implementation looked up
      from distcp-default.xml

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ef07f76/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 ed4a0b2..f16a5d2 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
@@ -174,10 +174,11 @@ public enum DistCpOptionSwitch {
               "copied to <= n bytes")),
 
   /**
-   * Specify bandwidth per map in MB
+   * Specify bandwidth per map in MB, accepts bandwidth as a fraction
    */
   BANDWIDTH(DistCpConstants.CONF_LABEL_BANDWIDTH_MB,
-      new Option("bandwidth", true, "Specify bandwidth per map in MB")),
+      new Option("bandwidth", true, "Specify bandwidth per map in MB,"
+          + " accepts bandwidth as a fraction.")),
 
   /**
    * Path containing a list of strings, which when found in the path of

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ef07f76/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 302b626..5b4ccf9 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
@@ -47,7 +47,7 @@ public class DistCpOptions {
   public static final int maxNumListstatusThreads = 40;
   private int numListstatusThreads = 0;  // Indicates that flag is not set.
   private int maxMaps = DistCpConstants.DEFAULT_MAPS;
-  private int mapBandwidth = DistCpConstants.DEFAULT_BANDWIDTH_MB;
+  private float mapBandwidth = DistCpConstants.DEFAULT_BANDWIDTH_MB;
 
   private String sslConfigurationFile;
 
@@ -366,7 +366,7 @@ public class DistCpOptions {
    *
    * @return Bandwidth in MB
    */
-  public int getMapBandwidth() {
+  public float getMapBandwidth() {
     return mapBandwidth;
   }
 
@@ -375,7 +375,7 @@ public class DistCpOptions {
    *
    * @param mapBandwidth - per map bandwidth
    */
-  public void setMapBandwidth(int mapBandwidth) {
+  public void setMapBandwidth(float mapBandwidth) {
     assert mapBandwidth > 0 : "Bandwidth " + mapBandwidth + " is invalid (should be > 0)";
     this.mapBandwidth = mapBandwidth;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ef07f76/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 37add1e..b414513 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
@@ -293,7 +293,7 @@ public class OptionsParser {
                                      DistCpOptions option) {
     if (command.hasOption(DistCpOptionSwitch.BANDWIDTH.getSwitch())) {
       try {
-        Integer mapBandwidth = Integer.parseInt(
+        Float mapBandwidth = Float.parseFloat(
             getVal(command, DistCpOptionSwitch.BANDWIDTH.getSwitch()).trim());
         if (mapBandwidth <= 0) {
           throw new IllegalArgumentException("Bandwidth specified is not " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ef07f76/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
index cca36df..f75fe76 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
@@ -62,6 +62,8 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
     BYTESEXPECTED,// Number of bytes expected to be copied.
     BYTESFAILED,  // Number of bytes that failed to be copied.
     BYTESSKIPPED, // Number of bytes that were skipped from copy.
+    SLEEP_TIME_MS, // Time map slept while trying to honor bandwidth cap.
+    BANDWIDTH_IN_BYTES, // Effective transfer rate in B/s.
   }
 
   /**
@@ -85,7 +87,9 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
   private EnumSet<FileAttribute> preserve = EnumSet.noneOf(FileAttribute.class);
 
   private FileSystem targetFS = null;
-  private Path    targetWorkPath = null;
+  private Path targetWorkPath = null;
+  private long startEpoch;
+  private long totalBytesCopied = 0;
 
   /**
    * Implementation of the Mapper::setup() method. This extracts the DistCp-
@@ -118,6 +122,7 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
     if (conf.get(DistCpConstants.CONF_LABEL_SSL_CONF) != null) {
       initializeSSLConf(context);
     }
+    startEpoch = System.currentTimeMillis();
   }
 
   /**
@@ -288,6 +293,7 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
     incrementCounter(context, Counter.BYTESEXPECTED, sourceFileStatus.getLen());
     incrementCounter(context, Counter.BYTESCOPIED, bytesCopied);
     incrementCounter(context, Counter.COPY, 1);
+    totalBytesCopied += bytesCopied;
   }
 
   private void createTargetDirsWithRetry(String description,
@@ -373,4 +379,13 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
       return false;
     }
   }
+
+  @Override
+  protected void cleanup(Context context)
+      throws IOException, InterruptedException {
+    super.cleanup(context);
+    long secs = (System.currentTimeMillis() - startEpoch) / 1000;
+    incrementCounter(context, Counter.BANDWIDTH_IN_BYTES,
+        totalBytesCopied / ((secs == 0 ? 1 : secs)));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ef07f76/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java
index 65d644b..6b5078c 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java
@@ -293,7 +293,7 @@ public class RetriableFileCopyCommand extends RetriableCommand {
       Configuration conf) throws IOException {
     try {
       FileSystem fs = path.getFileSystem(conf);
-      long bandwidthMB = conf.getInt(DistCpConstants.CONF_LABEL_BANDWIDTH_MB,
+      float bandwidthMB = conf.getFloat(DistCpConstants.CONF_LABEL_BANDWIDTH_MB,
               DistCpConstants.DEFAULT_BANDWIDTH_MB);
       FSDataInputStream in = fs.open(path);
       return new ThrottledInputStream(in, bandwidthMB * 1024 * 1024);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ef07f76/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/ThrottledInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/ThrottledInputStream.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/ThrottledInputStream.java
index 9e435d9..2be8ef0 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/ThrottledInputStream.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/ThrottledInputStream.java
@@ -39,7 +39,7 @@ import com.google.common.base.Preconditions;
 public class ThrottledInputStream extends InputStream {
 
   private final InputStream rawStream;
-  private final long maxBytesPerSec;
+  private final float maxBytesPerSec;
   private final long startTime = System.currentTimeMillis();
 
   private long bytesRead = 0;
@@ -51,8 +51,8 @@ public class ThrottledInputStream extends InputStream {
     this(rawStream, Long.MAX_VALUE);
   }
 
-  public ThrottledInputStream(InputStream rawStream, long maxBytesPerSec) {
-    assert maxBytesPerSec > 0 : "Bandwidth " + maxBytesPerSec + " is invalid"; 
+  public ThrottledInputStream(InputStream rawStream, float maxBytesPerSec) {
+    assert maxBytesPerSec > 0 : "Bandwidth " + maxBytesPerSec + " is invalid";
     this.rawStream = rawStream;
     this.maxBytesPerSec = maxBytesPerSec;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ef07f76/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 b9d9ada..616872b 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
@@ -32,6 +32,8 @@ import java.util.NoSuchElementException;
 
 public class TestOptionsParser {
 
+  private static final float DELTA = 0.001f;
+
   @Test
   public void testParseIgnoreFailure() {
     DistCpOptions options = OptionsParser.parse(new String[] {
@@ -104,14 +106,14 @@ public class TestOptionsParser {
     DistCpOptions options = OptionsParser.parse(new String[] {
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertEquals(options.getMapBandwidth(), DistCpConstants.DEFAULT_BANDWIDTH_MB);
+    Assert.assertEquals(options.getMapBandwidth(), DistCpConstants.DEFAULT_BANDWIDTH_MB, DELTA);
 
     options = OptionsParser.parse(new String[] {
         "-bandwidth",
-        "11",
+        "11.2",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertEquals(options.getMapBandwidth(), 11);
+    Assert.assertEquals(options.getMapBandwidth(), 11.2, DELTA);
   }
 
   @Test(expected=IllegalArgumentException.class)
@@ -585,8 +587,8 @@ public class TestOptionsParser {
     options.appendToConf(conf);
     Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.IGNORE_FAILURES.getConfigLabel(), false));
     Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.ATOMIC_COMMIT.getConfigLabel(), false));
-    Assert.assertEquals(conf.getInt(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1),
-        DistCpConstants.DEFAULT_BANDWIDTH_MB);
+    Assert.assertEquals(conf.getFloat(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1),
+        DistCpConstants.DEFAULT_BANDWIDTH_MB, DELTA);
 
     conf = new Configuration();
     Assert.assertFalse(conf.getBoolean(DistCpOptionSwitch.SYNC_FOLDERS.getConfigLabel(), false));
@@ -597,14 +599,14 @@ public class TestOptionsParser {
         "-delete",
         "-pu",
         "-bandwidth",
-        "11",
+        "11.2",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
     options.appendToConf(conf);
     Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.SYNC_FOLDERS.getConfigLabel(), false));
     Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.DELETE_MISSING.getConfigLabel(), false));
     Assert.assertEquals(conf.get(DistCpOptionSwitch.PRESERVE_STATUS.getConfigLabel()), "U");
-    Assert.assertEquals(conf.getInt(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1), 11);
+    Assert.assertEquals(conf.getFloat(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1), 11.2, DELTA);
   }
 
   @Test


[18/50] hadoop git commit: HDFS-8586. Dead Datanode is allocated for write when client is from deadnode (Contributed by Brahma Reddy Battula)

Posted by vv...@apache.org.
HDFS-8586. Dead Datanode is allocated for write when client is from deadnode (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/88ceb382
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/88ceb382
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/88ceb382

Branch: refs/heads/YARN-2139
Commit: 88ceb382ef45bd09cf004cf44aedbabaf3976759
Parents: a95d39f
Author: Vinayakumar B <vi...@apache.org>
Authored: Mon Jun 29 15:25:03 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Mon Jun 29 15:25:03 2015 +0530

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../BlockPlacementPolicyDefault.java            |  3 +-
 .../hdfs/server/namenode/TestDeadDatanode.java  | 42 ++++++++++++++++++++
 3 files changed, 47 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/88ceb382/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index b89e10c..3800184 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -968,6 +968,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8546. Prune cached replicas from DatanodeDescriptor state on replica
     invalidation. (wang)
 
+    HDFS-8586. Dead Datanode is allocated for write when client is from deadnode
+    (Brahma Reddy Battula via vinayakumarb)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/88ceb382/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 21ad01d..9023e0a 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
@@ -454,7 +454,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
           maxNodesPerRack, results, avoidStaleNodes, storageTypes);
     }
-    if (preferLocalNode && localMachine instanceof DatanodeDescriptor) {
+    if (preferLocalNode && localMachine instanceof DatanodeDescriptor
+        && clusterMap.contains(localMachine)) {
       DatanodeDescriptor localDatanode = (DatanodeDescriptor) localMachine;
       // otherwise try local machine first
       if (excludedNodes.add(localMachine)) { // was not in the excluded list

http://git-wip-us.apache.org/repos/asf/hadoop/blob/88ceb382/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
index ff70c3f..c5262d4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
@@ -18,9 +18,11 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.util.HashSet;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -31,6 +33,9 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
@@ -43,6 +48,7 @@ import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
 import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.net.Node;
 import org.junit.After;
 import org.junit.Test;
 
@@ -126,4 +132,40 @@ public class TestDeadDatanode {
     assertEquals(cmd[0].getAction(), RegisterCommand.REGISTER
         .getAction());
   }
+
+  @Test
+  public void testDeadNodeAsBlockTarget() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 500);
+    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
+    cluster.waitActive();
+
+    String poolId = cluster.getNamesystem().getBlockPoolId();
+    // wait for datanode to be marked live
+    DataNode dn = cluster.getDataNodes().get(0);
+    DatanodeRegistration reg = DataNodeTestUtils.getDNRegistrationForBP(cluster
+        .getDataNodes().get(0), poolId);
+    // Get the updated datanode descriptor
+    BlockManager bm = cluster.getNamesystem().getBlockManager();
+    DatanodeManager dm = bm.getDatanodeManager();
+    Node clientNode = dm.getDatanode(reg);
+
+    DFSTestUtil.waitForDatanodeState(cluster, reg.getDatanodeUuid(), true,
+        20000);
+
+    // Shutdown and wait for datanode to be marked dead
+    dn.shutdown();
+    DFSTestUtil.waitForDatanodeState(cluster, reg.getDatanodeUuid(), false,
+        20000);
+    // Get the updated datanode descriptor available in DNM
+    // choose the targets, but local node should not get selected as this is not
+    // part of the cluster anymore
+    DatanodeStorageInfo[] results = bm.chooseTarget4NewBlock("/hello", 3,
+        clientNode, new HashSet<Node>(), 256 * 1024 * 1024L, null, (byte) 7);
+    for (DatanodeStorageInfo datanodeStorageInfo : results) {
+      assertFalse("Dead node should not be choosen", datanodeStorageInfo
+          .getDatanodeDescriptor().equals(clientNode));
+    }
+  }
 }


[42/50] hadoop git commit: MAPREDUCE-6407. Migrate MAPREDUCE native build to new CMake framework (Alan Burlison via Colin P. McCabe)

Posted by vv...@apache.org.
MAPREDUCE-6407. Migrate MAPREDUCE native build to new CMake framework (Alan Burlison via Colin P. McCabe)


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

Branch: refs/heads/YARN-2139
Commit: 9c63825f10363a68e7eb34005fa4f4b7ddb27a03
Parents: 29df73d
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Tue Jun 30 16:06:46 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Tue Jun 30 16:17:06 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |   3 +
 .../src/CMakeLists.txt                          | 337 ++++++++-----------
 .../src/JNIFlags.cmake                          | 118 -------
 3 files changed, 137 insertions(+), 321 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c63825f/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 5a4d826..64b04cc 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -115,6 +115,9 @@ Trunk (Unreleased)
     MAPREDUCE-2632. Avoid calling the partitioner when the numReduceTasks is 1.
     (Ravi Teja Ch N V and Sunil G via kasha)
 
+    MAPREDUCE-6407. Migrate MAPREDUCE nativetask build to new CMake framework
+    (Alan Burlison via Colin P. McCabe)
+
   BUG FIXES
 
     MAPREDUCE-6191. Improve clearing stale state of Java serialization

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c63825f/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/CMakeLists.txt b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/CMakeLists.txt
index 80036e8..f878a94 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/CMakeLists.txt
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/CMakeLists.txt
@@ -18,236 +18,170 @@
 
 cmake_minimum_required(VERSION 2.6 FATAL_ERROR)
 
-# Default to release builds
-set(CMAKE_BUILD_TYPE, Release)
-
-include(JNIFlags.cmake NO_POLICY_SCOPE)
-
-# Compile a library with both shared and static variants
-function(add_dual_library LIBNAME)
-    add_library(${LIBNAME} SHARED ${ARGN})
-    add_library(${LIBNAME}_static STATIC ${ARGN})
-    set_target_properties(${LIBNAME}_static PROPERTIES OUTPUT_NAME ${LIBNAME})
-endfunction(add_dual_library)
-
-# Link both a static and a dynamic target against some libraries
-function(target_link_dual_libraries LIBNAME)
-    target_link_libraries(${LIBNAME} ${ARGN})
-    target_link_libraries(${LIBNAME}_static ${ARGN})
-endfunction(target_link_dual_libraries)
-
-function(output_directory TGT DIR)
-    SET_TARGET_PROPERTIES(${TGT} PROPERTIES
-        RUNTIME_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-    SET_TARGET_PROPERTIES(${TGT} PROPERTIES
-        ARCHIVE_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-    SET_TARGET_PROPERTIES(${TGT} PROPERTIES
-        LIBRARY_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-endfunction(output_directory TGT DIR)
-
-function(dual_output_directory TGT DIR)
-    output_directory(${TGT} "${DIR}")
-    output_directory(${TGT}_static "${DIR}")
-endfunction(dual_output_directory TGT DIR)
-
-#
-# This macro alters the behavior of find_package and find_library.
-# It does this by setting the CMAKE_FIND_LIBRARY_SUFFIXES global variable.
-# You should save that variable before calling this function and restore it
-# after you have accomplished your goal.
-#
-# The behavior is altered in two ways:
-# 1. We always find shared libraries, never static;
-# 2. We find shared libraries with the given version number.
-#
-# On Windows this function is a no-op.  Windows does not encode
-# version number information information into library path names.
-#
-macro(set_find_shared_library_version LVERS)
-    IF(${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
-        # Mac OS uses .dylib
-        SET(CMAKE_FIND_LIBRARY_SUFFIXES ".${LVERS}.dylib")
-    ELSEIF(${CMAKE_SYSTEM_NAME} MATCHES "FreeBSD")
-        # FreeBSD has always .so installed.
-        SET(CMAKE_FIND_LIBRARY_SUFFIXES ".so")
-    ELSEIF(${CMAKE_SYSTEM_NAME} MATCHES "Windows")
-        # Windows doesn't support finding shared libraries by version.
-    ELSE()
-        # Most UNIX variants use .so
-        SET(CMAKE_FIND_LIBRARY_SUFFIXES ".so.${LVERS}")
-    ENDIF()
-endmacro(set_find_shared_library_version LVERS)
-
-if (NOT GENERATED_JAVAH)
-    #Must identify where the generated headers have been placed
-    MESSAGE(FATAL_ERROR "You must set the cmake variable GENERATED_JAVAH")
-endif (NOT GENERATED_JAVAH)
-find_package(JNI REQUIRED)
-
-SET(STORED_CMAKE_FIND_LIBRARY_SUFFIXES CMAKE_FIND_LIBRARY_SUFFIXES)
-set_find_shared_library_version("1")
-SET(CMAKE_FIND_LIBRARY_SUFFIXES STORED_CMAKE_FIND_LIBRARY_SUFFIXES)
-
-# primitive configs
-set(PRFLAGS "-DSIMPLE_MEMCPY")
-set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${PRFLAGS} -fno-strict-aliasing -Wall -Wno-sign-compare")
-set(CMAKE_LD_FLAGS "${CMAKE_LD_FLAGS} -no-undefined -version-info 0:1:0
-    -L${_JAVA_HOME}/jre/lib/amd64/server -ljvm")
-set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${CMAKE_C_FLAGS} -g -O2 -DNDEBUG -fPIC")
-set(D main/native/)
-
-SET(STORED_CMAKE_FIND_LIBRARY_SUFFIXES CMAKE_FIND_LIBRARY_SUFFIXES)
-set_find_shared_library_version("1")
-SET(CMAKE_FIND_LIBRARY_SUFFIXES STORED_CMAKE_FIND_LIBRARY_SUFFIXES)
-
-INCLUDE(CheckFunctionExists)
-INCLUDE(CheckCSourceCompiles)
-#INCLUDE(CheckLibraryExists)
-INCLUDE(CheckIncludeFiles)
-#CHECK_FUNCTION_EXISTS(sync_file_range HAVE_SYNC_FILE_RANGE)
-#CHECK_FUNCTION_EXISTS(posix_fadvise HAVE_POSIX_FADVISE)
-#CHECK_LIBRARY_EXISTS(dl dlopen "" NEED_LINK_DL)
-CHECK_INCLUDE_FILES(fcntl.h HAVE_FCNTL_H)
-CHECK_INCLUDE_FILES(malloc.h HAVE_MALLOC_H)
-CHECK_INCLUDE_FILES(mach/mach.h HAVE_MACH_MACH_H)
-CHECK_INCLUDE_FILES(memory.h HAVE_MEMORY_H)
-CHECK_INCLUDE_FILES(stddef.h HAVE_STDDEF_H)
-CHECK_INCLUDE_FILES(stdint.h HAVE_STDINT_H)
-CHECK_INCLUDE_FILES(stdlib.h HAVE_STDLIB_H)
-CHECK_INCLUDE_FILES(string.h HAVE_STRING_H)
-CHECK_INCLUDE_FILES(unistd.h HAVE_UNITSTD_H)
-CHECK_FUNCTION_EXISTS(clock_gettime HAVE_CLOCK_GETTIME)
-CHECK_FUNCTION_EXISTS(localtime_r HAVE_LOCALTIME_R)
-CHECK_FUNCTION_EXISTS(memset HAVE_MEMSET)
-CHECK_FUNCTION_EXISTS(strchr HAVE_STRCHR)
-CHECK_FUNCTION_EXISTS(strtoul HAVE_STRTOUL)
-
-SET(STORED_CMAKE_FIND_LIBRARY_SUFFIXES CMAKE_FIND_LIBRARY_SUFFIXES)
-set_find_shared_library_version("1")
+list(APPEND CMAKE_MODULE_PATH ${CMAKE_SOURCE_DIR}/../../../../hadoop-common-project/hadoop-common/)
+include(HadoopCommon)
+
+# Add extra compiler and linker flags.
+# -Wno-sign-compare
+hadoop_add_compiler_flags("-DNDEBUG -DSIMPLE_MEMCPY -fno-strict-aliasing")
+
+# Source location.
+set(SRC main/native)
+
+# The caller must specify where the generated headers have been placed.
+if(NOT GENERATED_JAVAH)
+    message(FATAL_ERROR "You must set the CMake variable GENERATED_JAVAH")
+endif()
+
+# Configure JNI.
+include(HadoopJNI)
+
+# Probe for headers and functions.
+include(CheckFunctionExists)
+include(CheckIncludeFiles)
+
+check_include_files(fcntl.h HAVE_FCNTL_H)
+check_include_files(malloc.h HAVE_MALLOC_H)
+check_include_files(mach/mach.h HAVE_MACH_MACH_H)
+check_include_files(memory.h HAVE_MEMORY_H)
+check_include_files(stddef.h HAVE_STDDEF_H)
+check_include_files(stdint.h HAVE_STDINT_H)
+check_include_files(stdlib.h HAVE_STDLIB_H)
+Check_include_files(string.h HAVE_STRING_H)
+check_include_files(unistd.h HAVE_UNITSTD_H)
+check_function_exists(clock_gettime HAVE_CLOCK_GETTIME)
+check_function_exists(localtime_r HAVE_LOCALTIME_R)
+check_function_exists(memset HAVE_MEMSET)
+check_function_exists(strchr HAVE_STRCHR)
+check_function_exists(strtoul HAVE_STRTOUL)
+
+# Require snappy.
+set(STORED_CMAKE_FIND_LIBRARY_SUFFIXES CMAKE_FIND_LIBRARY_SUFFIXES)
+hadoop_set_find_shared_library_version("1")
 find_library(SNAPPY_LIBRARY
     NAMES snappy
     PATHS ${CUSTOM_SNAPPY_PREFIX} ${CUSTOM_SNAPPY_PREFIX}/lib
           ${CUSTOM_SNAPPY_PREFIX}/lib64 ${CUSTOM_SNAPPY_LIB})
-SET(CMAKE_FIND_LIBRARY_SUFFIXES STORED_CMAKE_FIND_LIBRARY_SUFFIXES)
+set(CMAKE_FIND_LIBRARY_SUFFIXES STORED_CMAKE_FIND_LIBRARY_SUFFIXES)
 find_path(SNAPPY_INCLUDE_DIR
     NAMES snappy.h
     PATHS ${CUSTOM_SNAPPY_PREFIX} ${CUSTOM_SNAPPY_PREFIX}/include
           ${CUSTOM_SNAPPY_INCLUDE})
-if (SNAPPY_LIBRARY AND SNAPPY_INCLUDE_DIR)
+if(SNAPPY_LIBRARY AND SNAPPY_INCLUDE_DIR)
     GET_FILENAME_COMPONENT(HADOOP_SNAPPY_LIBRARY ${SNAPPY_LIBRARY} NAME)
     set(SNAPPY_SOURCE_FILES
-        "${D}/src/codec/SnappyCodec.cc")
-else (SNAPPY_LIBRARY AND SNAPPY_INCLUDE_DIR)
+        "${SRC}/src/codec/SnappyCodec.cc")
+    set(REQUIRE_SNAPPY ${REQUIRE_SNAPPY}) # Stop warning about unused variable.
+    message(STATUS "Found Snappy: ${SNAPPY_LIBRARY}")
+else()
     set(SNAPPY_LIBRARY "")
     set(SNAPPY_INCLUDE_DIR "")
     set(SNAPPY_SOURCE_FILES "")
-    IF(REQUIRE_SNAPPY)
-        MESSAGE(FATAL_ERROR "Required snappy library could not be found.  SNAPPY_LIBRARY=${SNAPPY_LIBRARY}, SNAPPY_INCLUDE_DIR=${SNAPPY_INCLUDE_DIR}, CUSTOM_SNAPPY_INCLUDE_DIR=${CUSTOM_SNAPPY_INCLUDE_DIR}, CUSTOM_SNAPPY_PREFIX=${CUSTOM_SNAPPY_PREFIX}, CUSTOM_SNAPPY_INCLUDE=${CUSTOM_SNAPPY_INCLUDE}")
-    ENDIF(REQUIRE_SNAPPY)
-endif (SNAPPY_LIBRARY AND SNAPPY_INCLUDE_DIR)
+    if(REQUIRE_SNAPPY)
+        message(FATAL_ERROR "Required snappy library could not be found.  SNAPPY_LIBRARY=${SNAPPY_LIBRARY}, SNAPPY_INCLUDE_DIR=${SNAPPY_INCLUDE_DIR}, CUSTOM_SNAPPY_INCLUDE_DIR=${CUSTOM_SNAPPY_INCLUDE_DIR}, CUSTOM_SNAPPY_PREFIX=${CUSTOM_SNAPPY_PREFIX}, CUSTOM_SNAPPY_INCLUDE=${CUSTOM_SNAPPY_INCLUDE}")
+    endif()
+endif()
 
-CONFIGURE_FILE(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h)
+configure_file(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h)
 
 include_directories(
     ${GENERATED_JAVAH}
-    ${D}/src
-    ${D}/src/util
-    ${D}/src/lib
-    ${D}/test
+    ${SRC}/src
+    ${SRC}/src/util
+    ${SRC}/src/lib
+    ${SRC}/test
     ${CMAKE_CURRENT_SOURCE_DIR}
     ${CMAKE_BINARY_DIR}
     ${JNI_INCLUDE_DIRS}
     ${SNAPPY_INCLUDE_DIR}
 )
 # add gtest as system library to suppress gcc warnings
-include_directories(SYSTEM ${D}/gtest/include)
-
+include_directories(SYSTEM ${SRC}/gtest/include)
 
-SET(CMAKE_BUILD_WITH_INSTALL_RPATH TRUE)
+set(CMAKE_BUILD_WITH_INSTALL_RPATH TRUE)
 
-if (${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
+if(CMAKE_SYSTEM_NAME MATCHES "Darwin")
     # macosx does not have -lrt
     set(NT_DEPEND_LIBRARY dl pthread z ${SNAPPY_LIBRARY} ${JAVA_JVM_LIBRARY})
     set(SYSTEM_MAC TRUE)
-else (${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
+else()
     set(NT_DEPEND_LIBRARY dl rt pthread z ${SNAPPY_LIBRARY} ${JAVA_JVM_LIBRARY})
     set(SYSTEM_MAC FALSE)
-endif (${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
+endif()
 
 configure_file(main/native/test.sh test/test.sh)
 
-add_dual_library(nativetask
+hadoop_add_dual_library(nativetask
     ${CMAKE_BINARY_DIR}/lz4.c
-    ${D}/src/codec/BlockCodec.cc
-    ${D}/src/codec/GzipCodec.cc
-    ${D}/src/codec/Lz4Codec.cc
+    ${SRC}/src/codec/BlockCodec.cc
+    ${SRC}/src/codec/GzipCodec.cc
+    ${SRC}/src/codec/Lz4Codec.cc
     ${SNAPPY_SOURCE_FILES}
-    ${D}/src/handler/BatchHandler.cc
-    ${D}/src/handler/MCollectorOutputHandler.cc
-    ${D}/src/handler/AbstractMapHandler.cc
-    ${D}/src/handler/CombineHandler.cc
-    ${D}/src/lib/Buffers.cc
-    ${D}/src/lib/BufferStream.cc
-    ${D}/src/lib/Compressions.cc
-    ${D}/src/lib/PartitionBucket.cc
-    ${D}/src/lib/PartitionBucketIterator.cc
-    ${D}/src/lib/FileSystem.cc
-    ${D}/src/lib/IFile.cc
-    ${D}/src/lib/jniutils.cc
-    ${D}/src/lib/Log.cc
-    ${D}/src/lib/MapOutputCollector.cc
-    ${D}/src/lib/MapOutputSpec.cc
-    ${D}/src/lib/MemoryBlock.cc
-    ${D}/src/lib/Merge.cc
-    ${D}/src/lib/NativeLibrary.cc
-    ${D}/src/lib/Iterator.cc
-    ${D}/src/lib/NativeObjectFactory.cc
-    ${D}/src/lib/NativeRuntimeJniImpl.cc
-    ${D}/src/lib/NativeTask.cc
-    ${D}/src/lib/SpillInfo.cc
-    ${D}/src/lib/Path.cc
-    ${D}/src/lib/Streams.cc
-    ${D}/src/lib/TaskCounters.cc
-    ${D}/src/util/Checksum.cc
-    ${D}/src/util/Random.cc
-    ${D}/src/util/StringUtil.cc
-    ${D}/src/util/SyncUtils.cc
-    ${D}/src/util/Timer.cc
-    ${D}/src/util/WritableUtils.cc
+    ${SRC}/src/handler/BatchHandler.cc
+    ${SRC}/src/handler/MCollectorOutputHandler.cc
+    ${SRC}/src/handler/AbstractMapHandler.cc
+    ${SRC}/src/handler/CombineHandler.cc
+    ${SRC}/src/lib/Buffers.cc
+    ${SRC}/src/lib/BufferStream.cc
+    ${SRC}/src/lib/Compressions.cc
+    ${SRC}/src/lib/PartitionBucket.cc
+    ${SRC}/src/lib/PartitionBucketIterator.cc
+    ${SRC}/src/lib/FileSystem.cc
+    ${SRC}/src/lib/IFile.cc
+    ${SRC}/src/lib/jniutils.cc
+    ${SRC}/src/lib/Log.cc
+    ${SRC}/src/lib/MapOutputCollector.cc
+    ${SRC}/src/lib/MapOutputSpec.cc
+    ${SRC}/src/lib/MemoryBlock.cc
+    ${SRC}/src/lib/Merge.cc
+    ${SRC}/src/lib/NativeLibrary.cc
+    ${SRC}/src/lib/Iterator.cc
+    ${SRC}/src/lib/NativeObjectFactory.cc
+    ${SRC}/src/lib/NativeRuntimeJniImpl.cc
+    ${SRC}/src/lib/NativeTask.cc
+    ${SRC}/src/lib/SpillInfo.cc
+    ${SRC}/src/lib/Path.cc
+    ${SRC}/src/lib/Streams.cc
+    ${SRC}/src/lib/TaskCounters.cc
+    ${SRC}/src/util/Checksum.cc
+    ${SRC}/src/util/Random.cc
+    ${SRC}/src/util/StringUtil.cc
+    ${SRC}/src/util/SyncUtils.cc
+    ${SRC}/src/util/Timer.cc
+    ${SRC}/src/util/WritableUtils.cc
 )
 
 target_link_libraries(nativetask ${NT_DEPEND_LIBRARY})
 
-add_library(gtest ${D}/gtest/gtest-all.cc)
+add_library(gtest ${SRC}/gtest/gtest-all.cc)
 set_target_properties(gtest PROPERTIES COMPILE_FLAGS "-w")
 add_executable(nttest
-    ${D}/test/lib/TestByteArray.cc
-    ${D}/test/lib/TestByteBuffer.cc
-    ${D}/test/lib/TestComparatorForDualPivotQuickSort.cc
-    ${D}/test/lib/TestComparatorForStdSort.cc
-    ${D}/test/lib/TestFixSizeContainer.cc
-    ${D}/test/lib/TestMemoryPool.cc
-    ${D}/test/lib/TestIterator.cc
-    ${D}/test/lib/TestKVBuffer.cc
-    ${D}/test/lib/TestMemBlockIterator.cc
-    ${D}/test/lib/TestMemoryBlock.cc
-    ${D}/test/lib/TestPartitionBucket.cc
-    ${D}/test/lib/TestReadBuffer.cc
-    ${D}/test/lib/TestReadWriteBuffer.cc
-    ${D}/test/util/TestChecksum.cc
-    ${D}/test/util/TestStringUtil.cc
-    ${D}/test/util/TestWritableUtils.cc
-    ${D}/test/TestCommand.cc
-    ${D}/test/TestConfig.cc
-    ${D}/test/TestCounter.cc
-    ${D}/test/TestCompressions.cc
-    ${D}/test/TestFileSystem.cc
-    ${D}/test/TestIFile.cc
-    ${D}/test/TestPrimitives.cc
-    ${D}/test/TestSort.cc
-    ${D}/test/TestMain.cc
-    ${D}/test/test_commons.cc)
-
+    ${SRC}/test/lib/TestByteArray.cc
+    ${SRC}/test/lib/TestByteBuffer.cc
+    ${SRC}/test/lib/TestComparatorForDualPivotQuickSort.cc
+    ${SRC}/test/lib/TestComparatorForStdSort.cc
+    ${SRC}/test/lib/TestFixSizeContainer.cc
+    ${SRC}/test/lib/TestMemoryPool.cc
+    ${SRC}/test/lib/TestIterator.cc
+    ${SRC}/test/lib/TestKVBuffer.cc
+    ${SRC}/test/lib/TestMemBlockIterator.cc
+    ${SRC}/test/lib/TestMemoryBlock.cc
+    ${SRC}/test/lib/TestPartitionBucket.cc
+    ${SRC}/test/lib/TestReadBuffer.cc
+    ${SRC}/test/lib/TestReadWriteBuffer.cc
+    ${SRC}/test/util/TestChecksum.cc
+    ${SRC}/test/util/TestStringUtil.cc
+    ${SRC}/test/util/TestWritableUtils.cc
+    ${SRC}/test/TestCommand.cc
+    ${SRC}/test/TestConfig.cc
+    ${SRC}/test/TestCounter.cc
+    ${SRC}/test/TestCompressions.cc
+    ${SRC}/test/TestFileSystem.cc
+    ${SRC}/test/TestIFile.cc
+    ${SRC}/test/TestPrimitives.cc
+    ${SRC}/test/TestSort.cc
+    ${SRC}/test/TestMain.cc
+    ${SRC}/test/test_commons.cc)
 
 target_link_libraries(nttest
      nativetask_static
@@ -255,17 +189,14 @@ target_link_libraries(nttest
      ${NT_DEPEND_LIBRARY}
 )
 
-IF (${CMAKE_SYSTEM_NAME} MATCHES "Linux")
-    #
-    # By embedding '$ORIGIN' into the RPATH of libnativetask.so,
-    # dlopen will look in the directory containing libnativetask.so.
-    # However, $ORIGIN is not supported by all operating systems.
-    #
-    SET_TARGET_PROPERTIES(nativetask
-        PROPERTIES INSTALL_RPATH "\$ORIGIN/")
-ENDIF()
-
-SET(LIBNATIVETASK_VERSION "1.0.0")
-SET_TARGET_PROPERTIES(nativetask PROPERTIES SOVERSION ${LIBNATIVETASK_VERSION})
-dual_output_directory(nativetask target/usr/local/lib)
-output_directory(nttest test)
+# By embedding '$ORIGIN' into the RPATH of libnativetask.so, dlopen will look in
+# the directory containing libnativetask.so. However, $ORIGIN is not supported by
+# all operating systems.
+if(CMAKE_SYSTEM_NAME MATCHES "Linux|SunOS")
+    set_target_properties(nativetask PROPERTIES INSTALL_RPATH "\$ORIGIN/")
+endif()
+
+set(LIBNATIVETASK_VERSION "1.0.0")
+set_target_properties(nativetask PROPERTIES SOVERSION ${LIBNATIVETASK_VERSION})
+hadoop_dual_output_directory(nativetask target/usr/local/lib)
+hadoop_output_directory(nttest test)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c63825f/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/JNIFlags.cmake
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/JNIFlags.cmake b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/JNIFlags.cmake
deleted file mode 100644
index 0100b06..0000000
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/JNIFlags.cmake
+++ /dev/null
@@ -1,118 +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.
-#
-
-cmake_minimum_required(VERSION 2.6 FATAL_ERROR)
-
-# If JVM_ARCH_DATA_MODEL is 32, compile all binaries as 32-bit.
-# This variable is set by maven.
-if (JVM_ARCH_DATA_MODEL EQUAL 32)
-    # Force 32-bit code generation on amd64/x86_64, ppc64, sparc64
-    if (CMAKE_COMPILER_IS_GNUCC AND CMAKE_SYSTEM_PROCESSOR MATCHES ".*64")
-        set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -m32")
-        set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -m32")
-        set(CMAKE_LD_FLAGS "${CMAKE_LD_FLAGS} -m32")
-    endif ()
-    if (CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "amd64")
-        # Set CMAKE_SYSTEM_PROCESSOR to ensure that find_package(JNI) will use
-        # the 32-bit version of libjvm.so.
-        set(CMAKE_SYSTEM_PROCESSOR "i686")
-    endif ()
-endif (JVM_ARCH_DATA_MODEL EQUAL 32)
-
-# Determine float ABI of JVM on ARM Linux
-if (CMAKE_SYSTEM_PROCESSOR MATCHES "^arm" AND CMAKE_SYSTEM_NAME STREQUAL "Linux")
-    find_program(READELF readelf)
-    if (READELF MATCHES "NOTFOUND")
-        message(WARNING "readelf not found; JVM float ABI detection disabled")
-    else (READELF MATCHES "NOTFOUND")
-        execute_process(
-            COMMAND ${READELF} -A ${JAVA_JVM_LIBRARY}
-            OUTPUT_VARIABLE JVM_ELF_ARCH
-            ERROR_QUIET)
-        if (NOT JVM_ELF_ARCH MATCHES "Tag_ABI_VFP_args: VFP registers")
-            message("Soft-float JVM detected")
-
-            # Test compilation with -mfloat-abi=softfp using an arbitrary libc function
-            # (typically fails with "fatal error: bits/predefs.h: No such file or directory"
-            # if soft-float dev libraries are not installed)
-            include(CMakePushCheckState)
-            cmake_push_check_state()
-            set(CMAKE_REQUIRED_FLAGS "${CMAKE_REQUIRED_FLAGS} -mfloat-abi=softfp")
-            include(CheckSymbolExists)
-            check_symbol_exists(exit stdlib.h SOFTFP_AVAILABLE)
-            if (NOT SOFTFP_AVAILABLE)
-                message(FATAL_ERROR "Soft-float dev libraries required (e.g. 'apt-get install libc6-dev-armel' on Debian/Ubuntu)")
-            endif (NOT SOFTFP_AVAILABLE)
-            cmake_pop_check_state()
-
-            set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -mfloat-abi=softfp")
-        endif ()
-    endif (READELF MATCHES "NOTFOUND")
-endif (CMAKE_SYSTEM_PROCESSOR MATCHES "^arm" AND CMAKE_SYSTEM_NAME STREQUAL "Linux")
-
-IF("${CMAKE_SYSTEM}" MATCHES "Linux")
-    #
-    # Locate JNI_INCLUDE_DIRS and JNI_LIBRARIES.
-    # Since we were invoked from Maven, we know that the JAVA_HOME environment
-    # variable is valid.  So we ignore system paths here and just use JAVA_HOME.
-    #
-    FILE(TO_CMAKE_PATH "$ENV{JAVA_HOME}" _JAVA_HOME)
-    IF(CMAKE_SYSTEM_PROCESSOR MATCHES "^i.86$")
-        SET(_java_libarch "i386")
-    ELSEIF (CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "amd64")
-        SET(_java_libarch "amd64")
-    ELSEIF (CMAKE_SYSTEM_PROCESSOR MATCHES "^arm")
-        SET(_java_libarch "arm")
-    ELSE()
-        SET(_java_libarch ${CMAKE_SYSTEM_PROCESSOR})
-    ENDIF()
-    SET(_JDK_DIRS "${_JAVA_HOME}/jre/lib/${_java_libarch}/*"
-                  "${_JAVA_HOME}/jre/lib/${_java_libarch}"
-                  "${_JAVA_HOME}/jre/lib/*"
-                  "${_JAVA_HOME}/jre/lib"
-                  "${_JAVA_HOME}/lib/*"
-                  "${_JAVA_HOME}/lib"
-                  "${_JAVA_HOME}/include/*"
-                  "${_JAVA_HOME}/include"
-                  "${_JAVA_HOME}"
-    )
-    FIND_PATH(JAVA_INCLUDE_PATH
-        NAMES jni.h 
-        PATHS ${_JDK_DIRS}
-        NO_DEFAULT_PATH)
-    #In IBM java, it's jniport.h instead of jni_md.h
-    FIND_PATH(JAVA_INCLUDE_PATH2 
-        NAMES jni_md.h jniport.h
-        PATHS ${_JDK_DIRS}
-        NO_DEFAULT_PATH)
-    SET(JNI_INCLUDE_DIRS ${JAVA_INCLUDE_PATH} ${JAVA_INCLUDE_PATH2})
-    FIND_LIBRARY(JAVA_JVM_LIBRARY
-        NAMES rt jvm 
-        PATHS ${_JDK_DIRS}
-        NO_DEFAULT_PATH)
-    SET(JNI_LIBRARIES ${JAVA_JVM_LIBRARY})
-    MESSAGE("JAVA_HOME=${JAVA_HOME}, JAVA_JVM_LIBRARY=${JAVA_JVM_LIBRARY}")
-    MESSAGE("JAVA_INCLUDE_PATH=${JAVA_INCLUDE_PATH}, JAVA_INCLUDE_PATH2=${JAVA_INCLUDE_PATH2}")
-    IF(JAVA_JVM_LIBRARY AND JAVA_INCLUDE_PATH AND JAVA_INCLUDE_PATH2)
-        MESSAGE("Located all JNI components successfully.")
-    ELSE()
-        MESSAGE(FATAL_ERROR "Failed to find a viable JVM installation under JAVA_HOME.")
-    ENDIF()
-ELSE()
-    find_package(JNI REQUIRED)
-ENDIF()


[49/50] hadoop git commit: HADOOP-12164. Fix TestMove and TestFsShellReturnCode failed to get command name using reflection. (Lei Xu)

Posted by vv...@apache.org.
 HADOOP-12164. Fix TestMove and TestFsShellReturnCode failed to get command name using reflection. (Lei Xu)


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

Branch: refs/heads/YARN-2139
Commit: 532e38cb7f70606c2c96d05259670e1e91d60ab3
Parents: b8e792c
Author: Lei Xu <le...@apache.org>
Authored: Tue Jun 30 17:26:49 2015 -0700
Committer: Lei Xu <le...@apache.org>
Committed: Tue Jun 30 17:41:59 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +
 .../apache/hadoop/fs/TestFsShellReturnCode.java | 60 +++++++++++++++-----
 .../org/apache/hadoop/fs/shell/TestMove.java    |  1 +
 3 files changed, 49 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/532e38cb/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 74d428f..39e2e5e 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -916,6 +916,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12116. Fix unrecommended syntax usages in hadoop/hdfs/yarn script for
     cygwin in branch-2. (Li Lu via cnauroth)
 
+    HADOOP-12164. Fix TestMove and TestFsShellReturnCode failed to get command
+    name using reflection. (Lei (Eddy) Xu)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/532e38cb/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java
index dcc19df..a2276ec 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.fs;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SHELL_MISSING_DEFAULT_FS_WARNING_KEY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -402,7 +403,7 @@ public class TestFsShellReturnCode {
     // processing a file throws an interrupt, it should blow on first file
     assertEquals(1, InterruptCommand.processed);
     assertEquals(130, exitCode);
-    
+
     exitCode = shell.run(
         new String[]{ "-testInterrupt", d.toString() });
     // processing a file throws an interrupt, it should blow on file
@@ -412,17 +413,32 @@ public class TestFsShellReturnCode {
   }
 
   /**
+   * Faked Chown class for {@link testChownUserAndGroupValidity()}.
+   *
+   * The test only covers argument parsing, so override to skip processing.
+   */
+  private static class FakeChown extends FsShellPermissions.Chown {
+    public static String NAME = "chown";
+    @Override
+    protected void processArgument(PathData item) {
+    }
+  }
+
+  /**
    * Tests combinations of valid and invalid user and group arguments to chown.
    */
   @Test
   public void testChownUserAndGroupValidity() {
-    // This test only covers argument parsing, so override to skip processing.
-    FsCommand chown = new FsShellPermissions.Chown() {
-      @Override
-      protected void processArgument(PathData item) {
-      }
-    };
-    chown.setConf(new Configuration());
+    testChownUserAndGroupValidity(true);
+    testChownUserAndGroupValidity(false);
+  }
+
+  private void testChownUserAndGroupValidity(boolean enableWarning) {
+    Configuration conf = new Configuration();
+    conf.setBoolean(
+        HADOOP_SHELL_MISSING_DEFAULT_FS_WARNING_KEY, enableWarning);
+    FsCommand chown = new FakeChown();
+    chown.setConf(conf);
 
     // The following are valid (no exception expected).
     chown.run("user", "/path");
@@ -447,17 +463,31 @@ public class TestFsShellReturnCode {
   }
 
   /**
+   * Faked Chgrp class for {@link testChgrpGroupValidity()}.
+   * The test only covers argument parsing, so override to skip processing.
+   */
+  private static class FakeChgrp extends FsShellPermissions.Chgrp {
+    public static String NAME = "chgrp";
+    @Override
+    protected void processArgument(PathData item) {
+    }
+  }
+
+  /**
    * Tests valid and invalid group arguments to chgrp.
    */
   @Test
   public void testChgrpGroupValidity() {
-    // This test only covers argument parsing, so override to skip processing.
-    FsCommand chgrp = new FsShellPermissions.Chgrp() {
-      @Override
-      protected void processArgument(PathData item) {
-      }
-    };
-    chgrp.setConf(new Configuration());
+    testChgrpGroupValidity(true);
+    testChgrpGroupValidity(false);
+  }
+
+  private void testChgrpGroupValidity(boolean enableWarning) {
+    Configuration conf = new Configuration();
+    conf.setBoolean(
+        HADOOP_SHELL_MISSING_DEFAULT_FS_WARNING_KEY, enableWarning);
+    FsShellPermissions.Chgrp chgrp = new FakeChgrp();
+    chgrp.setConf(conf);
 
     // The following are valid (no exception expected).
     chgrp.run("group", "/path");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/532e38cb/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestMove.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestMove.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestMove.java
index 6599edf..94930e5 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestMove.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestMove.java
@@ -114,6 +114,7 @@ public class TestMove {
   }
     
   private static class InstrumentedRenameCommand extends MoveCommands.Rename {
+    public static String NAME = "InstrumentedRename";
     private Exception error = null;
     @Override
     public void displayError(Exception e) {


[24/50] hadoop git commit: HADOOP-12112. Make hadoop-common-project Native code -Wall-clean (alanburlison via cmccabe)

Posted by vv...@apache.org.
HADOOP-12112. Make hadoop-common-project Native code -Wall-clean (alanburlison via cmccabe)


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

Branch: refs/heads/YARN-2139
Commit: fad291ea6dbe49782e33a32cd6608088951e2c58
Parents: 5a27c3f
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Mon Jun 29 12:34:47 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Mon Jun 29 12:35:16 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt              | 3 +++
 .../main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c | 1 +
 .../native/src/org/apache/hadoop/net/unix/DomainSocket.c     | 8 ++++----
 .../src/main/native/src/org/apache/hadoop/util/NativeCrc32.c | 4 ++--
 4 files changed, 10 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fad291ea/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index e8e85a0..9008ead 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -691,6 +691,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12036. Consolidate all of the cmake extensions in one directory
     (alanburlison via cmccabe)
 
+    HADOOP-12112. Make hadoop-common-project Native code -Wall-clean
+    (alanburlison via cmccabe)
+
   BUG FIXES
 
     HADOOP-11802: DomainSocketWatcher thread terminates sometimes after there

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fad291ea/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
index 071d830..bc78ab2 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
@@ -592,6 +592,7 @@ done:
 #else
   THROW(env, "java/io/IOException",
     "The function Windows.createFileWithMode0() is not supported on this platform");
+  return NULL;
 #endif
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fad291ea/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/net/unix/DomainSocket.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/net/unix/DomainSocket.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/net/unix/DomainSocket.c
index 55ab0c3..a3f27ee 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/net/unix/DomainSocket.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/net/unix/DomainSocket.c
@@ -112,9 +112,9 @@ static jthrowable newSocketException(JNIEnv *env, int errnum,
  * to the heap if necessary.
  */
 struct flexibleBuffer {
-  int8_t *curBuf;
-  int8_t *allocBuf;
-  int8_t stackBuf[8196];
+  jbyte *curBuf;
+  jbyte *allocBuf;
+  jbyte stackBuf[8196];
 };
 
 static jthrowable flexBufInit(JNIEnv *env, struct flexibleBuffer *flexBuf, jint length)
@@ -637,7 +637,7 @@ JNIEnv *env, jclass clazz, jint fd)
  * @return               NULL on success; or the unraised exception representing
  *                       the problem.
  */
-static jthrowable write_fully(JNIEnv *env, int fd, int8_t *buf, int amt)
+static jthrowable write_fully(JNIEnv *env, int fd, jbyte *buf, int amt)
 {
   int err, res;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fad291ea/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/NativeCrc32.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/NativeCrc32.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/NativeCrc32.c
index 74e09e6..3fceb02 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/NativeCrc32.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/NativeCrc32.c
@@ -168,7 +168,7 @@ JNIEXPORT void JNICALL Java_org_apache_hadoop_util_NativeCrc32_nativeComputeChun
   // Setup complete. Actually verify checksums.
   ret = bulk_crc(data, data_len, sums, crc_type,
                             bytes_per_checksum, verify ? &error_data : NULL);
-  if (likely(verify && ret == CHECKSUMS_VALID || !verify && ret == 0)) {
+  if (likely((verify && ret == CHECKSUMS_VALID) || (!verify && ret == 0))) {
     return;
   } else if (unlikely(verify && ret == INVALID_CHECKSUM_DETECTED)) {
     long pos = base_pos + (error_data.bad_data - data);
@@ -261,7 +261,7 @@ JNIEXPORT void JNICALL Java_org_apache_hadoop_util_NativeCrc32_nativeComputeChun
         env, error_data.got_crc, error_data.expected_crc,
         j_filename, pos);
       return;
-    } else if (unlikely(verify && ret != CHECKSUMS_VALID || !verify && ret != 0)) {
+    } else if (unlikely((verify && ret != CHECKSUMS_VALID) || (!verify && ret != 0))) {
       THROW(env, "java/lang/AssertionError",
         "Bad response code from native bulk_crc");
       return;


[40/50] hadoop git commit: HADOOP-12158. Improve error message in TestCryptoStreamsWithOpensslAesCtrCryptoCodec when OpenSSL is not installed.

Posted by vv...@apache.org.
HADOOP-12158. Improve error message in TestCryptoStreamsWithOpensslAesCtrCryptoCodec when OpenSSL is not installed.


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

Branch: refs/heads/YARN-2139
Commit: 9ee7b6e6c4ab6bee6304fa7904993c7cbd9a6cd2
Parents: 6f2a41e
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Jun 30 15:04:18 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue Jun 30 15:04:33 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt             | 5 +++++
 .../TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java      | 9 +++++++--
 2 files changed, 12 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ee7b6e6/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index e332ea8..2aab8b4 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -666,6 +666,11 @@ Release 2.8.0 - UNRELEASED
 
     HADOOP-11958. MetricsSystemImpl fails to show backtrace when an error
     occurs (Jason Lowe via jeagles)
+
+    HADOOP-12158. Improve error message in
+    TestCryptoStreamsWithOpensslAesCtrCryptoCodec when OpenSSL is not
+    installed. (wang)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ee7b6e6/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java
index 4f90a0c..684ec09 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java
@@ -19,9 +19,11 @@ package org.apache.hadoop.crypto;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.junit.Assert;
 import org.junit.BeforeClass;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
 public class TestCryptoStreamsWithOpensslAesCtrCryptoCodec 
     extends TestCryptoStreams {
   
@@ -32,7 +34,10 @@ public class TestCryptoStreamsWithOpensslAesCtrCryptoCodec
         CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_KEY,
         OpensslAesCtrCryptoCodec.class.getName());
     codec = CryptoCodec.getInstance(conf);
-    Assert.assertEquals(OpensslAesCtrCryptoCodec.class.getCanonicalName(), 
+    assertNotNull("Unable to instantiate codec " +
+        OpensslAesCtrCryptoCodec.class.getName() + ", is the required "
+        + "version of OpenSSL installed?", codec);
+    assertEquals(OpensslAesCtrCryptoCodec.class.getCanonicalName(),
         codec.getClass().getCanonicalName());
   }
 }


[46/50] hadoop git commit: MAPREDUCE-6121. JobResourceUpdater#compareFs() doesn't handle HA namespaces (rchiang via rkanter)

Posted by vv...@apache.org.
MAPREDUCE-6121. JobResourceUpdater#compareFs() doesn't handle HA namespaces (rchiang via rkanter)


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

Branch: refs/heads/YARN-2139
Commit: 3a72bfd08281fd271bef4f41289125d39c41928c
Parents: aaafa0b
Author: Robert Kanter <rk...@apache.org>
Authored: Tue Jun 30 16:46:41 2015 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Tue Jun 30 16:49:27 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 ++
 .../hadoop/mapreduce/JobResourceUploader.java   | 38 +-------------------
 2 files changed, 4 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a72bfd0/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 64b04cc..0baecf8 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -518,6 +518,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6413. TestLocalJobSubmission is failing with unknown host
     (zhihai xu via jlowe)
 
+    MAPREDUCE-6121. JobResourceUpdater#compareFs() doesn't handle HA namespaces
+    (rchiang via rkanter)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a72bfd0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
index 134de35..f3e4d2f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
@@ -194,7 +194,7 @@ class JobResourceUploader {
 
     FileSystem remoteFs = null;
     remoteFs = originalPath.getFileSystem(conf);
-    if (compareFs(remoteFs, jtFs)) {
+    if (FileUtil.compareFs(remoteFs, jtFs)) {
       return originalPath;
     }
     // this might have name collisions. copy will throw an exception
@@ -205,42 +205,6 @@ class JobResourceUploader {
     return newPath;
   }
 
-  /*
-   * see if two file systems are the same or not.
-   */
-  private boolean compareFs(FileSystem srcFs, FileSystem destFs) {
-    URI srcUri = srcFs.getUri();
-    URI dstUri = destFs.getUri();
-    if (srcUri.getScheme() == null) {
-      return false;
-    }
-    if (!srcUri.getScheme().equals(dstUri.getScheme())) {
-      return false;
-    }
-    String srcHost = srcUri.getHost();
-    String dstHost = dstUri.getHost();
-    if ((srcHost != null) && (dstHost != null)) {
-      try {
-        srcHost = InetAddress.getByName(srcHost).getCanonicalHostName();
-        dstHost = InetAddress.getByName(dstHost).getCanonicalHostName();
-      } catch (UnknownHostException ue) {
-        return false;
-      }
-      if (!srcHost.equals(dstHost)) {
-        return false;
-      }
-    } else if (srcHost == null && dstHost != null) {
-      return false;
-    } else if (srcHost != null && dstHost == null) {
-      return false;
-    }
-    // check for ports
-    if (srcUri.getPort() != dstUri.getPort()) {
-      return false;
-    }
-    return true;
-  }
-
   private void copyJar(Path originalJarPath, Path submitJarFile,
       short replication) throws IOException {
     jtFs.copyFromLocalFile(originalJarPath, submitJarFile);


[09/50] hadoop git commit: HDFS-8656. Preserve compatibility of ClientProtocol#rollingUpgrade after finalization.

Posted by vv...@apache.org.
HDFS-8656. Preserve compatibility of ClientProtocol#rollingUpgrade after finalization.


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

Branch: refs/heads/YARN-2139
Commit: 60b858bfa65e0feb665e1a84784a3d45e9091c66
Parents: de480d6
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Jun 26 11:30:59 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Jun 26 11:30:59 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  3 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../hdfs/server/namenode/FSNamesystem.java      |  8 ++-
 .../hdfs/server/namenode/NameNodeMXBean.java    |  5 +-
 .../apache/hadoop/hdfs/TestRollingUpgrade.java  | 63 ++++++++++++++++----
 .../server/namenode/TestNameNodeMXBean.java     |  5 +-
 6 files changed, 68 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/60b858bf/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index 7e27078..ab41911 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -853,7 +853,8 @@ public interface ClientProtocol {
   /**
    * Rolling upgrade operations.
    * @param action either query, prepare or finalize.
-   * @return rolling upgrade information.
+   * @return rolling upgrade information. On query, if no upgrade is in
+   * progress, returns null.
    */
   @Idempotent
   public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60b858bf/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index bb1b3ff..e287ea4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1110,6 +1110,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8626. Reserved RBW space is not released if creation of RBW File
     fails. (kanaka kumar avvaru via Arpit Agarwal)
 
+    HDFS08656. Preserve compatibility of ClientProtocol#rollingUpgrade after
+    finalization. (wang)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60b858bf/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 d0f4e08..e95007b 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
@@ -6757,10 +6757,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.READ);
     readLock();
     try {
-      if (rollingUpgradeInfo != null) {
-        boolean hasRollbackImage = this.getFSImage().hasRollbackFSImage();
-        rollingUpgradeInfo.setCreatedRollbackImages(hasRollbackImage);
+      if (!isRollingUpgrade()) {
+        return null;
       }
+      Preconditions.checkNotNull(rollingUpgradeInfo);
+      boolean hasRollbackImage = this.getFSImage().hasRollbackFSImage();
+      rollingUpgradeInfo.setCreatedRollbackImages(hasRollbackImage);
       return rollingUpgradeInfo;
     } finally {
       readUnlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60b858bf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
index 3c317fd..0e4d445 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
@@ -81,9 +81,10 @@ public interface NameNodeMXBean {
   public boolean isUpgradeFinalized();
 
   /**
-   * Gets the RollingUpgrade information
+   * Gets the RollingUpgrade information.
    *
-   * @return Rolling upgrade information
+   * @return Rolling upgrade information if an upgrade is in progress. Else
+   * (e.g. if there is no upgrade or the upgrade is finalized), returns null.
    */
   public RollingUpgradeInfo.Bean getRollingUpgradeStatus();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60b858bf/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
index b50b1cb..72e16e4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
@@ -19,8 +19,18 @@ package org.apache.hadoop.hdfs;
 
 import java.io.File;
 import java.io.IOException;
+import java.lang.management.ManagementFactory;
 import java.util.concurrent.ThreadLocalRandom;
 
+import javax.management.AttributeNotFoundException;
+import javax.management.InstanceNotFoundException;
+import javax.management.MBeanException;
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.management.ReflectionException;
+import javax.management.openmbean.CompositeDataSupport;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -45,7 +55,9 @@ import org.apache.hadoop.io.IOUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNull;
 
 /**
  * This class tests rolling upgrade.
@@ -56,7 +68,7 @@ public class TestRollingUpgrade {
   public static void runCmd(DFSAdmin dfsadmin, boolean success,
       String... args) throws  Exception {
     if (success) {
-      Assert.assertEquals(0, dfsadmin.run(args));
+      assertEquals(0, dfsadmin.run(args));
     } else {
       Assert.assertTrue(dfsadmin.run(args) != 0);
     }
@@ -86,6 +98,7 @@ public class TestRollingUpgrade {
         //illegal argument "abc" to rollingUpgrade option
         runCmd(dfsadmin, false, "-rollingUpgrade", "abc");
 
+        checkMxBeanIsNull();
         //query rolling upgrade
         runCmd(dfsadmin, true, "-rollingUpgrade");
 
@@ -96,11 +109,16 @@ public class TestRollingUpgrade {
 
         //query rolling upgrade
         runCmd(dfsadmin, true, "-rollingUpgrade", "query");
+        checkMxBean();
 
         dfs.mkdirs(bar);
 
         //finalize rolling upgrade
         runCmd(dfsadmin, true, "-rollingUpgrade", "finalize");
+        // RollingUpgradeInfo should be null after finalization, both via
+        // Java API and in JMX
+        assertNull(dfs.rollingUpgrade(RollingUpgradeAction.QUERY));
+        checkMxBeanIsNull();
 
         dfs.mkdirs(baz);
 
@@ -197,7 +215,7 @@ public class TestRollingUpgrade {
         LOG.info("START\n" + info1);
 
         //query rolling upgrade
-        Assert.assertEquals(info1, dfs.rollingUpgrade(RollingUpgradeAction.QUERY));
+        assertEquals(info1, dfs.rollingUpgrade(RollingUpgradeAction.QUERY));
 
         dfs.mkdirs(bar);
         cluster.shutdown();
@@ -218,13 +236,13 @@ public class TestRollingUpgrade {
       Assert.assertFalse(dfs2.exists(baz));
 
       //query rolling upgrade in cluster2
-      Assert.assertEquals(info1, dfs2.rollingUpgrade(RollingUpgradeAction.QUERY));
+      assertEquals(info1, dfs2.rollingUpgrade(RollingUpgradeAction.QUERY));
 
       dfs2.mkdirs(baz);
 
       LOG.info("RESTART cluster 2");
       cluster2.restartNameNode();
-      Assert.assertEquals(info1, dfs2.rollingUpgrade(RollingUpgradeAction.QUERY));
+      assertEquals(info1, dfs2.rollingUpgrade(RollingUpgradeAction.QUERY));
       Assert.assertTrue(dfs2.exists(foo));
       Assert.assertTrue(dfs2.exists(bar));
       Assert.assertTrue(dfs2.exists(baz));
@@ -238,7 +256,7 @@ public class TestRollingUpgrade {
 
       LOG.info("RESTART cluster 2 again");
       cluster2.restartNameNode();
-      Assert.assertEquals(info1, dfs2.rollingUpgrade(RollingUpgradeAction.QUERY));
+      assertEquals(info1, dfs2.rollingUpgrade(RollingUpgradeAction.QUERY));
       Assert.assertTrue(dfs2.exists(foo));
       Assert.assertTrue(dfs2.exists(bar));
       Assert.assertTrue(dfs2.exists(baz));
@@ -259,8 +277,30 @@ public class TestRollingUpgrade {
     }
   }
 
+  private static CompositeDataSupport getBean()
+      throws MalformedObjectNameException, MBeanException,
+      AttributeNotFoundException, InstanceNotFoundException,
+      ReflectionException {
+    MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+    ObjectName mxbeanName =
+        new ObjectName("Hadoop:service=NameNode,name=NameNodeInfo");
+    return (CompositeDataSupport)mbs.getAttribute(mxbeanName,
+        "RollingUpgradeStatus");
+  }
+
+  private static void checkMxBeanIsNull() throws Exception {
+    CompositeDataSupport ruBean = getBean();
+    assertNull(ruBean);
+  }
+
+  private static void checkMxBean() throws Exception {
+    CompositeDataSupport ruBean = getBean();
+    assertNotEquals(0l, ruBean.get("startTime"));
+    assertEquals(0l, ruBean.get("finalizeTime"));
+  }
+
   @Test
-  public void testRollback() throws IOException {
+  public void testRollback() throws Exception {
     // start a cluster
     final Configuration conf = new HdfsConfiguration();
     MiniDFSCluster cluster = null;
@@ -279,10 +319,13 @@ public class TestRollingUpgrade {
       out.write(data, 0, data.length);
       out.close();
 
+      checkMxBeanIsNull();
       startRollingUpgrade(foo, bar, file, data, cluster);
+      checkMxBean();
       cluster.getFileSystem().rollEdits();
       cluster.getFileSystem().rollEdits();
       rollbackRollingUpgrade(foo, bar, file, data, cluster);
+      checkMxBeanIsNull();
 
       startRollingUpgrade(foo, bar, file, data, cluster);
       cluster.getFileSystem().rollEdits();
@@ -356,18 +399,18 @@ public class TestRollingUpgrade {
       // check the datanode
       final String dnAddr = dn.getDatanodeId().getIpcAddr(false);
       final String[] args1 = {"-getDatanodeInfo", dnAddr};
-      Assert.assertEquals(0, dfsadmin.run(args1));
+      runCmd(dfsadmin, true, args1);
 
       // issue shutdown to the datanode.
       final String[] args2 = {"-shutdownDatanode", dnAddr, "upgrade" };
-      Assert.assertEquals(0, dfsadmin.run(args2));
+      runCmd(dfsadmin, true, args2);
 
       // the datanode should be down.
       Thread.sleep(2000);
       Assert.assertFalse("DataNode should exit", dn.isDatanodeUp());
 
       // ping should fail.
-      Assert.assertEquals(-1, dfsadmin.run(args1));
+      assertEquals(-1, dfsadmin.run(args1));
     } finally {
       if (cluster != null) cluster.shutdown();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60b858bf/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 681e8a4..2087c44 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
@@ -28,7 +28,6 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.top.TopConf;
-import org.apache.hadoop.hdfs.server.namenode.top.window.RollingWindowManager;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.io.nativeio.NativeIO.POSIX.NoMlockCacheManipulator;
 import org.apache.hadoop.util.VersionInfo;
@@ -46,8 +45,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
-import static org.apache.hadoop.hdfs.server.namenode.top.window.RollingWindowManager.Op;
-import static org.apache.hadoop.hdfs.server.namenode.top.window.RollingWindowManager.TopWindow;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
@@ -197,6 +194,8 @@ public class TestNameNodeMXBean {
       assertEquals(NativeIO.POSIX.getCacheManipulator().getMemlockLimit() *
           cluster.getDataNodes().size(),
               mbs.getAttribute(mxbeanName, "CacheCapacity"));
+      assertNull("RollingUpgradeInfo should be null when there is no rolling"
+          + " upgrade", mbs.getAttribute(mxbeanName, "RollingUpgradeStatus"));
     } finally {
       if (cluster != null) {
         for (URI dir : cluster.getNameDirs(0)) {


[20/50] hadoop git commit: HDFS-8628. Update missing command option for fetchdt (Contributed by J.Andreina)

Posted by vv...@apache.org.
HDFS-8628. Update missing command option for fetchdt (Contributed by J.Andreina)


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

Branch: refs/heads/YARN-2139
Commit: fde20ffcef6dcd477f501b5f80c86665296711c5
Parents: c815344
Author: Vinayakumar B <vi...@apache.org>
Authored: Mon Jun 29 16:50:48 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Mon Jun 29 16:50:48 2015 +0530

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt               |  3 +++
 .../hadoop-hdfs/src/site/markdown/HDFSCommands.md         | 10 +++++++---
 2 files changed, 10 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fde20ffc/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 3800184..e55f340 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -971,6 +971,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8586. Dead Datanode is allocated for write when client is from deadnode
     (Brahma Reddy Battula via vinayakumarb)
 
+    HDFS-8628. Update missing command option for fetchdt
+    (J.Andreina via vinayakumarb)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fde20ffc/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
index 6841201..094dda1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -89,12 +89,16 @@ Run a filesystem command on the file system supported in Hadoop. The various COM
 
 ### `fetchdt`
 
-Usage: `hdfs fetchdt [--webservice <namenode_http_addr>] <path> `
+Usage: `hdfs fetchdt <opts> <token_file_path> `
 
 | COMMAND\_OPTION | Description |
 |:---- |:---- |
-| `--webservice` *https\_address* | use http protocol instead of RPC |
-| *fileName* | File name to store the token into. |
+| `--webservice` *NN_Url* | Url to contact NN on (starts with http or https)|
+| `--renewer` *name* | Name of the delegation token renewer |
+| `--cancel` | Cancel the delegation token |
+| `--renew` | Renew the delegation token.  Delegation token must have been fetched using the --renewer *name* option.|
+| `--print` | Print the delegation token |
+| *token_file_path* | File path to store the token into. |
 
 Gets Delegation Token from a NameNode. See [fetchdt](./HdfsUserGuide.html#fetchdt) for more info.