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 ji...@apache.org on 2015/05/06 01:20:41 UTC

[01/17] hadoop git commit: MAPREDUCE-6165. [JDK8] TestCombineFileInputFormat failed on JDK8. Contributed by Akira AJISAKA.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 d701acc9c -> 4da8490b5


MAPREDUCE-6165. [JDK8] TestCombineFileInputFormat failed on JDK8. Contributed by Akira AJISAKA.


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

Branch: refs/heads/HDFS-7240
Commit: 551615fa13f65ae996bae9c1bacff189539b6557
Parents: d701acc
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Tue May 5 10:23:13 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Tue May 5 10:23:13 2015 +0900

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |    3 +
 .../lib/input/CombineFileInputFormat.java       |   26 +-
 .../lib/input/TestCombineFileInputFormat.java   | 1138 ++++++++++++------
 3 files changed, 805 insertions(+), 362 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/551615fa/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 481757a..002fbe6 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -368,6 +368,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-5649. Reduce cannot use more than 2G memory for the final merge
     (Gera Shegalov via jlowe)
 
+    MAPREDUCE-6165. [JDK8] TestCombineFileInputFormat failed on JDK8.
+    (Akira AJISAKA via ozawa)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/551615fa/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineFileInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineFileInputFormat.java
index 040c54b..b2b7656 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineFileInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineFileInputFormat.java
@@ -29,7 +29,6 @@ import java.util.HashMap;
 import java.util.Set;
 import java.util.Iterator;
 import java.util.Map;
-import java.util.Map.Entry;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -289,6 +288,26 @@ public abstract class CombineFileInputFormat<K, V>
                  maxSize, minSizeNode, minSizeRack, splits);
   }
 
+  /**
+   * Process all the nodes and create splits that are local to a node.
+   * Generate one split per node iteration, and walk over nodes multiple times
+   * to distribute the splits across nodes.
+   * <p>
+   * Note: The order of processing the nodes is undetermined because the
+   * implementation of nodeToBlocks is {@link java.util.HashMap} and its order
+   * of the entries is undetermined.
+   * @param nodeToBlocks Mapping from a node to the list of blocks that
+   *                     it contains.
+   * @param blockToNodes Mapping from a block to the nodes on which
+   *                     it has replicas.
+   * @param rackToBlocks Mapping from a rack name to the list of blocks it has.
+   * @param totLength Total length of the input files.
+   * @param maxSize Max size of each split.
+   *                If set to 0, disable smoothing load.
+   * @param minSizeNode Minimum split size per node.
+   * @param minSizeRack Minimum split size per rack.
+   * @param splits New splits created by this method are added to the list.
+   */
   @VisibleForTesting
   void createSplits(Map<String, Set<OneBlockInfo>> nodeToBlocks,
                      Map<OneBlockInfo, String[]> blockToNodes,
@@ -309,11 +328,6 @@ public abstract class CombineFileInputFormat<K, V>
     Set<String> completedNodes = new HashSet<String>();
     
     while(true) {
-      // it is allowed for maxSize to be 0. Disable smoothing load for such cases
-
-      // process all nodes and create splits that are local to a node. Generate
-      // one split per node iteration, and walk over nodes multiple times to
-      // distribute the splits across nodes. 
       for (Iterator<Map.Entry<String, Set<OneBlockInfo>>> iter = nodeToBlocks
           .entrySet().iterator(); iter.hasNext();) {
         Map.Entry<String, Set<OneBlockInfo>> one = iter.next();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/551615fa/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
index 85c675c..b49f2d8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
@@ -22,6 +22,7 @@ import java.io.OutputStream;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -53,13 +54,22 @@ import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat.OneBlockInfo;
 import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat.OneFileInfo;
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+
+import org.junit.Before;
 import org.junit.Test;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
 
 import com.google.common.collect.HashMultiset;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.verify;
 
 public class TestCombineFileInputFormat {
 
@@ -92,6 +102,14 @@ public class TestCombineFileInputFormat {
   static final int BLOCKSIZE = 1024;
   static final byte[] databuf = new byte[BLOCKSIZE];
 
+  @Mock
+  private List<String> mockList;
+
+  @Before
+  public void initMocks() {
+    MockitoAnnotations.initMocks(this);
+  }
+
   private static final String DUMMY_FS_URI = "dummyfs:///";
 
   /** Dummy class to extend CombineFileInputFormat*/
@@ -299,7 +317,51 @@ public class TestCombineFileInputFormat {
     assertFalse(rr.nextKeyValue());
   }
 
+  /**
+   * For testing each split has the expected name, length, and offset.
+   */
+  private final class Split {
+    private String name;
+    private long length;
+    private long offset;
+
+    public Split(String name, long length, long offset) {
+      this.name = name;
+      this.length = length;
+      this.offset = offset;
+    }
+
+    public String getName() {
+      return name;
+    }
+
+    public long getLength() {
+      return length;
+    }
+
+    public long getOffset() {
+      return offset;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj instanceof Split) {
+        Split split = ((Split) obj);
+        return split.name.equals(name) && split.length == length
+            && split.offset == offset;
+      }
+      return false;
+    }
+  }
+
+  /**
+   * The test suppresses unchecked warnings in
+   * {@link org.mockito.Mockito#reset}. Although calling the method is
+   * a bad manner, we call the method instead of splitting the test
+   * (i.e. restarting MiniDFSCluster) to save time.
+   */
   @Test
+  @SuppressWarnings("unchecked")
   public void testSplitPlacement() throws Exception {
     MiniDFSCluster dfs = null;
     FileSystem fileSys = null;
@@ -326,10 +388,10 @@ public class TestCombineFileInputFormat {
         throw new IOException("Mkdirs failed to create " + inDir.toString());
       }
       Path file1 = new Path(dir1 + "/file1");
-      writeFile(conf, file1, (short)1, 1);
+      writeFile(conf, file1, (short) 1, 1);
       // create another file on the same datanode
       Path file5 = new Path(dir5 + "/file5");
-      writeFile(conf, file5, (short)1, 1);
+      writeFile(conf, file5, (short) 1, 1);
       // split it using a CombinedFile input format
       DummyInputFormat inFormat = new DummyInputFormat();
       Job job = Job.getInstance(conf);
@@ -350,13 +412,13 @@ public class TestCombineFileInputFormat {
       assertEquals(0, fileSplit.getOffset(1));
       assertEquals(BLOCKSIZE, fileSplit.getLength(1));
       assertEquals(hosts1[0], fileSplit.getLocations()[0]);
-      
+
       dfs.startDataNodes(conf, 1, true, null, rack2, hosts2, null);
       dfs.waitActive();
 
       // create file on two datanodes.
       Path file2 = new Path(dir2 + "/file2");
-      writeFile(conf, file2, (short)2, 2);
+      writeFile(conf, file2, (short) 2, 2);
 
       // split it using a CombinedFile input format
       inFormat = new DummyInputFormat();
@@ -365,34 +427,67 @@ public class TestCombineFileInputFormat {
       splits = inFormat.getSplits(job);
       System.out.println("Made splits(Test1): " + splits.size());
 
-      // make sure that each split has different locations
       for (InputSplit split : splits) {
         System.out.println("File split(Test1): " + split);
       }
-      assertEquals(2, splits.size());
-      fileSplit = (CombineFileSplit) splits.get(0);
-      assertEquals(2, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(file2.getName(), fileSplit.getPath(1).getName());
-      assertEquals(BLOCKSIZE, fileSplit.getOffset(1));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(1));
-      assertEquals(hosts2[0], fileSplit.getLocations()[0]); // should be on r2
-      fileSplit = (CombineFileSplit) splits.get(1);
-      assertEquals(1, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r1
+
+      for (InputSplit split : splits) {
+        fileSplit = (CombineFileSplit) split;
+        /**
+         * If rack1 is processed first by
+         * {@link CombineFileInputFormat#createSplits},
+         * create only one split on rack1. Otherwise create two splits.
+         */
+        if (splits.size() == 2) {
+          // first split is on rack2, contains file2
+          if (split.equals(splits.get(0))) {
+            assertEquals(2, fileSplit.getNumPaths());
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(file2.getName(), fileSplit.getPath(0).getName());
+            assertEquals(0, fileSplit.getOffset(0));
+            assertEquals(BLOCKSIZE, fileSplit.getLength(0));
+            assertEquals(file2.getName(), fileSplit.getPath(1).getName());
+            assertEquals(BLOCKSIZE, fileSplit.getOffset(1));
+            assertEquals(BLOCKSIZE, fileSplit.getLength(1));
+            assertEquals(hosts2[0], fileSplit.getLocations()[0]);
+          }
+          // second split is on rack1, contains file1
+          if (split.equals(splits.get(1))) {
+            assertEquals(1, fileSplit.getNumPaths());
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(file1.getName(), fileSplit.getPath(0).getName());
+            assertEquals(0, fileSplit.getOffset(0));
+            assertEquals(BLOCKSIZE, fileSplit.getLength(0));
+            assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+          }
+        } else if (splits.size() == 1) {
+          // first split is on rack1, contains file1 and file2.
+          assertEquals(3, fileSplit.getNumPaths());
+          Set<Split> expected = new HashSet<>();
+          expected.add(new Split(file1.getName(), BLOCKSIZE, 0));
+          expected.add(new Split(file2.getName(), BLOCKSIZE, 0));
+          expected.add(new Split(file2.getName(), BLOCKSIZE, BLOCKSIZE));
+          List<Split> actual = new ArrayList<>();
+          for (int i = 0; i < 3; i++) {
+            String name = fileSplit.getPath(i).getName();
+            long length = fileSplit.getLength(i);
+            long offset = fileSplit.getOffset(i);
+            actual.add(new Split(name, length, offset));
+          }
+          assertTrue(actual.containsAll(expected));
+          assertEquals(1, fileSplit.getLocations().length);
+          assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+        } else {
+          fail("Expected split size is 1 or 2, but actual size is "
+              + splits.size());
+        }
+      }
 
       // create another file on 3 datanodes and 3 racks.
       dfs.startDataNodes(conf, 1, true, null, rack3, hosts3, null);
       dfs.waitActive();
       Path file3 = new Path(dir3 + "/file3");
-      writeFile(conf, new Path(dir3 + "/file3"), (short)3, 3);
+      writeFile(conf, new Path(dir3 + "/file3"), (short) 3, 3);
       inFormat = new DummyInputFormat();
       FileInputFormat.setInputPaths(job, dir1 + "," + dir2 + "," + dir3);
       inFormat.setMinSplitSizeRack(BLOCKSIZE);
@@ -400,37 +495,98 @@ public class TestCombineFileInputFormat {
       for (InputSplit split : splits) {
         System.out.println("File split(Test2): " + split);
       }
-      assertEquals(3, splits.size());
-      fileSplit = (CombineFileSplit) splits.get(0);
-      assertEquals(3, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file3.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(file3.getName(), fileSplit.getPath(1).getName());
-      assertEquals(BLOCKSIZE, fileSplit.getOffset(1));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(1));
-      assertEquals(file3.getName(), fileSplit.getPath(2).getName());
-      assertEquals(2 * BLOCKSIZE, fileSplit.getOffset(2));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(2));
-      assertEquals(hosts3[0], fileSplit.getLocations()[0]); // should be on r3
-      fileSplit = (CombineFileSplit) splits.get(1);
-      assertEquals(2, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(file2.getName(), fileSplit.getPath(1).getName());
-      assertEquals(BLOCKSIZE, fileSplit.getOffset(1));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(1));
-      assertEquals(hosts2[0], fileSplit.getLocations()[0]); // should be on r2
-      fileSplit = (CombineFileSplit) splits.get(2);
-      assertEquals(1, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r1
+
+      Set<Split> expected = new HashSet<>();
+      expected.add(new Split(file1.getName(), BLOCKSIZE, 0));
+      expected.add(new Split(file2.getName(), BLOCKSIZE, 0));
+      expected.add(new Split(file2.getName(), BLOCKSIZE, BLOCKSIZE));
+      expected.add(new Split(file3.getName(), BLOCKSIZE, 0));
+      expected.add(new Split(file3.getName(), BLOCKSIZE, BLOCKSIZE));
+      expected.add(new Split(file3.getName(), BLOCKSIZE, BLOCKSIZE * 2));
+      List<Split> actual = new ArrayList<>();
+
+      for (InputSplit split : splits) {
+        fileSplit = (CombineFileSplit) split;
+        /**
+         * If rack1 is processed first by
+         * {@link CombineFileInputFormat#createSplits},
+         * create only one split on rack1.
+         * If rack2 or rack3 is processed first and rack1 is processed second,
+         * create one split on rack2 or rack3 and the other split is on rack1.
+         * Otherwise create 3 splits for each rack.
+         */
+        if (splits.size() == 3) {
+          // first split is on rack3, contains file3
+          if (split.equals(splits.get(0))) {
+            assertEquals(3, fileSplit.getNumPaths());
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(file3.getName(), fileSplit.getPath(0).getName());
+            assertEquals(0, fileSplit.getOffset(0));
+            assertEquals(BLOCKSIZE, fileSplit.getLength(0));
+            assertEquals(file3.getName(), fileSplit.getPath(1).getName());
+            assertEquals(BLOCKSIZE, fileSplit.getOffset(1));
+            assertEquals(BLOCKSIZE, fileSplit.getLength(1));
+            assertEquals(file3.getName(), fileSplit.getPath(2).getName());
+            assertEquals(2 * BLOCKSIZE, fileSplit.getOffset(2));
+            assertEquals(BLOCKSIZE, fileSplit.getLength(2));
+            assertEquals(hosts3[0], fileSplit.getLocations()[0]);
+          }
+          // second split is on rack2, contains file2
+          if (split.equals(splits.get(1))) {
+            assertEquals(2, fileSplit.getNumPaths());
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(file2.getName(), fileSplit.getPath(0).getName());
+            assertEquals(0, fileSplit.getOffset(0));
+            assertEquals(BLOCKSIZE, fileSplit.getLength(0));
+            assertEquals(file2.getName(), fileSplit.getPath(1).getName());
+            assertEquals(BLOCKSIZE, fileSplit.getOffset(1));
+            assertEquals(BLOCKSIZE, fileSplit.getLength(1));
+            assertEquals(hosts2[0], fileSplit.getLocations()[0]);
+          }
+          // third split is on rack1, contains file1
+          if (split.equals(splits.get(2))) {
+            assertEquals(1, fileSplit.getNumPaths());
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(file1.getName(), fileSplit.getPath(0).getName());
+            assertEquals(0, fileSplit.getOffset(0));
+            assertEquals(BLOCKSIZE, fileSplit.getLength(0));
+            assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+          }
+        } else if (splits.size() == 2) {
+          // first split is on rack2 or rack3, contains one or two files.
+          if (split.equals(splits.get(0))) {
+            assertEquals(1, fileSplit.getLocations().length);
+            if (fileSplit.getLocations()[0].equals(hosts2[0])) {
+              assertEquals(2, fileSplit.getNumPaths());
+            } else if (fileSplit.getLocations()[0].equals(hosts3[0])) {
+              assertEquals(3, fileSplit.getNumPaths());
+            } else {
+              fail("First split should be on rack2 or rack3.");
+            }
+          }
+          // second split is on rack1, contains the rest files.
+          if (split.equals(splits.get(1))) {
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+          }
+        } else if (splits.size() == 1) {
+          // first split is rack1, contains all three files.
+          assertEquals(1, fileSplit.getLocations().length);
+          assertEquals(6, fileSplit.getNumPaths());
+          assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+        } else {
+          fail("Split size should be 1, 2, or 3.");
+        }
+        for (int i = 0; i < fileSplit.getNumPaths(); i++) {
+          String name = fileSplit.getPath(i).getName();
+          long length = fileSplit.getLength(i);
+          long offset = fileSplit.getOffset(i);
+          actual.add(new Split(name, length, offset));
+        }
+      }
+
+      assertEquals(6, actual.size());
+      assertTrue(actual.containsAll(expected));
 
       // create file4 on all three racks
       Path file4 = new Path(dir4 + "/file4");
@@ -442,37 +598,85 @@ public class TestCombineFileInputFormat {
       for (InputSplit split : splits) {
         System.out.println("File split(Test3): " + split);
       }
-      assertEquals(3, splits.size());
-      fileSplit = (CombineFileSplit) splits.get(0);
-      assertEquals(6, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file3.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(file3.getName(), fileSplit.getPath(1).getName());
-      assertEquals(BLOCKSIZE, fileSplit.getOffset(1));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(1));
-      assertEquals(file3.getName(), fileSplit.getPath(2).getName());
-      assertEquals(2 * BLOCKSIZE, fileSplit.getOffset(2));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(2));
-      assertEquals(hosts3[0], fileSplit.getLocations()[0]); // should be on r3
-      fileSplit = (CombineFileSplit) splits.get(1);
-      assertEquals(2, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(file2.getName(), fileSplit.getPath(1).getName());
-      assertEquals(BLOCKSIZE, fileSplit.getOffset(1));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(1));
-      assertEquals(hosts2[0], fileSplit.getLocations()[0]); // should be on r2
-      fileSplit = (CombineFileSplit) splits.get(2);
-      assertEquals(1, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r1
+
+      expected.add(new Split(file4.getName(), BLOCKSIZE, 0));
+      expected.add(new Split(file4.getName(), BLOCKSIZE, BLOCKSIZE));
+      expected.add(new Split(file4.getName(), BLOCKSIZE, BLOCKSIZE * 2));
+      actual.clear();
+
+      for (InputSplit split : splits) {
+        fileSplit = (CombineFileSplit) split;
+        /**
+         * If rack1 is processed first by
+         * {@link CombineFileInputFormat#createSplits},
+         * create only one split on rack1.
+         * If rack2 or rack3 is processed first and rack1 is processed second,
+         * create one split on rack2 or rack3 and the other split is on rack1.
+         * Otherwise create 3 splits for each rack.
+         */
+        if (splits.size() == 3) {
+          // first split is on rack3, contains file3 and file4
+          if (split.equals(splits.get(0))) {
+            assertEquals(6, fileSplit.getNumPaths());
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(hosts3[0], fileSplit.getLocations()[0]);
+          }
+          // second split is on rack2, contains file2
+          if (split.equals(splits.get(1))) {
+            assertEquals(2, fileSplit.getNumPaths());
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(file2.getName(), fileSplit.getPath(0).getName());
+            assertEquals(0, fileSplit.getOffset(0));
+            assertEquals(BLOCKSIZE, fileSplit.getLength(0));
+            assertEquals(file2.getName(), fileSplit.getPath(1).getName());
+            assertEquals(BLOCKSIZE, fileSplit.getOffset(1));
+            assertEquals(BLOCKSIZE, fileSplit.getLength(1));
+            assertEquals(hosts2[0], fileSplit.getLocations()[0]);
+          }
+          // third split is on rack1, contains file1
+          if (split.equals(splits.get(2))) {
+            assertEquals(1, fileSplit.getNumPaths());
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(file1.getName(), fileSplit.getPath(0).getName());
+            assertEquals(0, fileSplit.getOffset(0));
+            assertEquals(BLOCKSIZE, fileSplit.getLength(0));
+            assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+          }
+        } else if (splits.size() == 2) {
+          // first split is on rack2 or rack3, contains two or three files.
+          if (split.equals(splits.get(0))) {
+            assertEquals(1, fileSplit.getLocations().length);
+            if (fileSplit.getLocations()[0].equals(hosts2[0])) {
+              assertEquals(5, fileSplit.getNumPaths());
+            } else if (fileSplit.getLocations()[0].equals(hosts3[0])) {
+              assertEquals(6, fileSplit.getNumPaths());
+            } else {
+              fail("First split should be on rack2 or rack3.");
+            }
+          }
+          // second split is on rack1, contains the rest files.
+          if (split.equals(splits.get(1))) {
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+          }
+        } else if (splits.size() == 1) {
+          // first split is rack1, contains all four files.
+          assertEquals(1, fileSplit.getLocations().length);
+          assertEquals(9, fileSplit.getNumPaths());
+          assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+        } else {
+          fail("Split size should be 1, 2, or 3.");
+        }
+        for (int i = 0; i < fileSplit.getNumPaths(); i++) {
+          String name = fileSplit.getPath(i).getName();
+          long length = fileSplit.getLength(i);
+          long offset = fileSplit.getOffset(i);
+          actual.add(new Split(name, length, offset));
+        }
+      }
+
+      assertEquals(9, actual.size());
+      assertTrue(actual.containsAll(expected));
 
       // maximum split size is 2 blocks 
       inFormat = new DummyInputFormat();
@@ -485,34 +689,26 @@ public class TestCombineFileInputFormat {
         System.out.println("File split(Test4): " + split);
       }
       assertEquals(5, splits.size());
-      fileSplit = (CombineFileSplit) splits.get(0);
-      assertEquals(2, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file3.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(file3.getName(), fileSplit.getPath(1).getName());
-      assertEquals(BLOCKSIZE, fileSplit.getOffset(1));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(1));
-      assertEquals("host3.rack3.com", fileSplit.getLocations()[0]);
-      fileSplit = (CombineFileSplit) splits.get(1);
-      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(file2.getName(), fileSplit.getPath(1).getName());
-      assertEquals(BLOCKSIZE, fileSplit.getOffset(1));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(1));
-      assertEquals("host2.rack2.com", fileSplit.getLocations()[0]);
-      fileSplit = (CombineFileSplit) splits.get(2);
-      assertEquals(2, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(file3.getName(), fileSplit.getPath(1).getName());
-      assertEquals(2 * BLOCKSIZE, fileSplit.getOffset(1));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(1));
-      assertEquals("host1.rack1.com", fileSplit.getLocations()[0]);
+
+      actual.clear();
+      reset(mockList);
+      for (InputSplit split : splits) {
+        fileSplit = (CombineFileSplit) split;
+        for (int i = 0; i < fileSplit.getNumPaths(); i++) {
+          String name = fileSplit.getPath(i).getName();
+          long length = fileSplit.getLength(i);
+          long offset = fileSplit.getOffset(i);
+          actual.add(new Split(name, length, offset));
+        }
+        mockList.add(fileSplit.getLocations()[0]);
+      }
+
+      assertEquals(9, actual.size());
+      assertTrue(actual.containsAll(expected));
+      // verify the splits are on all the racks
+      verify(mockList, atLeastOnce()).add(hosts1[0]);
+      verify(mockList, atLeastOnce()).add(hosts2[0]);
+      verify(mockList, atLeastOnce()).add(hosts3[0]);
 
       // maximum split size is 3 blocks 
       inFormat = new DummyInputFormat();
@@ -524,44 +720,26 @@ public class TestCombineFileInputFormat {
       for (InputSplit split : splits) {
         System.out.println("File split(Test5): " + split);
       }
+
       assertEquals(3, splits.size());
-      fileSplit = (CombineFileSplit) splits.get(0);
-      assertEquals(3, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file3.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(file3.getName(), fileSplit.getPath(1).getName());
-      assertEquals(BLOCKSIZE, fileSplit.getOffset(1));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(1));
-      assertEquals(file3.getName(), fileSplit.getPath(2).getName());
-      assertEquals(2 * BLOCKSIZE, fileSplit.getOffset(2));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(2));
-      assertEquals("host3.rack3.com", fileSplit.getLocations()[0]);
-      fileSplit = (CombineFileSplit) splits.get(1);
-      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(file2.getName(), fileSplit.getPath(1).getName());
-      assertEquals(BLOCKSIZE, fileSplit.getOffset(1));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(1));
-      assertEquals(file4.getName(), fileSplit.getPath(2).getName());
-      assertEquals(0, fileSplit.getOffset(2));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(2));
-      assertEquals("host2.rack2.com", fileSplit.getLocations()[0]);
-      fileSplit = (CombineFileSplit) splits.get(2);
-      assertEquals(3, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(file4.getName(), fileSplit.getPath(1).getName());
-      assertEquals(BLOCKSIZE, fileSplit.getOffset(1));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(1));
-      assertEquals(file4.getName(), fileSplit.getPath(2).getName());
-      assertEquals(2*BLOCKSIZE, fileSplit.getOffset(2));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(2));
-      assertEquals("host1.rack1.com", fileSplit.getLocations()[0]);
+
+      actual.clear();
+      reset(mockList);
+      for (InputSplit split : splits) {
+        fileSplit = (CombineFileSplit) split;
+        for (int i = 0; i < fileSplit.getNumPaths(); i++) {
+          String name = fileSplit.getPath(i).getName();
+          long length = fileSplit.getLength(i);
+          long offset = fileSplit.getOffset(i);
+          actual.add(new Split(name, length, offset));
+        }
+        mockList.add(fileSplit.getLocations()[0]);
+      }
+
+      assertEquals(9, actual.size());
+      assertTrue(actual.containsAll(expected));
+      verify(mockList, atLeastOnce()).add(hosts1[0]);
+      verify(mockList, atLeastOnce()).add(hosts2[0]);
 
       // maximum split size is 4 blocks 
       inFormat = new DummyInputFormat();
@@ -572,41 +750,23 @@ public class TestCombineFileInputFormat {
         System.out.println("File split(Test6): " + split);
       }
       assertEquals(3, splits.size());
-      fileSplit = (CombineFileSplit) splits.get(0);
-      assertEquals(4, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file3.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(file3.getName(), fileSplit.getPath(1).getName());
-      assertEquals(BLOCKSIZE, fileSplit.getOffset(1));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(1));
-      assertEquals(file3.getName(), fileSplit.getPath(2).getName());
-      assertEquals(2 * BLOCKSIZE, fileSplit.getOffset(2));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(2));
-      assertEquals("host3.rack3.com", fileSplit.getLocations()[0]);
-      fileSplit = (CombineFileSplit) splits.get(1);
-      assertEquals(4, fileSplit.getNumPaths());
-      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(file2.getName(), fileSplit.getPath(1).getName());
-      assertEquals(BLOCKSIZE, fileSplit.getOffset(1));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(1));
-      assertEquals(file4.getName(), fileSplit.getPath(2).getName());
-      assertEquals(BLOCKSIZE, fileSplit.getOffset(2));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(2));
-      assertEquals(file4.getName(), fileSplit.getPath(3).getName());
-      assertEquals( 2 * BLOCKSIZE, fileSplit.getOffset(3));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(3));
-      assertEquals("host2.rack2.com", fileSplit.getLocations()[0]);
-      fileSplit = (CombineFileSplit) splits.get(2);
-      assertEquals(1, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r1
+
+      actual.clear();
+      reset(mockList);
+      for (InputSplit split : splits) {
+        fileSplit = (CombineFileSplit) split;
+        for (int i = 0; i < fileSplit.getNumPaths(); i++) {
+          String name = fileSplit.getPath(i).getName();
+          long length = fileSplit.getLength(i);
+          long offset = fileSplit.getOffset(i);
+          actual.add(new Split(name, length, offset));
+        }
+        mockList.add(fileSplit.getLocations()[0]);
+      }
+
+      assertEquals(9, actual.size());
+      assertTrue(actual.containsAll(expected));
+      verify(mockList, atLeastOnce()).add(hosts1[0]);
 
       // maximum split size is 7 blocks and min is 3 blocks
       inFormat = new DummyInputFormat();
@@ -619,20 +779,31 @@ public class TestCombineFileInputFormat {
       for (InputSplit split : splits) {
         System.out.println("File split(Test7): " + split);
       }
+
       assertEquals(2, splits.size());
-      fileSplit = (CombineFileSplit) splits.get(0);
-      assertEquals(6, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals("host3.rack3.com", fileSplit.getLocations()[0]);
-      fileSplit = (CombineFileSplit) splits.get(1);
-      assertEquals(3, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals("host1.rack1.com", fileSplit.getLocations()[0]);
+
+      actual.clear();
+      reset(mockList);
+      for (InputSplit split : splits) {
+        fileSplit = (CombineFileSplit) split;
+        for (int i = 0; i < fileSplit.getNumPaths(); i++) {
+          String name = fileSplit.getPath(i).getName();
+          long length = fileSplit.getLength(i);
+          long offset = fileSplit.getOffset(i);
+          actual.add(new Split(name, length, offset));
+        }
+        mockList.add(fileSplit.getLocations()[0]);
+      }
+
+      assertEquals(9, actual.size());
+      assertTrue(actual.containsAll(expected));
+      verify(mockList, atLeastOnce()).add(hosts1[0]);
 
       // Rack 1 has file1, file2 and file3 and file4
       // Rack 2 has file2 and file3 and file4
       // Rack 3 has file3 and file4
-      // setup a filter so that only file1 and file2 can be combined
+      // setup a filter so that only (file1 and file2) or (file3 and file4)
+      // can be combined
       inFormat = new DummyInputFormat();
       FileInputFormat.addInputPath(job, inDir);
       inFormat.setMinSplitSizeRack(1); // everything is at least rack local
@@ -642,19 +813,101 @@ public class TestCombineFileInputFormat {
       for (InputSplit split : splits) {
         System.out.println("File split(Test1): " + split);
       }
-      assertEquals(3, splits.size());
-      fileSplit = (CombineFileSplit) splits.get(0);
-      assertEquals(2, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(hosts2[0], fileSplit.getLocations()[0]); // should be on r2
-      fileSplit = (CombineFileSplit) splits.get(1);
-      assertEquals(1, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r1
-      fileSplit = (CombineFileSplit) splits.get(2);
-      assertEquals(6, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(hosts3[0], fileSplit.getLocations()[0]); // should be on r3
+
+      for (InputSplit split : splits) {
+        fileSplit = (CombineFileSplit) split;
+        if (splits.size() == 2) {
+          // first split is on rack1, contains file1 and file2.
+          if (split.equals(splits.get(0))) {
+            assertEquals(3, fileSplit.getNumPaths());
+            expected.clear();
+            expected.add(new Split(file1.getName(), BLOCKSIZE, 0));
+            expected.add(new Split(file2.getName(), BLOCKSIZE, 0));
+            expected.add(new Split(file2.getName(), BLOCKSIZE, BLOCKSIZE));
+            actual.clear();
+            for (int i = 0; i < 3; i++) {
+              String name = fileSplit.getPath(i).getName();
+              long length = fileSplit.getLength(i);
+              long offset = fileSplit.getOffset(i);
+              actual.add(new Split(name, length, offset));
+            }
+            assertTrue(actual.containsAll(expected));
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+          }
+          if (split.equals(splits.get(1))) {
+            // second split contains the file3 and file4, however,
+            // the locations is undetermined.
+            assertEquals(6, fileSplit.getNumPaths());
+            expected.clear();
+            expected.add(new Split(file3.getName(), BLOCKSIZE, 0));
+            expected.add(new Split(file3.getName(), BLOCKSIZE, BLOCKSIZE));
+            expected.add(new Split(file3.getName(), BLOCKSIZE, BLOCKSIZE * 2));
+            expected.add(new Split(file4.getName(), BLOCKSIZE, 0));
+            expected.add(new Split(file4.getName(), BLOCKSIZE, BLOCKSIZE));
+            expected.add(new Split(file4.getName(), BLOCKSIZE, BLOCKSIZE * 2));
+            actual.clear();
+            for (int i = 0; i < 6; i++) {
+              String name = fileSplit.getPath(i).getName();
+              long length = fileSplit.getLength(i);
+              long offset = fileSplit.getOffset(i);
+              actual.add(new Split(name, length, offset));
+            }
+            assertTrue(actual.containsAll(expected));
+            assertEquals(1, fileSplit.getLocations().length);
+          }
+        } else if (splits.size() == 3) {
+          if (split.equals(splits.get(0))) {
+            // first split is on rack2, contains file2
+            assertEquals(2, fileSplit.getNumPaths());
+            expected.clear();
+            expected.add(new Split(file2.getName(), BLOCKSIZE, 0));
+            expected.add(new Split(file2.getName(), BLOCKSIZE, BLOCKSIZE));
+            actual.clear();
+            for (int i = 0; i < 2; i++) {
+              String name = fileSplit.getPath(i).getName();
+              long length = fileSplit.getLength(i);
+              long offset = fileSplit.getOffset(i);
+              actual.add(new Split(name, length, offset));
+            }
+            assertTrue(actual.containsAll(expected));
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(hosts2[0], fileSplit.getLocations()[0]);
+          }
+          if (split.equals(splits.get(1))) {
+            // second split is on rack1, contains file1
+            assertEquals(1, fileSplit.getNumPaths());
+            assertEquals(file1.getName(), fileSplit.getPath(0).getName());
+            assertEquals(BLOCKSIZE, fileSplit.getLength(0));
+            assertEquals(0, fileSplit.getOffset(0));
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+          }
+          if (split.equals(splits.get(2))) {
+            // third split contains file3 and file4, however,
+            // the locations is undetermined.
+            assertEquals(6, fileSplit.getNumPaths());
+            expected.clear();
+            expected.add(new Split(file3.getName(), BLOCKSIZE, 0));
+            expected.add(new Split(file3.getName(), BLOCKSIZE, BLOCKSIZE));
+            expected.add(new Split(file3.getName(), BLOCKSIZE, BLOCKSIZE * 2));
+            expected.add(new Split(file4.getName(), BLOCKSIZE, 0));
+            expected.add(new Split(file4.getName(), BLOCKSIZE, BLOCKSIZE));
+            expected.add(new Split(file4.getName(), BLOCKSIZE, BLOCKSIZE * 2));
+            actual.clear();
+            for (int i = 0; i < 6; i++) {
+              String name = fileSplit.getPath(i).getName();
+              long length = fileSplit.getLength(i);
+              long offset = fileSplit.getOffset(i);
+              actual.add(new Split(name, length, offset));
+            }
+            assertTrue(actual.containsAll(expected));
+            assertEquals(1, fileSplit.getLocations().length);
+          }
+        } else {
+          fail("Split size should be 2 or 3.");
+        }
+      }
 
       // measure performance when there are multiple pools and
       // many files in each pool.
@@ -844,7 +1097,14 @@ public class TestCombineFileInputFormat {
     assertEquals(3, nodeSplits.count(locations[1]));
   }
 
+  /**
+   * The test suppresses unchecked warnings in
+   * {@link org.mockito.Mockito#reset}. Although calling the method is
+   * a bad manner, we call the method instead of splitting the test
+   * (i.e. restarting MiniDFSCluster) to save time.
+   */
   @Test
+  @SuppressWarnings("unchecked")
   public void testSplitPlacementForCompressedFiles() throws Exception {
     MiniDFSCluster dfs = null;
     FileSystem fileSys = null;
@@ -915,21 +1175,55 @@ public class TestCombineFileInputFormat {
       for (InputSplit split : splits) {
         System.out.println("File split(Test1): " + split);
       }
-      assertEquals(2, splits.size());
-      fileSplit = (CombineFileSplit) splits.get(0);
-      assertEquals(1, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(f2.getLen(), fileSplit.getLength(0));
-      assertEquals(hosts2[0], fileSplit.getLocations()[0]); // should be on r2
-      fileSplit = (CombineFileSplit) splits.get(1);
-      assertEquals(1, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(f1.getLen(), fileSplit.getLength(0));
-      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r1
+
+      Set<Split> expected = new HashSet<>();
+      expected.add(new Split(file1.getName(), f1.getLen(), 0));
+      expected.add(new Split(file2.getName(), f2.getLen(), 0));
+      List<Split> actual = new ArrayList<>();
+
+      /**
+       * If rack1 is processed first by
+       * {@link CombineFileInputFormat#createSplits},
+       * create only one split on rack1. Otherwise create two splits.
+       */
+      for (InputSplit split : splits) {
+        fileSplit = (CombineFileSplit) split;
+        if (splits.size() == 2) {
+          if (split.equals(splits.get(0))) {
+            // first split is on rack2, contains file2.
+            assertEquals(1, fileSplit.getNumPaths());
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(file2.getName(), fileSplit.getPath(0).getName());
+            assertEquals(0, fileSplit.getOffset(0));
+            assertEquals(f2.getLen(), fileSplit.getLength(0));
+            assertEquals(hosts2[0], fileSplit.getLocations()[0]);
+          }
+          if (split.equals(splits.get(1))) {
+            // second split is on rack1, contains file1.
+            assertEquals(1, fileSplit.getNumPaths());
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(file1.getName(), fileSplit.getPath(0).getName());
+            assertEquals(0, fileSplit.getOffset(0));
+            assertEquals(f1.getLen(), fileSplit.getLength(0));
+            assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+          }
+        } else if (splits.size() == 1) {
+          // first split is on rack1, contains file1 and file2.
+          assertEquals(2, fileSplit.getNumPaths());
+          assertEquals(1, fileSplit.getLocations().length);
+          assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+        } else {
+          fail("Split size should be 1 or 2.");
+        }
+        for (int i = 0; i < fileSplit.getNumPaths(); i++) {
+          String name = fileSplit.getPath(i).getName();
+          long length = fileSplit.getLength(i);
+          long offset = fileSplit.getOffset(i);
+          actual.add(new Split(name, length, offset));
+        }
+      }
+      assertEquals(2, actual.size());
+      assertTrue(actual.containsAll(expected));
 
       // create another file on 3 datanodes and 3 racks.
       dfs.startDataNodes(conf, 1, true, null, rack3, hosts3, null);
@@ -943,28 +1237,83 @@ public class TestCombineFileInputFormat {
       for (InputSplit split : splits) {
         System.out.println("File split(Test2): " + split);
       }
-      assertEquals(3, splits.size());
-      fileSplit = (CombineFileSplit) splits.get(0);
-      assertEquals(1, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file3.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(f3.getLen(), fileSplit.getLength(0));
-      assertEquals(hosts3[0], fileSplit.getLocations()[0]); // should be on r3
-      fileSplit = (CombineFileSplit) splits.get(1);
-      assertEquals(1, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(f2.getLen(), fileSplit.getLength(0));
-      assertEquals(hosts2[0], fileSplit.getLocations()[0]); // should be on r2
-      fileSplit = (CombineFileSplit) splits.get(2);
-      assertEquals(1, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(f1.getLen(), fileSplit.getLength(0));
-      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r1
+
+      expected.add(new Split(file3.getName(), f3.getLen(), 0));
+      actual.clear();
+
+      for (InputSplit split : splits) {
+        fileSplit = (CombineFileSplit) split;
+        /**
+         * If rack1 is processed first by
+         * {@link CombineFileInputFormat#createSplits},
+         * create only one split on rack1.
+         * If rack2 or rack3 is processed first and rack1 is processed second,
+         * create one split on rack2 or rack3 and the other split is on rack1.
+         * Otherwise create 3 splits for each rack.
+         */
+        if (splits.size() == 3) {
+          // first split is on rack3, contains file3
+          if (split.equals(splits.get(0))) {
+            assertEquals(1, fileSplit.getNumPaths());
+            assertEquals(file3.getName(), fileSplit.getPath(0).getName());
+            assertEquals(f3.getLen(), fileSplit.getLength(0));
+            assertEquals(0, fileSplit.getOffset(0));
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(hosts3[0], fileSplit.getLocations()[0]);
+          }
+          // second split is on rack2, contains file2
+          if (split.equals(splits.get(1))) {
+            assertEquals(1, fileSplit.getNumPaths());
+            assertEquals(file2.getName(), fileSplit.getPath(0).getName());
+            assertEquals(f2.getLen(), fileSplit.getLength(0));
+            assertEquals(0, fileSplit.getOffset(0));
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(hosts2[0], fileSplit.getLocations()[0]);
+          }
+          // third split is on rack1, contains file1
+          if (split.equals(splits.get(2))) {
+            assertEquals(1, fileSplit.getNumPaths());
+            assertEquals(file1.getName(), fileSplit.getPath(0).getName());
+            assertEquals(f1.getLen(), fileSplit.getLength(0));
+            assertEquals(0, fileSplit.getOffset(0));
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+          }
+        } else if (splits.size() == 2) {
+          // first split is on rack2 or rack3, contains one or two files.
+          if (split.equals(splits.get(0))) {
+            assertEquals(1, fileSplit.getLocations().length);
+            if (fileSplit.getLocations()[0].equals(hosts2[0])) {
+              assertEquals(2, fileSplit.getNumPaths());
+            } else if (fileSplit.getLocations()[0].equals(hosts3[0])) {
+              assertEquals(1, fileSplit.getNumPaths());
+            } else {
+              fail("First split should be on rack2 or rack3.");
+            }
+          }
+          // second split is on rack1, contains the rest files.
+          if (split.equals(splits.get(1))) {
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+          }
+        } else if (splits.size() == 1) {
+          // first split is rack1, contains all three files.
+          assertEquals(1, fileSplit.getLocations().length);
+          assertEquals(3, fileSplit.getNumPaths());
+          assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+        } else {
+          fail("Split size should be 1, 2, or 3.");
+        }
+        for (int i = 0; i < fileSplit.getNumPaths(); i++) {
+          String name = fileSplit.getPath(i).getName();
+          long length = fileSplit.getLength(i);
+          long offset = fileSplit.getOffset(i);
+          actual.add(new Split(name, length, offset));
+        }
+      }
+
+      assertEquals(3, actual.size());
+      assertTrue(actual.containsAll(expected));
 
       // create file4 on all three racks
       Path file4 = new Path(dir4 + "/file4.gz");
@@ -977,31 +1326,79 @@ public class TestCombineFileInputFormat {
       for (InputSplit split : splits) {
         System.out.println("File split(Test3): " + split);
       }
-      assertEquals(3, splits.size());
-      fileSplit = (CombineFileSplit) splits.get(0);
-      assertEquals(2, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file3.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(f3.getLen(), fileSplit.getLength(0));
-      assertEquals(file4.getName(), fileSplit.getPath(1).getName());
-      assertEquals(0, fileSplit.getOffset(1));
-      assertEquals(f4.getLen(), fileSplit.getLength(1));
-      assertEquals(hosts3[0], fileSplit.getLocations()[0]); // should be on r3
-      fileSplit = (CombineFileSplit) splits.get(1);
-      assertEquals(1, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(f2.getLen(), fileSplit.getLength(0));
-      assertEquals(hosts2[0], fileSplit.getLocations()[0]); // should be on r2
-      fileSplit = (CombineFileSplit) splits.get(2);
-      assertEquals(1, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(f1.getLen(), fileSplit.getLength(0));
-      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r1
+
+      expected.add(new Split(file3.getName(), f3.getLen(), 0));
+      actual.clear();
+
+      for (InputSplit split : splits) {
+        fileSplit = (CombineFileSplit) split;
+        /**
+         * If rack1 is processed first by
+         * {@link CombineFileInputFormat#createSplits},
+         * create only one split on rack1.
+         * If rack2 or rack3 is processed first and rack1 is processed second,
+         * create one split on rack2 or rack3 and the other split is on rack1.
+         * Otherwise create 3 splits for each rack.
+         */
+        if (splits.size() == 3) {
+          // first split is on rack3, contains file3 and file4
+          if (split.equals(splits.get(0))) {
+            assertEquals(2, fileSplit.getNumPaths());
+            assertEquals(hosts3[0], fileSplit.getLocations()[0]);
+          }
+          // second split is on rack2, contains file2
+          if (split.equals(splits.get(1))) {
+            assertEquals(1, fileSplit.getNumPaths());
+            assertEquals(file2.getName(), fileSplit.getPath(0).getName());
+            assertEquals(f2.getLen(), fileSplit.getLength(0));
+            assertEquals(0, fileSplit.getOffset(0));
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(hosts2[0], fileSplit.getLocations()[0]);
+          }
+          // third split is on rack1, contains file1
+          if (split.equals(splits.get(2))) {
+            assertEquals(1, fileSplit.getNumPaths());
+            assertEquals(file1.getName(), fileSplit.getPath(0).getName());
+            assertEquals(f1.getLen(), fileSplit.getLength(0));
+            assertEquals(0, fileSplit.getOffset(0));
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+          }
+        } else if (splits.size() == 2) {
+          // first split is on rack2 or rack3, contains two or three files.
+          if (split.equals(splits.get(0))) {
+            assertEquals(1, fileSplit.getLocations().length);
+            if (fileSplit.getLocations()[0].equals(hosts2[0])) {
+              assertEquals(3, fileSplit.getNumPaths());
+            } else if (fileSplit.getLocations()[0].equals(hosts3[0])) {
+              assertEquals(2, fileSplit.getNumPaths());
+            } else {
+              fail("First split should be on rack2 or rack3.");
+            }
+          }
+          // second split is on rack1, contains the rest files.
+          if (split.equals(splits.get(1))) {
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+          }
+        } else if (splits.size() == 1) {
+          // first split is rack1, contains all four files.
+          assertEquals(1, fileSplit.getLocations().length);
+          assertEquals(4, fileSplit.getNumPaths());
+          assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+        } else {
+          fail("Split size should be 1, 2, or 3.");
+        }
+        for (int i = 0; i < fileSplit.getNumPaths(); i++) {
+          String name = fileSplit.getPath(i).getName();
+          long length = fileSplit.getLength(i);
+          long offset = fileSplit.getOffset(i);
+          actual.add(new Split(name, length, offset));
+        }
+      }
+
+      assertEquals(4, actual.size());
+      assertTrue(actual.containsAll(expected));
 
       // maximum split size is file1's length
       inFormat = new DummyInputFormat();
@@ -1014,32 +1411,24 @@ public class TestCombineFileInputFormat {
         System.out.println("File split(Test4): " + split);
       }
       assertEquals(4, splits.size());
-      fileSplit = (CombineFileSplit) splits.get(0);
-      assertEquals(1, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file3.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(f3.getLen(), fileSplit.getLength(0));
-      assertEquals(hosts3[0], fileSplit.getLocations()[0]); // should be on r3
-      fileSplit = (CombineFileSplit) splits.get(1);
-      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(f2.getLen(), fileSplit.getLength(0));
-      assertEquals(hosts2[0], fileSplit.getLocations()[0]); // should be on r3
-      fileSplit = (CombineFileSplit) splits.get(2);
-      assertEquals(1, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(f1.getLen(), fileSplit.getLength(0));
-      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r2
-      fileSplit = (CombineFileSplit) splits.get(3);
-      assertEquals(1, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file4.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(f4.getLen(), fileSplit.getLength(0));
-      assertEquals(hosts3[0], fileSplit.getLocations()[0]); // should be on r1
+
+      actual.clear();
+      for (InputSplit split : splits) {
+        fileSplit = (CombineFileSplit) split;
+        for (int i = 0; i < fileSplit.getNumPaths(); i++) {
+          String name = fileSplit.getPath(i).getName();
+          long length = fileSplit.getLength(i);
+          long offset = fileSplit.getOffset(i);
+          actual.add(new Split(name, length, offset));
+        }
+        mockList.add(fileSplit.getLocations()[0]);
+      }
+
+      assertEquals(4, actual.size());
+      assertTrue(actual.containsAll(expected));
+      verify(mockList, atLeastOnce()).add(hosts1[0]);
+      verify(mockList, atLeastOnce()).add(hosts2[0]);
+      verify(mockList, atLeastOnce()).add(hosts3[0]);
 
       // maximum split size is twice file1's length
       inFormat = new DummyInputFormat();
@@ -1051,31 +1440,33 @@ public class TestCombineFileInputFormat {
       for (InputSplit split : splits) {
         System.out.println("File split(Test5): " + split);
       }
-      assertEquals(3, splits.size());
-      fileSplit = (CombineFileSplit) splits.get(0);
-      assertEquals(2, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file3.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(f3.getLen(), fileSplit.getLength(0));
-      assertEquals(file4.getName(), fileSplit.getPath(1).getName());
-      assertEquals(0, fileSplit.getOffset(1));
-      assertEquals(f4.getLen(), fileSplit.getLength(1));
-      assertEquals(hosts3[0], fileSplit.getLocations()[0]);
-      fileSplit = (CombineFileSplit) splits.get(1);
-      assertEquals(1, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(f2.getLen(), fileSplit.getLength(0));
-      assertEquals(hosts2[0], fileSplit.getLocations()[0]); // should be on r2
-      fileSplit = (CombineFileSplit) splits.get(2);
-      assertEquals(1, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(f1.getLen(), fileSplit.getLength(0));
-      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r1
+
+      actual.clear();
+      reset(mockList);
+      for (InputSplit split : splits) {
+        fileSplit = (CombineFileSplit) split;
+        for (int i = 0; i < fileSplit.getNumPaths(); i++) {
+          String name = fileSplit.getPath(i).getName();
+          long length = fileSplit.getLength(i);
+          long offset = fileSplit.getOffset(i);
+          actual.add(new Split(name, length, offset));
+        }
+        mockList.add(fileSplit.getLocations()[0]);
+      }
+      assertEquals(4, actual.size());
+      assertTrue(actual.containsAll(expected));
+
+      if (splits.size() == 3) {
+        // splits are on all the racks
+        verify(mockList, times(1)).add(hosts1[0]);
+        verify(mockList, times(1)).add(hosts2[0]);
+        verify(mockList, times(1)).add(hosts3[0]);
+      } else if (splits.size() == 2) {
+        // one split is on rack1, another split is on rack2 or rack3
+        verify(mockList, times(1)).add(hosts1[0]);
+      } else {
+        fail("Split size should be 2 or 3.");
+      }
 
       // maximum split size is 4 times file1's length 
       inFormat = new DummyInputFormat();
@@ -1087,26 +1478,29 @@ public class TestCombineFileInputFormat {
       for (InputSplit split : splits) {
         System.out.println("File split(Test6): " + split);
       }
-      assertEquals(2, splits.size());
-      fileSplit = (CombineFileSplit) splits.get(0);
-      assertEquals(2, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file3.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(f3.getLen(), fileSplit.getLength(0));
-      assertEquals(file4.getName(), fileSplit.getPath(1).getName());
-      assertEquals(0, fileSplit.getOffset(1));
-      assertEquals(f4.getLen(), fileSplit.getLength(1));
-      assertEquals(hosts3[0], fileSplit.getLocations()[0]);
-      fileSplit = (CombineFileSplit) splits.get(1);
-      assertEquals(2, fileSplit.getNumPaths());
-      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(f1.getLen(), fileSplit.getLength(0));
-      assertEquals(file2.getName(), fileSplit.getPath(1).getName());
-      assertEquals(0, fileSplit.getOffset(1), BLOCKSIZE);
-      assertEquals(f2.getLen(), fileSplit.getLength(1));
-      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r1
+
+      /**
+       * If rack1 is processed first by
+       * {@link CombineFileInputFormat#createSplits},
+       * create only one split on rack1. Otherwise create two splits.
+       */
+      assertTrue("Split size should be 1 or 2.",
+          splits.size() == 1 || splits.size() == 2);
+      actual.clear();
+      reset(mockList);
+      for (InputSplit split : splits) {
+        fileSplit = (CombineFileSplit) split;
+        for (int i = 0; i < fileSplit.getNumPaths(); i++) {
+          String name = fileSplit.getPath(i).getName();
+          long length = fileSplit.getLength(i);
+          long offset = fileSplit.getOffset(i);
+          actual.add(new Split(name, length, offset));
+        }
+        mockList.add(fileSplit.getLocations()[0]);
+      }
+      assertEquals(4, actual.size());
+      assertTrue(actual.containsAll(expected));
+      verify(mockList, times(1)).add(hosts1[0]);
 
       // maximum split size and min-split-size per rack is 4 times file1's length
       inFormat = new DummyInputFormat();
@@ -1146,25 +1540,57 @@ public class TestCombineFileInputFormat {
       inFormat = new DummyInputFormat();
       FileInputFormat.addInputPath(job, inDir);
       inFormat.setMinSplitSizeRack(1); // everything is at least rack local
-      inFormat.createPool(new TestFilter(dir1), 
-                          new TestFilter(dir2));
+      inFormat.createPool(new TestFilter(dir1),
+          new TestFilter(dir2));
       splits = inFormat.getSplits(job);
       for (InputSplit split : splits) {
         System.out.println("File split(Test9): " + split);
       }
-      assertEquals(3, splits.size());
-      fileSplit = (CombineFileSplit) splits.get(0);
-      assertEquals(1, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(hosts2[0], fileSplit.getLocations()[0]); // should be on r2
-      fileSplit = (CombineFileSplit) splits.get(1);
-      assertEquals(1, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r1
-      fileSplit = (CombineFileSplit) splits.get(2);
-      assertEquals(2, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(hosts3[0], fileSplit.getLocations()[0]); // should be on r3
+
+      actual.clear();
+      for (InputSplit split : splits) {
+        fileSplit = (CombineFileSplit) split;
+        if (splits.size() == 3) {
+          // If rack2 is processed first
+          if (split.equals(splits.get(0))) {
+            assertEquals(1, fileSplit.getNumPaths());
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(hosts2[0], fileSplit.getLocations()[0]);
+          }
+          if (split.equals(splits.get(1))) {
+            assertEquals(1, fileSplit.getNumPaths());
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+          }
+          if (split.equals(splits.get(2))) {
+            assertEquals(2, fileSplit.getNumPaths());
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(hosts3[0], fileSplit.getLocations()[0]);
+          }
+        } else if (splits.size() == 2) {
+          // If rack1 is processed first
+          if (split.equals(splits.get(0))) {
+            assertEquals(2, fileSplit.getNumPaths());
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(hosts1[0], fileSplit.getLocations()[0]);
+          }
+          if (split.equals(splits.get(1))) {
+            assertEquals(2, fileSplit.getNumPaths());
+            assertEquals(1, fileSplit.getLocations().length);
+            assertEquals(hosts3[0], fileSplit.getLocations()[0]);
+          }
+        } else {
+          fail("Split size should be 2 or 3.");
+        }
+        for (int i = 0; i < fileSplit.getNumPaths(); i++) {
+          String name = fileSplit.getPath(i).getName();
+          long length = fileSplit.getLength(i);
+          long offset = fileSplit.getOffset(i);
+          actual.add(new Split(name, length, offset));
+        }
+      }
+      assertEquals(4, actual.size());
+      assertTrue(actual.containsAll(expected));
 
       // measure performance when there are multiple pools and
       // many files in each pool.


[09/17] hadoop git commit: HDFS-8305: HDFS INotify: the destination field of RenameOp should always end with the file name (cmccabe)

Posted by ji...@apache.org.
HDFS-8305: HDFS INotify: the destination field of RenameOp should always end with the file name (cmccabe)


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

Branch: refs/heads/HDFS-7240
Commit: fcd4cb751665adb241081e42b3403c3856b6c6fe
Parents: b7dd3a4
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Tue May 5 10:50:09 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Tue May 5 10:50:09 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt              |  3 +++
 .../hadoop/hdfs/server/namenode/FSDirRenameOp.java       |  2 +-
 .../apache/hadoop/hdfs/server/namenode/FSEditLog.java    | 10 +++++++---
 .../hadoop/hdfs/TestDFSInotifyEventInputStream.java      | 11 +++++++++++
 4 files changed, 22 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd4cb75/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 cd9b7b8..08ab7e7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -682,6 +682,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8091: ACLStatus and XAttributes should be presented to
     INodeAttributesProvider before returning to client (asuresh)
 
+    HDFS-8305: HDFS INotify: the destination field of RenameOp should always
+    end with the file name (cmccabe)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd4cb75/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
index c57cae2..4a20a62 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
@@ -476,7 +476,7 @@ class FSDirRenameOp {
       fsd.writeUnlock();
     }
     if (stat) {
-      fsd.getEditLog().logRename(src, dst, mtime, logRetryCache);
+      fsd.getEditLog().logRename(src, actualDst, mtime, logRetryCache);
       return true;
     }
     return false;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd4cb75/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index bda827a..28e150c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -815,7 +815,9 @@ public class FSEditLog implements LogsPurgeable {
   }
   
   /** 
-   * Add rename record to edit log
+   * Add rename record to edit log.
+   *
+   * The destination should be the file name, not the destination directory.
    * TODO: use String parameters until just before writing to disk
    */
   void logRename(String src, String dst, long timestamp, boolean toLogRpcIds) {
@@ -826,9 +828,11 @@ public class FSEditLog implements LogsPurgeable {
     logRpcIds(op, toLogRpcIds);
     logEdit(op);
   }
-  
+
   /** 
-   * Add rename record to edit log
+   * Add rename record to edit log.
+   *
+   * The destination should be the file name, not the destination directory.
    */
   void logRename(String src, String dst, long timestamp, boolean toLogRpcIds,
       Options.Rename... options) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd4cb75/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
index 6e91e06..ba33bd3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
@@ -135,6 +135,7 @@ public class TestDFSInotifyEventInputStream {
       client.setAcl("/file5", AclEntry.parseAclSpec(
           "user::rwx,user:foo:rw-,group::r--,other::---", true));
       client.removeAcl("/file5"); // SetAclOp -> MetadataUpdateEvent
+      client.rename("/file5", "/dir"); // RenameOldOp -> RenameEvent
 
       EventBatch batch = null;
 
@@ -343,6 +344,16 @@ public class TestDFSInotifyEventInputStream {
           Event.MetadataUpdateEvent.MetadataType.ACLS);
       Assert.assertTrue(mue8.getAcls() == null);
 
+      // RenameOp (2)
+      batch = waitForNextEvents(eis);
+      Assert.assertEquals(1, batch.getEvents().length);
+      txid = checkTxid(batch, txid);
+      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.RENAME);
+      Event.RenameEvent re3 = (Event.RenameEvent) batch.getEvents()[0];
+      Assert.assertTrue(re3.getDstPath().equals("/dir/file5"));
+      Assert.assertTrue(re3.getSrcPath().equals("/file5"));
+      Assert.assertTrue(re.getTimestamp() > 0);
+
       // Returns null when there are no further events
       Assert.assertTrue(eis.poll() == null);
 


[14/17] hadoop git commit: HDFS-7847. Modify NNThroughputBenchmark to be able to operate on a remote NameNode (Charles Lamb via Colin P. McCabe)

Posted by ji...@apache.org.
HDFS-7847. Modify NNThroughputBenchmark to be able to operate on a remote NameNode (Charles Lamb 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/ffce9a34
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ffce9a34
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ffce9a34

Branch: refs/heads/HDFS-7240
Commit: ffce9a3413277a69444fcb890460c885de56db69
Parents: e4c3b52
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Tue May 5 11:27:36 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Tue May 5 11:34:58 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  40 ++++++
 .../server/namenode/NNThroughputBenchmark.java  | 136 +++++++++++++------
 3 files changed, 137 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffce9a34/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 c89e6fe..01de9b1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -510,6 +510,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-7758. Retire FsDatasetSpi#getVolumes() and use
     FsDatasetSpi#getVolumeRefs() instead (Lei (Eddy) Xu via Colin P. McCabe)
 
+    HDFS-7847. Modify NNThroughputBenchmark to be able to operate on a remote
+    NameNode (Charles Lamb via Colin P. McCabe)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffce9a34/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 a8df991..cfee997 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
@@ -48,6 +48,7 @@ import java.lang.reflect.Modifier;
 import java.net.HttpURLConnection;
 import java.net.InetSocketAddress;
 import java.net.Socket;
+import java.net.URI;
 import java.net.URL;
 import java.net.URLConnection;
 import java.nio.ByteBuffer;
@@ -64,6 +65,7 @@ import java.util.Random;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.logging.Log;
@@ -129,12 +131,14 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
+import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
@@ -147,6 +151,7 @@ import org.apache.log4j.Level;
 import org.junit.Assume;
 import org.mockito.internal.util.reflection.Whitebox;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Charsets;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
@@ -1756,6 +1761,41 @@ public class DFSTestUtil {
   }
 
   /**
+   * Get the NamenodeProtocol RPC proxy for the NN associated with this
+   * DFSClient object
+   *
+   * @param nameNodeUri the URI of the NN to get a proxy for.
+   *
+   * @return the Namenode RPC proxy associated with this DFSClient object
+   */
+  @VisibleForTesting
+  public static NamenodeProtocol getNamenodeProtocolProxy(Configuration conf,
+      URI nameNodeUri, UserGroupInformation ugi)
+      throws IOException {
+    return NameNodeProxies.createNonHAProxy(conf,
+        NameNode.getAddress(nameNodeUri), NamenodeProtocol.class, ugi, false).
+        getProxy();
+  }
+
+  /**
+   * Get the RefreshUserMappingsProtocol RPC proxy for the NN associated with
+   * this DFSClient object
+   *
+   * @param nameNodeUri the URI of the NN to get a proxy for.
+   *
+   * @return the RefreshUserMappingsProtocol RPC proxy associated with this
+   * DFSClient object
+   */
+  @VisibleForTesting
+  public static RefreshUserMappingsProtocol getRefreshUserMappingsProtocolProxy(
+      Configuration conf, URI nameNodeUri) throws IOException {
+    final AtomicBoolean nnFallbackToSimpleAuth = new AtomicBoolean(false);
+    return NameNodeProxies.createProxy(conf,
+        nameNodeUri, RefreshUserMappingsProtocol.class,
+        nnFallbackToSimpleAuth).getProxy();
+  }
+
+  /**
    * Set the datanode dead
    */
   public static void setDatanodeDead(DatanodeInfo dn) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffce9a34/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
index db0185d..2964f9a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.EnumSet;
@@ -30,19 +31,24 @@ import com.google.common.base.Preconditions;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -53,6 +59,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
@@ -63,6 +70,8 @@ import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.Groups;
+import org.apache.hadoop.security.RefreshUserMappingsProtocol;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
@@ -96,6 +105,9 @@ import org.apache.log4j.LogManager;
  * By default the refresh is never called.</li>
  * <li>-keepResults do not clean up the name-space after execution.</li>
  * <li>-useExisting do not recreate the name-space, use existing data.</li>
+ * <li>-namenode will run the test against a namenode in another
+ * process or on another host. If you use this option, the namenode
+ * must have dfs.namenode.fs-limits.min-block-size set to 16.</li>
  * </ol>
  * 
  * The benchmark first generates inputs for each thread so that the
@@ -111,11 +123,20 @@ public class NNThroughputBenchmark implements Tool {
   private static final Log LOG = LogFactory.getLog(NNThroughputBenchmark.class);
   private static final int BLOCK_SIZE = 16;
   private static final String GENERAL_OPTIONS_USAGE = 
-    "     [-keepResults] | [-logLevel L] | [-UGCacheRefreshCount G]";
+    "     [-keepResults] | [-logLevel L] | [-UGCacheRefreshCount G] |" +
+    " [-namenode <namenode URI>]\n" +
+    "     If using -namenode, set the namenode's" +
+    "         dfs.namenode.fs-limits.min-block-size to 16.";
 
   static Configuration config;
   static NameNode nameNode;
-  static NamenodeProtocols nameNodeProto;
+  static NamenodeProtocol nameNodeProto;
+  static ClientProtocol clientProto;
+  static DatanodeProtocol dataNodeProto;
+  static RefreshUserMappingsProtocol refreshUserMappingsProto;
+  static String bpid = null;
+
+  private String namenodeUri = null; // NN URI to use, if specified
 
   NNThroughputBenchmark(Configuration conf) throws IOException {
     config = conf;
@@ -264,7 +285,7 @@ public class NNThroughputBenchmark implements Tool {
         for(StatsDaemon d : daemons)
           d.start();
       } finally {
-        while(isInPorgress()) {
+        while(isInProgress()) {
           // try {Thread.sleep(500);} catch (InterruptedException e) {}
         }
         elapsedTime = Time.now() - start;
@@ -275,7 +296,7 @@ public class NNThroughputBenchmark implements Tool {
       }
     }
 
-    private boolean isInPorgress() {
+    private boolean isInProgress() {
       for(StatsDaemon d : daemons)
         if(d.isInProgress())
           return true;
@@ -283,10 +304,10 @@ public class NNThroughputBenchmark implements Tool {
     }
 
     void cleanUp() throws IOException {
-      nameNodeProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE,
+      clientProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE,
           false);
       if(!keepResults)
-        nameNodeProto.delete(getBaseDir(), true);
+        clientProto.delete(getBaseDir(), true);
     }
 
     int getNumOpsExecuted() {
@@ -360,6 +381,12 @@ public class NNThroughputBenchmark implements Tool {
         args.remove(ugrcIndex);
       }
 
+      try {
+        namenodeUri = StringUtils.popOptionWithArgument("-namenode", args);
+      } catch (IllegalArgumentException iae) {
+        printUsage();
+      }
+
       String type = args.get(1);
       if(OP_ALL_NAME.equals(type)) {
         type = getOpName();
@@ -418,7 +445,7 @@ public class NNThroughputBenchmark implements Tool {
     void benchmarkOne() throws IOException {
       for(int idx = 0; idx < opsPerThread; idx++) {
         if((localNumOpsExecuted+1) % statsOp.ugcRefreshCount == 0)
-          nameNodeProto.refreshUserToGroupsMappings();
+          refreshUserMappingsProto.refreshUserToGroupsMappings();
         long stat = statsOp.executeOp(daemonId, idx, arg1);
         localNumOpsExecuted++;
         localCumulativeTime += stat;
@@ -484,10 +511,10 @@ public class NNThroughputBenchmark implements Tool {
     @Override
     long executeOp(int daemonId, int inputIdx, String ignore) 
     throws IOException {
-      nameNodeProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE,
+      clientProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE,
           false);
       long start = Time.now();
-      nameNodeProto.delete(BASE_DIR_NAME, true);
+      clientProto.delete(BASE_DIR_NAME, true);
       long end = Time.now();
       return end-start;
     }
@@ -553,7 +580,7 @@ public class NNThroughputBenchmark implements Tool {
     @Override
     void generateInputs(int[] opsPerThread) throws IOException {
       assert opsPerThread.length == numThreads : "Error opsPerThread.length"; 
-      nameNodeProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE,
+      clientProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE,
           false);
       // int generatedFileIdx = 0;
       LOG.info("Generate " + numOpsRequired + " intputs for " + getOpName());
@@ -588,13 +615,13 @@ public class NNThroughputBenchmark implements Tool {
     throws IOException {
       long start = Time.now();
       // dummyActionNoSynch(fileIdx);
-      nameNodeProto.create(fileNames[daemonId][inputIdx], FsPermission.getDefault(),
+      clientProto.create(fileNames[daemonId][inputIdx], FsPermission.getDefault(),
                       clientName, new EnumSetWritable<CreateFlag>(EnumSet
               .of(CreateFlag.CREATE, CreateFlag.OVERWRITE)), true, 
-          replication, BLOCK_SIZE, null);
+          replication, BLOCK_SIZE, CryptoProtocolVersion.supported());
       long end = Time.now();
       for(boolean written = !closeUponCreate; !written; 
-        written = nameNodeProto.complete(fileNames[daemonId][inputIdx],
+        written = clientProto.complete(fileNames[daemonId][inputIdx],
                                     clientName, null, HdfsConstants.GRANDFATHER_INODE_ID));
       return end-start;
     }
@@ -657,7 +684,7 @@ public class NNThroughputBenchmark implements Tool {
     @Override
     void generateInputs(int[] opsPerThread) throws IOException {
       assert opsPerThread.length == numThreads : "Error opsPerThread.length";
-      nameNodeProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE,
+      clientProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE,
           false);
       LOG.info("Generate " + numOpsRequired + " inputs for " + getOpName());
       dirPaths = new String[numThreads][];
@@ -685,7 +712,7 @@ public class NNThroughputBenchmark implements Tool {
     long executeOp(int daemonId, int inputIdx, String clientName)
         throws IOException {
       long start = Time.now();
-      nameNodeProto.mkdirs(dirPaths[daemonId][inputIdx],
+      clientProto.mkdirs(dirPaths[daemonId][inputIdx],
           FsPermission.getDefault(), true);
       long end = Time.now();
       return end-start;
@@ -757,11 +784,11 @@ public class NNThroughputBenchmark implements Tool {
       }
       // use the same files for open
       super.generateInputs(opsPerThread);
-      if(nameNodeProto.getFileInfo(opCreate.getBaseDir()) != null
-          && nameNodeProto.getFileInfo(getBaseDir()) == null) {
-        nameNodeProto.rename(opCreate.getBaseDir(), getBaseDir());
+      if(clientProto.getFileInfo(opCreate.getBaseDir()) != null
+          && clientProto.getFileInfo(getBaseDir()) == null) {
+        clientProto.rename(opCreate.getBaseDir(), getBaseDir());
       }
-      if(nameNodeProto.getFileInfo(getBaseDir()) == null) {
+      if(clientProto.getFileInfo(getBaseDir()) == null) {
         throw new IOException(getBaseDir() + " does not exist.");
       }
     }
@@ -773,7 +800,7 @@ public class NNThroughputBenchmark implements Tool {
     long executeOp(int daemonId, int inputIdx, String ignore) 
     throws IOException {
       long start = Time.now();
-      nameNodeProto.getBlockLocations(fileNames[daemonId][inputIdx], 0L, BLOCK_SIZE);
+      clientProto.getBlockLocations(fileNames[daemonId][inputIdx], 0L, BLOCK_SIZE);
       long end = Time.now();
       return end-start;
     }
@@ -803,7 +830,7 @@ public class NNThroughputBenchmark implements Tool {
     long executeOp(int daemonId, int inputIdx, String ignore) 
     throws IOException {
       long start = Time.now();
-      nameNodeProto.delete(fileNames[daemonId][inputIdx], false);
+      clientProto.delete(fileNames[daemonId][inputIdx], false);
       long end = Time.now();
       return end-start;
     }
@@ -833,7 +860,7 @@ public class NNThroughputBenchmark implements Tool {
     long executeOp(int daemonId, int inputIdx, String ignore) 
     throws IOException {
       long start = Time.now();
-      nameNodeProto.getFileInfo(fileNames[daemonId][inputIdx]);
+      clientProto.getFileInfo(fileNames[daemonId][inputIdx]);
       long end = Time.now();
       return end-start;
     }
@@ -877,7 +904,7 @@ public class NNThroughputBenchmark implements Tool {
     long executeOp(int daemonId, int inputIdx, String ignore) 
     throws IOException {
       long start = Time.now();
-      nameNodeProto.rename(fileNames[daemonId][inputIdx],
+      clientProto.rename(fileNames[daemonId][inputIdx],
                       destNames[daemonId][inputIdx]);
       long end = Time.now();
       return end-start;
@@ -933,14 +960,14 @@ public class NNThroughputBenchmark implements Tool {
           new DataStorage(nsInfo),
           new ExportedBlockKeys(), VersionInfo.getVersion());
       // register datanode
-      dnRegistration = nameNodeProto.registerDatanode(dnRegistration);
+      dnRegistration = dataNodeProto.registerDatanode(dnRegistration);
+      dnRegistration.setNamespaceInfo(nsInfo);
       //first block reports
       storage = new DatanodeStorage(DatanodeStorage.generateUuid());
       final StorageBlockReport[] reports = {
           new StorageBlockReport(storage, BlockListAsLongs.EMPTY)
       };
-      nameNodeProto.blockReport(dnRegistration, 
-          nameNode.getNamesystem().getBlockPoolId(), reports,
+      dataNodeProto.blockReport(dnRegistration, bpid, reports,
               new BlockReportContext(1, 0, System.nanoTime()));
     }
 
@@ -953,7 +980,7 @@ public class NNThroughputBenchmark implements Tool {
       // TODO:FEDERATION currently a single block pool is supported
       StorageReport[] rep = { new StorageReport(storage, false,
           DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED) };
-      DatanodeCommand[] cmds = nameNodeProto.sendHeartbeat(dnRegistration, rep,
+      DatanodeCommand[] cmds = dataNodeProto.sendHeartbeat(dnRegistration, rep,
           0L, 0L, 0, 0, 0, null).getCommands();
       if(cmds != null) {
         for (DatanodeCommand cmd : cmds ) {
@@ -1002,7 +1029,7 @@ public class NNThroughputBenchmark implements Tool {
       // register datanode
       StorageReport[] rep = { new StorageReport(storage,
           false, DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED) };
-      DatanodeCommand[] cmds = nameNodeProto.sendHeartbeat(dnRegistration,
+      DatanodeCommand[] cmds = dataNodeProto.sendHeartbeat(dnRegistration,
           rep, 0L, 0L, 0, 0, 0, null).getCommands();
       if (cmds != null) {
         for (DatanodeCommand cmd : cmds) {
@@ -1041,8 +1068,7 @@ public class NNThroughputBenchmark implements Tool {
                   null) };
           StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
               targetStorageID, rdBlocks) };
-          nameNodeProto.blockReceivedAndDeleted(receivedDNReg, nameNode
-              .getNamesystem().getBlockPoolId(), report);
+          dataNodeProto.blockReceivedAndDeleted(receivedDNReg, bpid, report);
         }
       }
       return blocks.length;
@@ -1133,15 +1159,15 @@ public class NNThroughputBenchmark implements Tool {
       FileNameGenerator nameGenerator;
       nameGenerator = new FileNameGenerator(getBaseDir(), 100);
       String clientName = getClientName(007);
-      nameNodeProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE,
+      clientProto.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE,
           false);
       for(int idx=0; idx < nrFiles; idx++) {
         String fileName = nameGenerator.getNextFileName("ThroughputBench");
-        nameNodeProto.create(fileName, FsPermission.getDefault(), clientName,
+        clientProto.create(fileName, FsPermission.getDefault(), clientName,
             new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)), true, replication,
-            BLOCK_SIZE, null);
+            BLOCK_SIZE, CryptoProtocolVersion.supported());
         ExtendedBlock lastBlock = addBlocks(fileName, clientName);
-        nameNodeProto.complete(fileName, clientName, lastBlock, HdfsConstants.GRANDFATHER_INODE_ID);
+        clientProto.complete(fileName, clientName, lastBlock, HdfsConstants.GRANDFATHER_INODE_ID);
       }
       // prepare block reports
       for(int idx=0; idx < nrDatanodes; idx++) {
@@ -1153,7 +1179,7 @@ public class NNThroughputBenchmark implements Tool {
     throws IOException {
       ExtendedBlock prevBlock = null;
       for(int jdx = 0; jdx < blocksPerFile; jdx++) {
-        LocatedBlock loc = nameNodeProto.addBlock(fileName, clientName,
+        LocatedBlock loc = clientProto.addBlock(fileName, clientName,
             prevBlock, null, HdfsConstants.GRANDFATHER_INODE_ID, null);
         prevBlock = loc.getBlock();
         for(DatanodeInfo dnInfo : loc.getLocations()) {
@@ -1164,8 +1190,8 @@ public class NNThroughputBenchmark implements Tool {
               ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, null) };
           StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
               datanodes[dnIdx].storage.getStorageID(), rdBlocks) };
-          nameNodeProto.blockReceivedAndDeleted(datanodes[dnIdx].dnRegistration, loc
-              .getBlock().getBlockPoolId(), report);
+          dataNodeProto.blockReceivedAndDeleted(datanodes[dnIdx].dnRegistration,
+              bpid, report);
         }
       }
       return prevBlock;
@@ -1186,8 +1212,7 @@ public class NNThroughputBenchmark implements Tool {
       long start = Time.now();
       StorageBlockReport[] report = { new StorageBlockReport(
           dn.storage, dn.getBlockReportList()) };
-      nameNodeProto.blockReport(dn.dnRegistration,
-          nameNode.getNamesystem().getBlockPoolId(), report,
+      dataNodeProto.blockReport(dn.dnRegistration, bpid, report,
           new BlockReportContext(1, 0, System.nanoTime()));
       long end = Time.now();
       return end-start;
@@ -1318,7 +1343,7 @@ public class NNThroughputBenchmark implements Tool {
         LOG.info("Datanode " + dn + " is decommissioned.");
       }
       excludeFile.close();
-      nameNodeProto.refreshNodes();
+      clientProto.refreshNodes();
     }
 
     /**
@@ -1414,8 +1439,6 @@ public class NNThroughputBenchmark implements Tool {
 
     // Start the NameNode
     String[] argv = new String[] {};
-    nameNode = NameNode.createNameNode(argv, config);
-    nameNodeProto = nameNode.getRpcServer();
 
     List<OperationStatsBase> ops = new ArrayList<OperationStatsBase>();
     OperationStatsBase opStat = null;
@@ -1456,6 +1479,29 @@ public class NNThroughputBenchmark implements Tool {
         opStat = new CleanAllStats(args);
         ops.add(opStat);
       }
+
+      if (namenodeUri == null) {
+        nameNode = NameNode.createNameNode(argv, config);
+        NamenodeProtocols nnProtos = nameNode.getRpcServer();
+        nameNodeProto = nnProtos;
+        clientProto = nnProtos;
+        dataNodeProto = nnProtos;
+        refreshUserMappingsProto = nnProtos;
+        bpid = nameNode.getNamesystem().getBlockPoolId();
+      } else {
+        FileSystem.setDefaultUri(getConf(), namenodeUri);
+        DistributedFileSystem dfs = (DistributedFileSystem)
+            FileSystem.get(getConf());
+        final URI nnUri = new URI(namenodeUri);
+        nameNodeProto = DFSTestUtil.getNamenodeProtocolProxy(config, nnUri,
+            UserGroupInformation.getCurrentUser());
+        clientProto = dfs.getClient().getNamenode();
+        dataNodeProto = new DatanodeProtocolClientSideTranslatorPB(
+            NameNode.getAddress(nnUri), config);
+        refreshUserMappingsProto =
+            DFSTestUtil.getRefreshUserMappingsProtocolProxy(config, nnUri);
+        getBlockPoolId(dfs);
+      }
       if(ops.size() == 0)
         printUsage();
       // run each benchmark
@@ -1476,6 +1522,12 @@ public class NNThroughputBenchmark implements Tool {
     return 0;
   }
 
+  private void getBlockPoolId(DistributedFileSystem unused)
+    throws IOException {
+    final NamespaceInfo nsInfo = nameNodeProto.versionRequest();
+    bpid = nsInfo.getBlockPoolID();
+  }
+
   public static void main(String[] args) throws Exception {
     NNThroughputBenchmark bench = null;
     try {


[17/17] hadoop git commit: HDFS-8314. Move HdfsServerConstants#IO_FILE_BUFFER_SIZE and SMALL_BUFFER_SIZE to the users. Contributed by Li Lu.

Posted by ji...@apache.org.
HDFS-8314. Move HdfsServerConstants#IO_FILE_BUFFER_SIZE and SMALL_BUFFER_SIZE to the users. 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/4da8490b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4da8490b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4da8490b

Branch: refs/heads/HDFS-7240
Commit: 4da8490b512a33a255ed27309860859388d7c168
Parents: 9809a16
Author: Haohui Mai <wh...@apache.org>
Authored: Tue May 5 15:41:22 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue May 5 15:41:22 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  6 +++--
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    | 11 +++++++++
 .../org/apache/hadoop/hdfs/DataStreamer.java    |  7 +++---
 .../hadoop/hdfs/server/balancer/Dispatcher.java |  7 ++++--
 .../hdfs/server/common/HdfsServerConstants.java |  8 ------
 .../server/datanode/BlockMetadataHeader.java    |  7 ++++--
 .../hdfs/server/datanode/BlockReceiver.java     |  5 ++--
 .../hdfs/server/datanode/BlockSender.java       | 14 ++++++++---
 .../hadoop/hdfs/server/datanode/DataNode.java   |  2 +-
 .../hdfs/server/datanode/DataXceiver.java       | 26 ++++++++++++--------
 .../datanode/fsdataset/impl/BlockPoolSlice.java |  7 ++++--
 .../datanode/fsdataset/impl/FsDatasetImpl.java  | 26 ++++++++++++--------
 .../impl/RamDiskAsyncLazyPersistService.java    |  8 ++++--
 .../hdfs/server/namenode/TransferFsImage.java   | 10 +++++---
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  2 +-
 16 files changed, 95 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4da8490b/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 cc6758f..92a82c8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -513,6 +513,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-7847. Modify NNThroughputBenchmark to be able to operate on a remote
     NameNode (Charles Lamb via Colin P. McCabe)
 
+    HDFS-8314. Move HdfsServerConstants#IO_FILE_BUFFER_SIZE and
+    SMALL_BUFFER_SIZE to the users. (Li Lu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4da8490b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 99b8d2c..c145959 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -238,6 +238,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       new DFSHedgedReadMetrics();
   private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL;
   private final Sampler<?> traceSampler;
+  private final int smallBufferSize;
 
   public DfsClientConf getConf() {
     return dfsClientConf;
@@ -309,6 +310,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     this.stats = stats;
     this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
     this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
+    this.smallBufferSize = DFSUtil.getSmallBufferSize(conf);
 
     this.ugi = UserGroupInformation.getCurrentUser();
     
@@ -1902,7 +1904,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
           //connect to a datanode
           IOStreamPair pair = connectToDN(datanodes[j], timeout, lb);
           out = new DataOutputStream(new BufferedOutputStream(pair.out,
-              HdfsServerConstants.SMALL_BUFFER_SIZE));
+              smallBufferSize));
           in = new DataInputStream(pair.in);
 
           if (LOG.isDebugEnabled()) {
@@ -2067,7 +2069,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
 
     try {
       DataOutputStream out = new DataOutputStream(new BufferedOutputStream(pair.out,
-          HdfsServerConstants.SMALL_BUFFER_SIZE));
+          smallBufferSize));
       DataInputStream in = new DataInputStream(pair.in);
   
       new Sender(out).readBlock(lb.getBlock(), lb.getBlockToken(), clientName,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4da8490b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index 42ff7fa..a925a60 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -70,6 +70,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
 import org.apache.hadoop.crypto.key.KeyProviderFactory;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
@@ -1514,4 +1515,14 @@ public class DFSUtil {
         .createKeyProviderCryptoExtension(keyProvider);
     return cryptoProvider;
   }
+
+  public static int getIoFileBufferSize(Configuration conf) {
+    return conf.getInt(
+      CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
+      CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
+  }
+
+  public static int getSmallBufferSize(Configuration conf) {
+    return Math.min(getIoFileBufferSize(conf) / 2, 512);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4da8490b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index 43787ab..96bf212 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -71,7 +71,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.util.ByteArrayManager;
@@ -92,7 +91,6 @@ import org.apache.htrace.Trace;
 import org.apache.htrace.TraceInfo;
 import org.apache.htrace.TraceScope;
 
-import com.google.common.base.Preconditions;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
@@ -123,6 +121,7 @@ import com.google.common.cache.RemovalNotification;
 @InterfaceAudience.Private
 class DataStreamer extends Daemon {
   static final Log LOG = LogFactory.getLog(DataStreamer.class);
+
   /**
    * Create a socket for a write pipeline
    *
@@ -1145,7 +1144,7 @@ class DataStreamer extends Daemon {
       unbufOut = saslStreams.out;
       unbufIn = saslStreams.in;
       out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-          HdfsServerConstants.SMALL_BUFFER_SIZE));
+          DFSUtil.getSmallBufferSize(dfsClient.getConfiguration())));
       in = new DataInputStream(unbufIn);
 
       //send the TRANSFER_BLOCK request
@@ -1425,7 +1424,7 @@ class DataStreamer extends Daemon {
         unbufOut = saslStreams.out;
         unbufIn = saslStreams.in;
         out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-            HdfsServerConstants.SMALL_BUFFER_SIZE));
+            DFSUtil.getSmallBufferSize(dfsClient.getConfiguration())));
         blockReplyStream = new DataInputStream(unbufIn);
 
         //

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4da8490b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
index ba16905..4a8f40f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
@@ -118,6 +118,8 @@ public class Dispatcher {
   /** The maximum number of concurrent blocks moves at a datanode */
   private final int maxConcurrentMovesPerNode;
 
+  private final int ioFileBufferSize;
+
   private static class GlobalBlockMap {
     private final Map<Block, DBlock> map = new HashMap<Block, DBlock>();
 
@@ -308,9 +310,9 @@ public class Dispatcher {
         unbufOut = saslStreams.out;
         unbufIn = saslStreams.in;
         out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-            HdfsServerConstants.IO_FILE_BUFFER_SIZE));
+            ioFileBufferSize));
         in = new DataInputStream(new BufferedInputStream(unbufIn,
-            HdfsServerConstants.IO_FILE_BUFFER_SIZE));
+            ioFileBufferSize));
 
         sendRequest(out, eb, accessToken);
         receiveResponse(in);
@@ -801,6 +803,7 @@ public class Dispatcher {
     this.saslClient = new SaslDataTransferClient(conf,
         DataTransferSaslUtil.getSaslPropertiesResolver(conf),
         TrustedChannelResolver.getInstance(conf), nnc.fallbackToSimpleAuth);
+    this.ioFileBufferSize = DFSUtil.getIoFileBufferSize(conf);
   }
 
   public DistributedFileSystem getDistributedFileSystem() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4da8490b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
index 47b0818..31af6c8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
@@ -24,9 +24,7 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeLayoutVersion;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
@@ -56,12 +54,6 @@ public interface HdfsServerConstants {
   // to 1k.
   int MAX_PATH_LENGTH = 8000;
   int MAX_PATH_DEPTH = 1000;
-  int IO_FILE_BUFFER_SIZE = new HdfsConfiguration().getInt(
-      CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
-      CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
-  // Used for writing header etc.
-  int SMALL_BUFFER_SIZE = Math.min(IO_FILE_BUFFER_SIZE / 2,
-      512);
   // An invalid transaction ID that will never be seen in a real namesystem.
   long INVALID_TXID = -12345;
   // Number of generation stamps reserved for legacy blocks.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4da8490b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
index 04700b8..4977fd7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
@@ -33,7 +33,8 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.DataChecksum;
 
@@ -60,6 +61,8 @@ public class BlockMetadataHeader {
    */
   private final short version;
   private DataChecksum checksum = null;
+
+  private static final HdfsConfiguration conf = new HdfsConfiguration();
     
   @VisibleForTesting
   public BlockMetadataHeader(short version, DataChecksum checksum) {
@@ -85,7 +88,7 @@ public class BlockMetadataHeader {
     DataInputStream in = null;
     try {
       in = new DataInputStream(new BufferedInputStream(
-        new FileInputStream(metaFile), HdfsServerConstants.IO_FILE_BUFFER_SIZE));
+        new FileInputStream(metaFile), DFSUtil.getIoFileBufferSize(conf)));
       return readDataChecksum(in, metaFile);
     } finally {
       IOUtils.closeStream(in);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4da8490b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
index 10692d4..90e2194 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
@@ -39,6 +39,7 @@ import org.apache.commons.logging.Log;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.FSOutputSummer;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
@@ -47,7 +48,6 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -246,7 +246,8 @@ class BlockReceiver implements Closeable {
             out.getClass());
       }
       this.checksumOut = new DataOutputStream(new BufferedOutputStream(
-          streams.getChecksumOut(), HdfsServerConstants.SMALL_BUFFER_SIZE));
+          streams.getChecksumOut(), DFSUtil.getSmallBufferSize(
+          datanode.getConf())));
       // write data chunk header if creating a new replica
       if (isCreate) {
         BlockMetadataHeader.writeHeader(checksumOut, diskChecksum);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4da8490b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
index 6b958a2..79f4dd7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
@@ -34,9 +34,10 @@ import java.util.Arrays;
 
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
@@ -104,8 +105,13 @@ class BlockSender implements java.io.Closeable {
    * not sure if there will be much more improvement.
    */
   private static final int MIN_BUFFER_WITH_TRANSFERTO = 64*1024;
+  private static final int IO_FILE_BUFFER_SIZE;
+  static {
+    HdfsConfiguration conf = new HdfsConfiguration();
+    IO_FILE_BUFFER_SIZE = DFSUtil.getIoFileBufferSize(conf);
+  }
   private static final int TRANSFERTO_BUFFER_SIZE = Math.max(
-      HdfsServerConstants.IO_FILE_BUFFER_SIZE, MIN_BUFFER_WITH_TRANSFERTO);
+      IO_FILE_BUFFER_SIZE, MIN_BUFFER_WITH_TRANSFERTO);
   
   /** the block to read from */
   private final ExtendedBlock block;
@@ -298,7 +304,7 @@ class BlockSender implements java.io.Closeable {
             // storage and computes the checksum.
             if (metaIn.getLength() > BlockMetadataHeader.getHeaderSize()) {
               checksumIn = new DataInputStream(new BufferedInputStream(
-                  metaIn, HdfsServerConstants.IO_FILE_BUFFER_SIZE));
+                  metaIn, IO_FILE_BUFFER_SIZE));
   
               csum = BlockMetadataHeader.readDataChecksum(checksumIn, block);
               keepMetaInOpen = true;
@@ -747,7 +753,7 @@ class BlockSender implements java.io.Closeable {
         pktBufSize += checksumSize * maxChunksPerPacket;
       } else {
         maxChunksPerPacket = Math.max(1,
-            numberOfChunks(HdfsServerConstants.IO_FILE_BUFFER_SIZE));
+            numberOfChunks(IO_FILE_BUFFER_SIZE));
         // Packet size includes both checksum and data
         pktBufSize += (chunkSize + checksumSize) * maxChunksPerPacket;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4da8490b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 22b1520..d2b2939 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -2156,7 +2156,7 @@ public class DataNode extends ReconfigurableBase
         unbufIn = saslStreams.in;
         
         out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-            HdfsServerConstants.SMALL_BUFFER_SIZE));
+            DFSUtil.getSmallBufferSize(conf)));
         in = new DataInputStream(unbufIn);
         blockSender = new BlockSender(b, 0, b.getNumBytes(), 
             false, false, true, DataNode.this, null, cachingStrategy);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4da8490b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index de25579..26d669c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -48,7 +48,9 @@ import java.util.Arrays;
 
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.ExtendedBlockId;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -109,7 +111,9 @@ class DataXceiver extends Receiver implements Runnable {
   private final InputStream socketIn;
   private OutputStream socketOut;
   private BlockReceiver blockReceiver = null;
-  
+  private final int ioFileBufferSize;
+  private final int smallBufferSize;
+
   /**
    * Client Name used in previous operation. Not available on first request
    * on the socket.
@@ -131,6 +135,8 @@ class DataXceiver extends Receiver implements Runnable {
     this.datanode = datanode;
     this.dataXceiverServer = dataXceiverServer;
     this.connectToDnViaHostname = datanode.getDnConf().connectToDnViaHostname;
+    this.ioFileBufferSize = DFSUtil.getIoFileBufferSize(datanode.getConf());
+    this.smallBufferSize = DFSUtil.getSmallBufferSize(datanode.getConf());
     remoteAddress = peer.getRemoteAddressString();
     final int colonIdx = remoteAddress.indexOf(':');
     remoteAddressWithoutPort =
@@ -191,7 +197,7 @@ class DataXceiver extends Receiver implements Runnable {
           socketIn, datanode.getXferAddress().getPort(),
           datanode.getDatanodeId());
         input = new BufferedInputStream(saslStreams.in,
-          HdfsServerConstants.SMALL_BUFFER_SIZE);
+            smallBufferSize);
         socketOut = saslStreams.out;
       } catch (InvalidMagicNumberException imne) {
         if (imne.isHandshake4Encryption()) {
@@ -514,7 +520,7 @@ class DataXceiver extends Receiver implements Runnable {
     long read = 0;
     OutputStream baseStream = getOutputStream();
     DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
-        baseStream, HdfsServerConstants.SMALL_BUFFER_SIZE));
+        baseStream, smallBufferSize));
     checkAccess(out, true, block, blockToken,
         Op.READ_BLOCK, BlockTokenIdentifier.AccessMode.READ);
   
@@ -658,7 +664,7 @@ class DataXceiver extends Receiver implements Runnable {
     final DataOutputStream replyOut = new DataOutputStream(
         new BufferedOutputStream(
             getOutputStream(),
-            HdfsServerConstants.SMALL_BUFFER_SIZE));
+            smallBufferSize));
     checkAccess(replyOut, isClient, block, blockToken,
         Op.WRITE_BLOCK, BlockTokenIdentifier.AccessMode.WRITE);
 
@@ -717,7 +723,7 @@ class DataXceiver extends Receiver implements Runnable {
           unbufMirrorOut = saslStreams.out;
           unbufMirrorIn = saslStreams.in;
           mirrorOut = new DataOutputStream(new BufferedOutputStream(unbufMirrorOut,
-              HdfsServerConstants.SMALL_BUFFER_SIZE));
+              smallBufferSize));
           mirrorIn = new DataInputStream(unbufMirrorIn);
 
           // Do not propagate allowLazyPersist to downstream DataNodes.
@@ -932,7 +938,7 @@ class DataXceiver extends Receiver implements Runnable {
         .getMetaDataInputStream(block);
     
     final DataInputStream checksumIn = new DataInputStream(
-        new BufferedInputStream(metadataIn, HdfsServerConstants.IO_FILE_BUFFER_SIZE));
+        new BufferedInputStream(metadataIn, ioFileBufferSize));
     updateCurrentThreadName("Getting checksum for block " + block);
     try {
       //read metadata file
@@ -1024,7 +1030,7 @@ class DataXceiver extends Receiver implements Runnable {
       // set up response stream
       OutputStream baseStream = getOutputStream();
       reply = new DataOutputStream(new BufferedOutputStream(
-          baseStream, HdfsServerConstants.SMALL_BUFFER_SIZE));
+          baseStream, smallBufferSize));
 
       // send status first
       writeSuccessWithChecksumInfo(blockSender, reply);
@@ -1131,10 +1137,10 @@ class DataXceiver extends Receiver implements Runnable {
         unbufProxyOut = saslStreams.out;
         unbufProxyIn = saslStreams.in;
         
-        proxyOut = new DataOutputStream(new BufferedOutputStream(unbufProxyOut, 
-            HdfsServerConstants.SMALL_BUFFER_SIZE));
+        proxyOut = new DataOutputStream(new BufferedOutputStream(unbufProxyOut,
+            smallBufferSize));
         proxyReply = new DataInputStream(new BufferedInputStream(unbufProxyIn,
-            HdfsServerConstants.IO_FILE_BUFFER_SIZE));
+            ioFileBufferSize));
         
         /* send request to the proxy */
         IoeDuringCopyBlockOperation = true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4da8490b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
index 414d078..a47d564 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
@@ -38,10 +38,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DU;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
@@ -76,6 +76,7 @@ class BlockPoolSlice {
   private final File lazypersistDir;
   private final File rbwDir; // directory store RBW replica
   private final File tmpDir; // directory store Temporary replica
+  private final int ioFileBufferSize;
   private static final String DU_CACHE_FILE = "dfsUsed";
   private volatile boolean dfsUsedSaved = false;
   private static final int SHUTDOWN_HOOK_PRIORITY = 30;
@@ -108,6 +109,8 @@ class BlockPoolSlice {
       }
     }
 
+    this.ioFileBufferSize = DFSUtil.getIoFileBufferSize(conf);
+
     this.deleteDuplicateReplicas = conf.getBoolean(
         DFSConfigKeys.DFS_DATANODE_DUPLICATE_REPLICA_DELETION,
         DFSConfigKeys.DFS_DATANODE_DUPLICATE_REPLICA_DELETION_DEFAULT);
@@ -612,7 +615,7 @@ class BlockPoolSlice {
       }
       checksumIn = new DataInputStream(
           new BufferedInputStream(new FileInputStream(metaFile),
-              HdfsServerConstants.IO_FILE_BUFFER_SIZE));
+              ioFileBufferSize));
 
       // read and handle the common header here. For now just a version
       final DataChecksum checksum = BlockMetadataHeader.readDataChecksum(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4da8490b/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 31ac414..9c0d86d 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
@@ -58,7 +58,9 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.ExtendedBlockId;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
@@ -66,7 +68,6 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
@@ -247,6 +248,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
   private static final int MAX_BLOCK_EVICTIONS_PER_ITERATION = 3;
 
+  private final int smallBufferSize;
 
   // Used for synchronizing access to usage stats
   private final Object statsLock = new Object();
@@ -264,6 +266,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     this.datanode = datanode;
     this.dataStorage = storage;
     this.conf = conf;
+    this.smallBufferSize = DFSUtil.getSmallBufferSize(conf);
     // The number of volumes required for operation is the total number 
     // of volumes minus the number of failed volumes we can tolerate.
     final int volFailuresTolerated =
@@ -837,19 +840,21 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
    * @throws IOException
    */
   static File[] copyBlockFiles(long blockId, long genStamp, File srcMeta,
-      File srcFile, File destRoot, boolean calculateChecksum)
-      throws IOException {
+      File srcFile, File destRoot, boolean calculateChecksum,
+      int smallBufferSize) throws IOException {
     final File destDir = DatanodeUtil.idToBlockDir(destRoot, blockId);
     final File dstFile = new File(destDir, srcFile.getName());
     final File dstMeta = FsDatasetUtil.getMetaFile(dstFile, genStamp);
-    return copyBlockFiles(srcMeta, srcFile, dstMeta, dstFile, calculateChecksum);
+    return copyBlockFiles(srcMeta, srcFile, dstMeta, dstFile, calculateChecksum,
+        smallBufferSize);
   }
 
   static File[] copyBlockFiles(File srcMeta, File srcFile, File dstMeta,
-                               File dstFile, boolean calculateChecksum)
+                               File dstFile, boolean calculateChecksum,
+                               int smallBufferSize)
       throws IOException {
     if (calculateChecksum) {
-      computeChecksum(srcMeta, dstMeta, srcFile);
+      computeChecksum(srcMeta, dstMeta, srcFile, smallBufferSize);
     } else {
       try {
         Storage.nativeCopyFileUnbuffered(srcMeta, dstMeta, true);
@@ -913,7 +918,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       File[] blockFiles = copyBlockFiles(block.getBlockId(),
           block.getGenerationStamp(), oldMetaFile, oldBlockFile,
           targetVolume.getTmpDir(block.getBlockPoolId()),
-          replicaInfo.isOnTransientStorage());
+          replicaInfo.isOnTransientStorage(), smallBufferSize);
 
       ReplicaInfo newReplicaInfo = new ReplicaInPipeline(
           replicaInfo.getBlockId(), replicaInfo.getGenerationStamp(),
@@ -941,7 +946,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
    * @param blockFile block file for which the checksum will be computed
    * @throws IOException
    */
-  private static void computeChecksum(File srcMeta, File dstMeta, File blockFile)
+  private static void computeChecksum(File srcMeta, File dstMeta,
+      File blockFile, int smallBufferSize)
       throws IOException {
     final DataChecksum checksum = BlockMetadataHeader.readDataChecksum(srcMeta);
     final byte[] data = new byte[1 << 16];
@@ -957,7 +963,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
         }
       }
       metaOut = new DataOutputStream(new BufferedOutputStream(
-          new FileOutputStream(dstMeta), HdfsServerConstants.SMALL_BUFFER_SIZE));
+          new FileOutputStream(dstMeta), smallBufferSize));
       BlockMetadataHeader.writeHeader(metaOut, checksum);
 
       int offset = 0;
@@ -2480,7 +2486,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       final File dstMetaFile = FsDatasetUtil.getMetaFile(dstBlockFile, newGS);
       return copyBlockFiles(replicaInfo.getMetaFile(),
           replicaInfo.getBlockFile(),
-          dstMetaFile, dstBlockFile, true);
+          dstMetaFile, dstBlockFile, true, smallBufferSize);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4da8490b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java
index cf8de0a..884df2e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java
@@ -20,10 +20,11 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 
-import javax.ws.rs.HEAD;
 import java.io.File;
 import java.io.IOException;
 import java.util.HashMap;
@@ -55,6 +56,7 @@ class RamDiskAsyncLazyPersistService {
   private final ThreadGroup threadGroup;
   private Map<File, ThreadPoolExecutor> executors
       = new HashMap<File, ThreadPoolExecutor>();
+  private final static HdfsConfiguration EMPTY_HDFS_CONF = new HdfsConfiguration();
 
   /**
    * Create a RamDiskAsyncLazyPersistService with a set of volumes (specified by their
@@ -234,9 +236,11 @@ class RamDiskAsyncLazyPersistService {
       boolean succeeded = false;
       final FsDatasetImpl dataset = (FsDatasetImpl)datanode.getFSDataset();
       try (FsVolumeReference ref = this.targetVolume) {
+        int smallBufferSize = DFSUtil.getSmallBufferSize(EMPTY_HDFS_CONF);
         // No FsDatasetImpl lock for the file copy
         File targetFiles[] = FsDatasetImpl.copyBlockFiles(
-            blockId, genStamp, metaFile, blockFile, lazyPersistDir, true);
+            blockId, genStamp, metaFile, blockFile, lazyPersistDir, true,
+            smallBufferSize);
 
         // Lock FsDataSetImpl during onCompleteLazyPersist callback
         dataset.onCompleteLazyPersist(bpId, blockId,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4da8490b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
index 041c3cb..9783cca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
@@ -43,8 +43,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
@@ -77,6 +77,7 @@ public class TransferFsImage {
 
   private final static String CONTENT_TYPE = "Content-Type";
   private final static String CONTENT_TRANSFER_ENCODING = "Content-Transfer-Encoding";
+  private final static int IO_FILE_BUFFER_SIZE;
 
   @VisibleForTesting
   static int timeout = 0;
@@ -88,6 +89,7 @@ public class TransferFsImage {
     connectionFactory = URLConnectionFactory
         .newDefaultURLConnectionFactory(conf);
     isSpnegoEnabled = UserGroupInformation.isSecurityEnabled();
+    IO_FILE_BUFFER_SIZE = DFSUtil.getIoFileBufferSize(conf);
   }
 
   private static final Log LOG = LogFactory.getLog(TransferFsImage.class);
@@ -336,7 +338,7 @@ public class TransferFsImage {
   private static void copyFileToStream(OutputStream out, File localfile,
       FileInputStream infile, DataTransferThrottler throttler,
       Canceler canceler) throws IOException {
-    byte buf[] = new byte[HdfsServerConstants.IO_FILE_BUFFER_SIZE];
+    byte buf[] = new byte[IO_FILE_BUFFER_SIZE];
     try {
       CheckpointFaultInjector.getInstance()
           .aboutToSendFile(localfile);
@@ -345,7 +347,7 @@ public class TransferFsImage {
             shouldSendShortFile(localfile)) {
           // Test sending image shorter than localfile
           long len = localfile.length();
-          buf = new byte[(int)Math.min(len/2, HdfsServerConstants.IO_FILE_BUFFER_SIZE)];
+          buf = new byte[(int)Math.min(len/2, IO_FILE_BUFFER_SIZE)];
           // This will read at most half of the image
           // and the rest of the image will be sent over the wire
           infile.read(buf);
@@ -510,7 +512,7 @@ public class TransferFsImage {
       }
       
       int num = 1;
-      byte[] buf = new byte[HdfsServerConstants.IO_FILE_BUFFER_SIZE];
+      byte[] buf = new byte[IO_FILE_BUFFER_SIZE];
       while (num > 0) {
         num = stream.read(buf);
         if (num > 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4da8490b/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 cfee997..a88a459 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
@@ -956,7 +956,7 @@ public class DFSTestUtil {
     final long writeTimeout = dfsClient.getDatanodeWriteTimeout(datanodes.length);
     final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
         NetUtils.getOutputStream(s, writeTimeout),
-        HdfsServerConstants.SMALL_BUFFER_SIZE));
+        DFSUtil.getSmallBufferSize(dfsClient.getConfiguration())));
     final DataInputStream in = new DataInputStream(NetUtils.getInputStream(s));
 
     // send the request


[04/17] hadoop git commit: YARN-3552. RM Web UI shows -1 running containers for completed apps. Contributed by Rohith

Posted by ji...@apache.org.
YARN-3552. RM Web UI shows -1 running containers for completed apps. Contributed by Rohith


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

Branch: refs/heads/HDFS-7240
Commit: 9356cf8676fd18f78655e8a6f2e6c946997dbd40
Parents: 318081c
Author: Jason Lowe <jl...@apache.org>
Authored: Tue May 5 13:42:32 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Tue May 5 13:42:32 2015 +0000

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 3 +++
 .../server/resourcemanager/webapp/FairSchedulerAppsBlock.java     | 3 ++-
 .../hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java    | 3 ++-
 3 files changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9356cf86/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 287205d..97d3208 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -305,6 +305,9 @@ Release 2.8.0 - UNRELEASED
     YARN-2725. Added test cases of retrying creating znode in ZKRMStateStore.
     (Tsuyoshi Ozawa via jianhe)
 
+    YARN-3552. RM Web UI shows -1 running containers for completed apps
+    (Rohith via jlowe)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9356cf86/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
index 9650dfe..8f8ece0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
@@ -133,7 +133,8 @@ public class FairSchedulerAppsBlock extends HtmlBlock {
       .append(appInfo.getFinishTime()).append("\",\"")
       .append(appInfo.getState()).append("\",\"")
       .append(appInfo.getFinalStatus()).append("\",\"")
-      .append(appInfo.getRunningContainers()).append("\",\"")
+      .append(appInfo.getRunningContainers() == -1 ? "N/A" : String
+         .valueOf(appInfo.getRunningContainers())).append("\",\"")
       // Progress bar
       .append("<br title='").append(percent)
       .append("'> <div class='").append(C_PROGRESSBAR).append("' title='")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9356cf86/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
index 715d9d9..3054150 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
@@ -110,7 +110,8 @@ public class RMAppsBlock extends AppsBlock {
         .append("\",\"")
         .append(app.getFinalAppStatus())
         .append("\",\"")
-        .append(String.valueOf(app.getRunningContainers()))
+        .append(app.getRunningContainers() == -1 ? "N/A" : String
+            .valueOf(app.getRunningContainers()))
         .append("\",\"")
         // Progress bar
         .append("<br title='").append(percent).append("'> <div class='")


[10/17] hadoop git commit: HADOOP-11904. test-patch.sh goes into an infinite loop on non-maven builds (aw)

Posted by ji...@apache.org.
HADOOP-11904. test-patch.sh goes into an infinite loop on non-maven builds (aw)


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

Branch: refs/heads/HDFS-7240
Commit: 3ff91e9e9302d94b0d18cccebd02d3815c06ce90
Parents: fcd4cb7
Author: Allen Wittenauer <aw...@apache.org>
Authored: Tue May 5 11:02:15 2015 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Tue May 5 11:02:15 2015 -0700

----------------------------------------------------------------------
 dev-support/test-patch.sh                       | 8 +++++++-
 hadoop-common-project/hadoop-common/CHANGES.txt | 3 +++
 2 files changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ff91e9e/dev-support/test-patch.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.sh b/dev-support/test-patch.sh
index 5107718..9f48c64 100755
--- a/dev-support/test-patch.sh
+++ b/dev-support/test-patch.sh
@@ -915,6 +915,13 @@ function git_checkout
   fi
 
   add_jira_footer "git revision" "${PATCH_BRANCH} / ${GIT_REVISION}"
+
+  if [[ ! -f ${BASEDIR}/pom.xml ]]; then
+    hadoop_error "ERROR: This verison of test-patch.sh only supports Maven-based builds. Aborting."
+    add_jira_table -1 pre-patch "Unsupported build system."
+    output_to_jira 1
+    cleanup_and_exit 1
+  fi
   return 0
 }
 
@@ -1331,7 +1338,6 @@ function apply_patch_file
 ## @return       none; otherwise relaunches
 function check_reexec
 {
-  set +x
   local commentfile=${PATCH_DIR}/tp.${RANDOM}
 
   if [[ ${REEXECED} == true ]]; then

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ff91e9e/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 49106ae..5b2654a 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -604,6 +604,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11916. TestStringUtils#testLowerAndUpperStrings failed on MAC
     due to a JVM bug. (Ming Ma via ozawa)
 
+    HADOOP-11904. test-patch.sh goes into an infinite loop on non-maven
+    builds (aw)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[05/17] hadoop git commit: HADOOP-11120. hadoop fs -rmr gives wrong advice. Contributed by Juliet Houghland.

Posted by ji...@apache.org.
HADOOP-11120. hadoop fs -rmr gives wrong advice. Contributed by Juliet Houghland.


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

Branch: refs/heads/HDFS-7240
Commit: 05adc76ace6bf28e4a3ff874044c2c41e3eba63f
Parents: 9356cf8
Author: Andrew Wang <wa...@apache.org>
Authored: Tue May 5 08:37:37 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue May 5 08:37:37 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                    | 2 ++
 .../src/main/java/org/apache/hadoop/fs/shell/Delete.java           | 2 +-
 .../hadoop-common/src/test/resources/testConf.xml                  | 2 +-
 3 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/05adc76a/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 0b3c971..8b0e67c 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -528,6 +528,8 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11328. ZKFailoverController does not log Exception when doRun raises
     errors. (Tianyin Xu via ozawa)
 
+    HADOOP-11120. hadoop fs -rmr gives wrong advice. (Juliet Hougland via wang)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05adc76a/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 6798fbe..f882817 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
@@ -141,7 +141,7 @@ class Delete {
 
     @Override
     public String getReplacementCommand() {
-      return "rm -r";
+      return "-rm -r";
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05adc76a/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml b/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
index 9b72960..57cce14 100644
--- a/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
+++ b/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
@@ -453,7 +453,7 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^\s*\(DEPRECATED\) Same as 'rm -r'\s*</expected-output>
+          <expected-output>^\s*\(DEPRECATED\) Same as '-rm -r'\s*</expected-output>
         </comparator>
       </comparators>
     </test>


[13/17] hadoop git commit: YARN-3343. Increased TestCapacitySchedulerNodeLabelUpdate#testNodeUpdate timeout. Contributed by Rohith Sharmaks

Posted by ji...@apache.org.
YARN-3343. Increased TestCapacitySchedulerNodeLabelUpdate#testNodeUpdate timeout. Contributed by Rohith Sharmaks


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

Branch: refs/heads/HDFS-7240
Commit: e4c3b52c896291012f869ebc0a21e85e643fadd1
Parents: d33419a
Author: Jian He <ji...@apache.org>
Authored: Tue May 5 11:33:47 2015 -0700
Committer: Jian He <ji...@apache.org>
Committed: Tue May 5 11:33:47 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 3 +++
 .../scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java  | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4c3b52c/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 6dac3c8..00dd205 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -313,6 +313,9 @@ Release 2.8.0 - UNRELEASED
     YARN-2123. Progress bars in Web UI always at 100% due to non-US locale.
     (Akira AJISAKA via xgong)
 
+    YARN-3343. Increased TestCapacitySchedulerNodeLabelUpdate#testNodeUpdate
+    timeout. (Rohith Sharmaks via jianhe)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4c3b52c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java
index c5439d8..e60e496 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java
@@ -95,7 +95,7 @@ public class TestCapacitySchedulerNodeLabelUpdate {
         .getMemory());
   }
 
-  @Test (timeout = 30000)
+  @Test (timeout = 60000)
   public void testNodeUpdate() throws Exception {
     // set node -> label
     mgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x", "y", "z"));


[15/17] hadoop git commit: HDFS-8219. setStoragePolicy with folder behavior is different after cluster restart. (surendra singh lilhore via Xiaoyu Yao)

Posted by ji...@apache.org.
HDFS-8219. setStoragePolicy with folder behavior is different after cluster restart. (surendra singh lilhore via Xiaoyu Yao)


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

Branch: refs/heads/HDFS-7240
Commit: 0100b155019496d077f958904de7d385697d65d9
Parents: ffce9a3
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Tue May 5 13:41:14 2015 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Tue May 5 13:41:14 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |  2 +-
 .../hadoop/hdfs/TestBlockStoragePolicy.java     | 45 ++++++++++++++++++++
 3 files changed, 49 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0100b155/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 01de9b1..cc6758f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -626,6 +626,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8290. WebHDFS calls before namesystem initialization can cause
     NullPointerException. (cnauroth)
 
+    HDFS-8219. setStoragePolicy with folder behavior is different after cluster restart.
+    (surendra singh lilhore via Xiaoyu Yao)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0100b155/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index 28e150c..83e52bc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -739,7 +739,7 @@ public class FSEditLog implements LogsPurgeable {
       .setClientMachine(
           newNode.getFileUnderConstructionFeature().getClientMachine())
       .setOverwrite(overwrite)
-      .setStoragePolicyId(newNode.getStoragePolicyID());
+      .setStoragePolicyId(newNode.getLocalStoragePolicyID());
 
     AclFeature f = newNode.getAclFeature();
     if (f != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0100b155/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
index 9621dc8..5e3b55f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
@@ -26,6 +26,7 @@ import java.util.*;
 
 import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
@@ -1172,4 +1173,48 @@ public class TestBlockStoragePolicy {
       cluster.shutdown();
     }
   }
+
+  @Test
+  public void testGetFileStoragePolicyAfterRestartNN() throws Exception {
+    //HDFS8219
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(REPLICATION)
+        .storageTypes(
+            new StorageType[] {StorageType.DISK, StorageType.ARCHIVE})
+        .build();
+    cluster.waitActive();
+    final DistributedFileSystem fs = cluster.getFileSystem();
+    try {
+      final String file = "/testScheduleWithinSameNode/file";
+      Path dir = new Path("/testScheduleWithinSameNode");
+      fs.mkdirs(dir);
+      // 2. Set Dir policy
+      fs.setStoragePolicy(dir, "COLD");
+      // 3. Create file
+      final FSDataOutputStream out = fs.create(new Path(file));
+      out.writeChars("testScheduleWithinSameNode");
+      out.close();
+      // 4. Set Dir policy
+      fs.setStoragePolicy(dir, "HOT");
+      HdfsFileStatus status = fs.getClient().getFileInfo(file);
+      // 5. get file policy, it should be parent policy.
+      Assert
+          .assertTrue(
+              "File storage policy should be HOT",
+              status.getStoragePolicy()
+              == HdfsServerConstants.HOT_STORAGE_POLICY_ID);
+      // 6. restart NameNode for reloading edits logs.
+      cluster.restartNameNode(true);
+      // 7. get file policy, it should be parent policy.
+      status = fs.getClient().getFileInfo(file);
+      Assert
+          .assertTrue(
+              "File storage policy should be HOT",
+              status.getStoragePolicy()
+              == HdfsServerConstants.HOT_STORAGE_POLICY_ID);
+
+    } finally {
+      cluster.shutdown();
+    }
+  }
 }


[08/17] hadoop git commit: YARN-2123. Progress bars in Web UI always at 100% due to non-US locale. Contributed by Akira AJISAKA

Posted by ji...@apache.org.
YARN-2123. Progress bars in Web UI always at 100% due to non-US locale.
Contributed by Akira AJISAKA


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

Branch: refs/heads/HDFS-7240
Commit: b7dd3a4f04f712b7594c4e6e7ce50fd314f7c342
Parents: 9b01f81
Author: Xuan <xg...@apache.org>
Authored: Tue May 5 10:40:16 2015 -0700
Committer: Xuan <xg...@apache.org>
Committed: Tue May 5 10:40:16 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java | 4 ++--
 .../apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java   | 4 ++--
 .../apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java  | 9 +++++----
 hadoop-yarn-project/CHANGES.txt                             | 3 +++
 .../main/java/org/apache/hadoop/yarn/util/StringHelper.java | 5 -----
 .../org/apache/hadoop/yarn/server/webapp/AppsBlock.java     | 3 ++-
 .../server/resourcemanager/resource/ResourceWeights.java    | 2 +-
 .../resourcemanager/webapp/CapacitySchedulerPage.java       | 9 +++++----
 .../server/resourcemanager/webapp/DefaultSchedulerPage.java | 7 ++++---
 .../resourcemanager/webapp/FairSchedulerAppsBlock.java      | 3 ++-
 .../server/resourcemanager/webapp/FairSchedulerPage.java    | 7 ++++---
 .../yarn/server/resourcemanager/webapp/RMAppsBlock.java     | 3 ++-
 12 files changed, 32 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7dd3a4f/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
index 8aa8bb6..e293fd2 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.mapreduce.v2.app.webapp;
 
-import static org.apache.hadoop.yarn.util.StringHelper.percent;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES_ID;
@@ -31,6 +30,7 @@ import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
 import org.apache.hadoop.mapreduce.v2.util.MRWebAppUtil;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
@@ -77,7 +77,7 @@ public class TaskPage extends AppView {
 
       for (TaskAttempt attempt : getTaskAttempts()) {
         TaskAttemptInfo ta = new TaskAttemptInfo(attempt, true);
-        String progress = percent(ta.getProgress() / 100);
+        String progress = StringUtils.formatPercent(ta.getProgress() / 100, 2);
 
         String nodeHttpAddr = ta.getNode();
         String diag = ta.getNote() == null ? "" : ta.getNote();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7dd3a4f/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java
index 64aae59..7c1aa49 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.mapreduce.v2.app.webapp;
 import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_STATE;
 import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_TYPE;
 import static org.apache.hadoop.yarn.util.StringHelper.join;
-import static org.apache.hadoop.yarn.util.StringHelper.percent;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.C_PROGRESSBAR;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.C_PROGRESSBAR_VALUE;
 
@@ -30,6 +29,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskInfo;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
@@ -94,7 +94,7 @@ public class TasksBlock extends HtmlBlock {
 
       TaskInfo info = new TaskInfo(task);
       String tid = info.getId();
-      String pct = percent(info.getProgress() / 100);
+      String pct = StringUtils.formatPercent(info.getProgress() / 100, 2);
       tasksTableData.append("[\"<a href='").append(url("task", tid))
       .append("'>").append(tid).append("</a>\",\"")
       //Progress bar

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7dd3a4f/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java
index 50ebd16..d823cad 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.mapreduce.v2.app.webapp.dao;
 
-import static org.apache.hadoop.yarn.util.StringHelper.percent;
-
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -39,6 +37,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRApps.TaskAttemptStateUI;
 import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.util.Times;
 
 @XmlRootElement(name = "job")
@@ -102,11 +101,13 @@ public class JobInfo {
     this.mapsTotal = job.getTotalMaps();
     this.mapsCompleted = job.getCompletedMaps();
     this.mapProgress = report.getMapProgress() * 100;
-    this.mapProgressPercent = percent(report.getMapProgress());
+    this.mapProgressPercent =
+        StringUtils.formatPercent(report.getMapProgress(), 2);
     this.reducesTotal = job.getTotalReduces();
     this.reducesCompleted = job.getCompletedReduces();
     this.reduceProgress = report.getReduceProgress() * 100;
-    this.reduceProgressPercent = percent(report.getReduceProgress());
+    this.reduceProgressPercent =
+        StringUtils.formatPercent(report.getReduceProgress(), 2);
 
     this.acls = new ArrayList<ConfEntryInfo>();
     if (hasAccess) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7dd3a4f/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index a6b7f17..6dac3c8 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -310,6 +310,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3552. RM Web UI shows -1 running containers for completed apps
     (Rohith via jlowe)
 
+    YARN-2123. Progress bars in Web UI always at 100% due to non-US locale.
+    (Akira AJISAKA via xgong)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7dd3a4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/StringHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/StringHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/StringHelper.java
index 3343a17..0240fbc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/StringHelper.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/StringHelper.java
@@ -174,9 +174,4 @@ public final class StringHelper {
     }
     sb.append(part);
   }
-  
-  public static String percent(double value) {
-    return String.format("%.2f", value * 100);
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7dd3a4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
index 93248ea..5df70c4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
@@ -32,6 +32,7 @@ import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
@@ -123,7 +124,7 @@ public class AppsBlock extends HtmlBlock {
         continue;
       }
       AppInfo app = new AppInfo(appReport);
-      String percent = String.format("%.1f", app.getProgress());
+      String percent = StringUtils.format("%.1f", app.getProgress());
       appsTableData
         .append("[\"<a href='")
         .append(url("app", app.getAppId()))

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7dd3a4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/ResourceWeights.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/resource/ResourceWeights.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/ResourceWeights.java
index d6e9e45..4c62318 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/ResourceWeights.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/ResourceWeights.java
@@ -63,7 +63,7 @@ public class ResourceWeights {
       }
       ResourceType resourceType = ResourceType.values()[i];
       sb.append(StringUtils.toLowerCase(resourceType.name()));
-      sb.append(String.format(" weight=%.1f", getWeight(resourceType)));
+      sb.append(StringUtils.format(" weight=%.1f", getWeight(resourceType)));
     }
     sb.append(">");
     return sb.toString();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7dd3a4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
index fa22a0d..4381a34 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
@@ -94,7 +94,8 @@ class CapacitySchedulerPage extends RmView {
           _("Configured Capacity:", percent(lqinfo.getCapacity() / 100)).
           _("Configured Max Capacity:", percent(lqinfo.getMaxCapacity() / 100)).
           _("Configured Minimum User Limit Percent:", Integer.toString(lqinfo.getUserLimit()) + "%").
-          _("Configured User Limit Factor:", String.format("%.1f", lqinfo.getUserLimitFactor())).
+          _("Configured User Limit Factor:", StringUtils.format(
+              "%.1f", lqinfo.getUserLimitFactor())).
           _("Accessible Node Labels:", StringUtils.join(",", lqinfo.getNodeLabels())).
           _("Ordering Policy: ", lqinfo.getOrderingPolicyInfo()).
           _("Preemption:", lqinfo.getPreemptionDisabled() ? "disabled" : "enabled");
@@ -432,14 +433,14 @@ class CapacitySchedulerPage extends RmView {
   }
 
   static String percent(float f) {
-    return String.format("%.1f%%", f * 100);
+    return StringUtils.formatPercent(f, 1);
   }
 
   static String width(float f) {
-    return String.format("width:%.1f%%", f * 100);
+    return StringUtils.format("width:%.1f%%", f * 100);
   }
 
   static String left(float f) {
-    return String.format("left:%.1f%%", f * 100);
+    return StringUtils.format("left:%.1f%%", f * 100);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7dd3a4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DefaultSchedulerPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DefaultSchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DefaultSchedulerPage.java
index 1c8828c..36d8309 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DefaultSchedulerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DefaultSchedulerPage.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
 import static org.apache.hadoop.yarn.util.StringHelper.join;
 
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.FifoSchedulerInfo;
@@ -149,14 +150,14 @@ class DefaultSchedulerPage extends RmView {
   }
 
   static String percent(float f) {
-    return String.format("%.1f%%", f * 100);
+    return StringUtils.formatPercent(f, 1);
   }
 
   static String width(float f) {
-    return String.format("width:%.1f%%", f * 100);
+    return StringUtils.format("width:%.1f%%", f * 100);
   }
 
   static String left(float f) {
-    return String.format("left:%.1f%%", f * 100);
+    return StringUtils.format("left:%.1f%%", f * 100);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7dd3a4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
index 8f8ece0..80099d0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
@@ -31,6 +31,7 @@ import java.util.concurrent.ConcurrentMap;
 
 import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
@@ -110,7 +111,7 @@ public class FairSchedulerAppsBlock extends HtmlBlock {
         continue;
       }
       AppInfo appInfo = new AppInfo(rm, app, true, WebAppUtils.getHttpSchemePrefix(conf));
-      String percent = String.format("%.1f", appInfo.getProgress());
+      String percent = StringUtils.format("%.1f", appInfo.getProgress());
       ApplicationAttemptId attemptId = app.getCurrentAppAttempt().getAppAttemptId();
       int fairShare = fsinfo.getAppFairShare(attemptId);
       if (fairShare == FairSchedulerInfo.INVALID_FAIR_SHARE) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7dd3a4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java
index 76d7e6f..d87fb5c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java
@@ -22,6 +22,7 @@ import static org.apache.hadoop.yarn.util.StringHelper.join;
 
 import java.util.Collection;
 
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.FairSchedulerInfo;
@@ -241,14 +242,14 @@ public class FairSchedulerPage extends RmView {
   }
 
   static String percent(float f) {
-    return String.format("%.1f%%", f * 100);
+    return StringUtils.formatPercent(f, 1);
   }
 
   static String width(float f) {
-    return String.format("width:%.1f%%", f * 100);
+    return StringUtils.format("width:%.1f%%", f * 100);
   }
 
   static String left(float f) {
-    return String.format("left:%.1f%%", f * 100);
+    return StringUtils.format("left:%.1f%%", f * 100);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7dd3a4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
index 3054150..1d97bef 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
@@ -25,6 +25,7 @@ import static org.apache.hadoop.yarn.webapp.view.JQueryUI.C_PROGRESSBAR_VALUE;
 import java.util.Set;
 
 import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
@@ -82,7 +83,7 @@ public class RMAppsBlock extends AppsBlock {
       if (nodes != null) {
         blacklistedNodesCount = String.valueOf(nodes.size());
       }
-      String percent = String.format("%.1f", app.getProgress());
+      String percent = StringUtils.format("%.1f", app.getProgress());
       appsTableData
         .append("[\"<a href='")
         .append(url("app", app.getAppId()))


[03/17] hadoop git commit: HDFS-7916. 'reportBadBlocks' from datanodes to standby Node BPServiceActor goes for infinite loop (Contributed by Vinayakumar B) Reverted earlier commit

Posted by ji...@apache.org.
HDFS-7916. 'reportBadBlocks' from datanodes to standby Node BPServiceActor goes for infinite loop (Contributed by Vinayakumar B)
Reverted earlier commit


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

Branch: refs/heads/HDFS-7240
Commit: 318081ccd7af1ec02ec18f35ea95c579326be728
Parents: 338e88a
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue May 5 11:05:37 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Tue May 5 11:05:37 2015 +0530

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                      | 3 ---
 .../apache/hadoop/hdfs/server/datanode/ReportBadBlockAction.java | 4 ----
 2 files changed, 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/318081cc/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 365b005..cd9b7b8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -664,9 +664,6 @@ Release 2.7.1 - UNRELEASED
     HDFS-8179. DFSClient#getServerDefaults returns null within 1
     hour of system start. (Xiaoyu Yao via Arpit Agarwal)
 
-    HDFS-7916. 'reportBadBlocks' from datanodes to standby Node BPServiceActor
-    goes for infinite loop (vinayakumarb)
-
     HDFS-8163. Using monotonicNow for block report scheduling causes
     test failures on recently restarted systems. (Arpit Agarwal)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/318081cc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReportBadBlockAction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReportBadBlockAction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReportBadBlockAction.java
index 991b56d..7155eae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReportBadBlockAction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReportBadBlockAction.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
-import org.apache.hadoop.ipc.StandbyException;
 
 /**
  * ReportBadBlockAction is an instruction issued by {{BPOfferService}} to
@@ -60,9 +59,6 @@ public class ReportBadBlockAction implements BPServiceActorAction {
 
     try {
       bpNamenode.reportBadBlocks(locatedBlock);
-    } catch (StandbyException e) {
-      DataNode.LOG.warn("Failed to report bad block " + block
-          + " to standby namenode");
     } catch (IOException e) {
       throw new BPServiceActorActionException("Failed to report bad block "
           + block + " to namenode: ");


[11/17] hadoop git commit: HDFS-7758. Retire FsDatasetSpi#getVolumes() and use FsDatasetSpi#getVolumeRefs() instead (Lei (Eddy) Xu via Colin P. McCabe)

Posted by ji...@apache.org.
HDFS-7758. Retire FsDatasetSpi#getVolumes() and use FsDatasetSpi#getVolumeRefs() instead (Lei (Eddy) Xu via Colin P. McCabe)


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

Branch: refs/heads/HDFS-7240
Commit: 24d3a2d4fdd836ac9a5bc755a7fb9354f7a582b1
Parents: 3ff91e9
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Tue May 5 10:55:04 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Tue May 5 11:08:59 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hdfs/server/datanode/DirectoryScanner.java  |  69 +++++-----
 .../server/datanode/fsdataset/FsDatasetSpi.java |  99 +++++++++++++-
 .../datanode/fsdataset/FsVolumeReference.java   |  13 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |  29 ++---
 .../datanode/fsdataset/impl/FsVolumeImpl.java   |   7 +-
 .../datanode/fsdataset/impl/FsVolumeList.java   |   7 +-
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |  23 ++--
 .../hdfs/MiniDFSClusterWithNodeGroup.java       |  17 +--
 .../TestNameNodePrunesMissingStorages.java      |  18 +--
 .../server/datanode/SimulatedFSDataset.java     |   2 +-
 .../TestBlockHasMultipleReplicasOnSameDN.java   |  14 +-
 .../hdfs/server/datanode/TestBlockScanner.java  |   8 +-
 .../datanode/TestDataNodeHotSwapVolumes.java    |  21 ++-
 .../datanode/TestDataNodeVolumeFailure.java     |   9 +-
 .../server/datanode/TestDirectoryScanner.java   | 130 +++++++++++--------
 .../hdfs/server/datanode/TestDiskError.java     |  12 +-
 .../datanode/TestIncrementalBlockReports.java   |   6 +-
 .../datanode/TestIncrementalBrVariations.java   |  81 ++++++------
 .../server/datanode/TestTriggerBlockReport.java |   8 +-
 .../extdataset/ExternalDatasetImpl.java         |   2 +-
 .../fsdataset/impl/LazyPersistTestCase.java     |  72 +++++-----
 .../fsdataset/impl/TestDatanodeRestart.java     |  20 ++-
 .../fsdataset/impl/TestFsDatasetImpl.java       |  39 ++++--
 .../fsdataset/impl/TestFsVolumeList.java        |   9 +-
 .../fsdataset/impl/TestRbwSpaceReservation.java |  43 +++---
 .../fsdataset/impl/TestWriteToReplica.java      |  70 ++++++----
 .../hdfs/server/mover/TestStorageMover.java     |  19 ++-
 28 files changed, 515 insertions(+), 335 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/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 08ab7e7..c89e6fe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -507,6 +507,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8237. Move all protocol classes used by ClientProtocol to hdfs-client.
     (wheat9)
 
+    HDFS-7758. Retire FsDatasetSpi#getVolumes() and use
+    FsDatasetSpi#getVolumeRefs() instead (Lei (Eddy) Xu via Colin P. McCabe)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
index 62885a9..8453094 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
@@ -527,59 +527,48 @@ public class DirectoryScanner implements Runnable {
     diffRecord.add(new ScanInfo(blockId, null, null, vol));
   }
 
-  /** Is the given volume still valid in the dataset? */
-  private static boolean isValid(final FsDatasetSpi<?> dataset,
-      final FsVolumeSpi volume) {
-    for (FsVolumeSpi vol : dataset.getVolumes()) {
-      if (vol == volume) {
-        return true;
-      }
-    }
-    return false;
-  }
-
   /** Get lists of blocks on the disk sorted by blockId, per blockpool */
   private Map<String, ScanInfo[]> getDiskReport() {
+    ScanInfoPerBlockPool list = new ScanInfoPerBlockPool();
+    ScanInfoPerBlockPool[] dirReports = null;
     // First get list of data directories
-    final List<? extends FsVolumeSpi> volumes = dataset.getVolumes();
+    try (FsDatasetSpi.FsVolumeReferences volumes =
+        dataset.getFsVolumeReferences()) {
 
-    // Use an array since the threads may return out of order and
-    // compilersInProgress#keySet may return out of order as well.
-    ScanInfoPerBlockPool[] dirReports = new ScanInfoPerBlockPool[volumes.size()];
+      // Use an array since the threads may return out of order and
+      // compilersInProgress#keySet may return out of order as well.
+      dirReports = new ScanInfoPerBlockPool[volumes.size()];
 
-    Map<Integer, Future<ScanInfoPerBlockPool>> compilersInProgress =
-      new HashMap<Integer, Future<ScanInfoPerBlockPool>>();
+      Map<Integer, Future<ScanInfoPerBlockPool>> compilersInProgress =
+          new HashMap<Integer, Future<ScanInfoPerBlockPool>>();
 
-    for (int i = 0; i < volumes.size(); i++) {
-      if (isValid(dataset, volumes.get(i))) {
+      for (int i = 0; i < volumes.size(); i++) {
         ReportCompiler reportCompiler =
-          new ReportCompiler(datanode,volumes.get(i));
-        Future<ScanInfoPerBlockPool> result = 
-          reportCompileThreadPool.submit(reportCompiler);
+            new ReportCompiler(datanode, volumes.get(i));
+        Future<ScanInfoPerBlockPool> result =
+            reportCompileThreadPool.submit(reportCompiler);
         compilersInProgress.put(i, result);
       }
-    }
-    
-    for (Entry<Integer, Future<ScanInfoPerBlockPool>> report :
-        compilersInProgress.entrySet()) {
-      try {
-        dirReports[report.getKey()] = report.getValue().get();
-      } catch (Exception ex) {
-        LOG.error("Error compiling report", ex);
-        // Propagate ex to DataBlockScanner to deal with
-        throw new RuntimeException(ex);
+
+      for (Entry<Integer, Future<ScanInfoPerBlockPool>> report :
+          compilersInProgress.entrySet()) {
+        try {
+          dirReports[report.getKey()] = report.getValue().get();
+        } catch (Exception ex) {
+          LOG.error("Error compiling report", ex);
+          // Propagate ex to DataBlockScanner to deal with
+          throw new RuntimeException(ex);
+        }
       }
+    } catch (IOException e) {
+      LOG.error("Unexpected IOException by closing FsVolumeReference", e);
     }
-
-    // Compile consolidated report for all the volumes
-    ScanInfoPerBlockPool list = new ScanInfoPerBlockPool();
-    for (int i = 0; i < volumes.size(); i++) {
-      if (isValid(dataset, volumes.get(i))) {
-        // volume is still valid
-        list.addAll(dirReports[i]);
+    if (dirReports != null) {
+      // Compile consolidated report for all the volumes
+      for (ScanInfoPerBlockPool report : dirReports) {
+        list.addAll(report);
       }
     }
-
     return list.toSortedArrays();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
index 8a741de..76c4f02 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
@@ -18,13 +18,16 @@
 package org.apache.hadoop.hdfs.server.datanode.fsdataset;
 
 
+import java.io.Closeable;
 import java.io.EOFException;
 import java.io.File;
 import java.io.FileDescriptor;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.Collection;
+import java.nio.channels.ClosedChannelException;
+import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -50,7 +53,6 @@ import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.datanode.UnexpectedReplicaStateException;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetFactory;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
 import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -58,7 +60,6 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
-import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.ReflectionUtils;
 
 /**
@@ -92,8 +93,96 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
     }
   }
 
-  /** @return a list of volumes. */
-  public List<V> getVolumes();
+  /**
+   * It behaviors as an unmodifiable list of FsVolume. Individual FsVolume can
+   * be obtained by using {@link #get(int)}.
+   *
+   * This also holds the reference counts for these volumes. It releases all the
+   * reference counts in {@link #close()}.
+   */
+  class FsVolumeReferences implements Iterable<FsVolumeSpi>, Closeable {
+    private final List<FsVolumeReference> references;
+
+    public <S extends FsVolumeSpi> FsVolumeReferences(List<S> curVolumes) {
+      references = new ArrayList<>();
+      for (FsVolumeSpi v : curVolumes) {
+        try {
+          references.add(v.obtainReference());
+        } catch (ClosedChannelException e) {
+          // This volume has been closed.
+        }
+      }
+    }
+
+    private static class FsVolumeSpiIterator implements
+        Iterator<FsVolumeSpi> {
+      private final List<FsVolumeReference> references;
+      private int idx = 0;
+
+      FsVolumeSpiIterator(List<FsVolumeReference> refs) {
+        references = refs;
+      }
+
+      @Override
+      public boolean hasNext() {
+        return idx < references.size();
+      }
+
+      @Override
+      public FsVolumeSpi next() {
+        int refIdx = idx++;
+        return references.get(refIdx).getVolume();
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    }
+
+    @Override
+    public Iterator<FsVolumeSpi> iterator() {
+      return new FsVolumeSpiIterator(references);
+    }
+
+    /**
+     * Get the number of volumes.
+     */
+    public int size() {
+      return references.size();
+    }
+
+    /**
+     * Get the volume for a given index.
+     */
+    public FsVolumeSpi get(int index) {
+      return references.get(index).getVolume();
+    }
+
+    @Override
+    public void close() throws IOException {
+      IOException ioe = null;
+      for (FsVolumeReference ref : references) {
+        try {
+          ref.close();
+        } catch (IOException e) {
+          ioe = e;
+        }
+      }
+      references.clear();
+      if (ioe != null) {
+        throw ioe;
+      }
+    }
+  }
+
+  /**
+   * Returns a list of FsVolumes that hold reference counts.
+   *
+   * The caller must release the reference of each volume by calling
+   * {@link FsVolumeReferences#close()}.
+   */
+  public FsVolumeReferences getFsVolumeReferences();
 
   /**
    * Add a new volume to the FsDataset.<p/>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeReference.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeReference.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeReference.java
index e61a059..8d39719 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeReference.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeReference.java
@@ -21,7 +21,7 @@ import java.io.Closeable;
 import java.io.IOException;
 
 /**
- * This is the interface for holding reference count as AutoClosable resource.
+ * This holds volume reference count as AutoClosable resource.
  * It increases the reference count by one in the constructor, and decreases
  * the reference count by one in {@link #close()}.
  *
@@ -37,12 +37,15 @@ import java.io.IOException;
  */
 public interface FsVolumeReference extends Closeable {
   /**
-   * Descrese the reference count of the volume.
+   * Decrease the reference count of the volume.
    * @throws IOException it never throws IOException.
    */
   @Override
-  public void close() throws IOException;
+  void close() throws IOException;
 
-  /** Returns the underlying volume object */
-  public FsVolumeSpi getVolume();
+  /**
+   * Returns the underlying volume object. Return null if the reference was
+   * released.
+   */
+  FsVolumeSpi getVolume();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/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 d72b8bf..31ac414 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
@@ -138,8 +138,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   }
 
   @Override // FsDatasetSpi
-  public List<FsVolumeImpl> getVolumes() {
-    return volumes.getVolumes();
+  public FsVolumeReferences getFsVolumeReferences() {
+    return new FsVolumeReferences(volumes.getVolumes());
   }
 
   @Override
@@ -152,7 +152,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       throws IOException {
     List<StorageReport> reports;
     synchronized (statsLock) {
-      List<FsVolumeImpl> curVolumes = getVolumes();
+      List<FsVolumeImpl> curVolumes = volumes.getVolumes();
       reports = new ArrayList<>(curVolumes.size());
       for (FsVolumeImpl volume : curVolumes) {
         try (FsVolumeReference ref = volume.obtainReference()) {
@@ -231,7 +231,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     
   final DataNode datanode;
   final DataStorage dataStorage;
-  final FsVolumeList volumes;
+  private final FsVolumeList volumes;
   final Map<String, DatanodeStorage> storageMap;
   final FsDatasetAsyncDiskService asyncDiskService;
   final Daemon lazyWriter;
@@ -540,7 +540,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
    */
   @Override // FsDatasetSpi
   public boolean hasEnoughResource() {
-    return getVolumes().size() >= validVolsRequired; 
+    return volumes.getVolumes().size() >= validVolsRequired;
   }
 
   /**
@@ -1628,7 +1628,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     Map<String, BlockListAsLongs.Builder> builders =
         new HashMap<String, BlockListAsLongs.Builder>();
 
-    List<FsVolumeImpl> curVolumes = getVolumes();
+    List<FsVolumeImpl> curVolumes = volumes.getVolumes();
     for (FsVolumeSpi v : curVolumes) {
       builders.put(v.getStorageID(), BlockListAsLongs.builder());
     }
@@ -2535,7 +2535,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
   private Collection<VolumeInfo> getVolumeInfo() {
     Collection<VolumeInfo> info = new ArrayList<VolumeInfo>();
-    for (FsVolumeImpl volume : getVolumes()) {
+    for (FsVolumeImpl volume : volumes.getVolumes()) {
       long used = 0;
       long free = 0;
       try (FsVolumeReference ref = volume.obtainReference()) {
@@ -2571,7 +2571,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   @Override //FsDatasetSpi
   public synchronized void deleteBlockPool(String bpid, boolean force)
       throws IOException {
-    List<FsVolumeImpl> curVolumes = getVolumes();
+    List<FsVolumeImpl> curVolumes = volumes.getVolumes();
     if (!force) {
       for (FsVolumeImpl volume : curVolumes) {
         try (FsVolumeReference ref = volume.obtainReference()) {
@@ -2622,7 +2622,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   @Override // FsDatasetSpi
   public HdfsBlocksMetadata getHdfsBlocksMetadata(String poolId,
       long[] blockIds) throws IOException {
-    List<FsVolumeImpl> curVolumes = getVolumes();
+    List<FsVolumeImpl> curVolumes = volumes.getVolumes();
     // List of VolumeIds, one per volume on the datanode
     List<byte[]> blocksVolumeIds = new ArrayList<>(curVolumes.size());
     // List of indexes into the list of VolumeIds, pointing at the VolumeId of
@@ -2730,7 +2730,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   }
 
   private boolean ramDiskConfigured() {
-    for (FsVolumeImpl v: getVolumes()){
+    for (FsVolumeImpl v: volumes.getVolumes()){
       if (v.isTransientStorage()) {
         return true;
       }
@@ -2742,7 +2742,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   // added or removed.
   // This should only be called when the FsDataSetImpl#volumes list is finalized.
   private void setupAsyncLazyPersistThreads() {
-    for (FsVolumeImpl v: getVolumes()){
+    for (FsVolumeImpl v: volumes.getVolumes()){
       setupAsyncLazyPersistThread(v);
     }
   }
@@ -2880,14 +2880,13 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
       // Don't worry about fragmentation for now. We don't expect more than one
       // transient volume per DN.
-      for (FsVolumeImpl v : getVolumes()) {
-        try (FsVolumeReference ref = v.obtainReference()) {
+      try (FsVolumeReferences volumes = getFsVolumeReferences()) {
+        for (FsVolumeSpi fvs : volumes) {
+          FsVolumeImpl v = (FsVolumeImpl) fvs;
           if (v.isTransientStorage()) {
             capacity += v.getCapacity();
             free += v.getAvailable();
           }
-        } catch (ClosedChannelException e) {
-          // ignore.
         }
       }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
index 922ede5..bc96a02 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
@@ -198,7 +198,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
   }
 
   private static class FsVolumeReferenceImpl implements FsVolumeReference {
-    private final FsVolumeImpl volume;
+    private FsVolumeImpl volume;
 
     FsVolumeReferenceImpl(FsVolumeImpl volume) throws ClosedChannelException {
       this.volume = volume;
@@ -211,7 +211,10 @@ public class FsVolumeImpl implements FsVolumeSpi {
      */
     @Override
     public void close() throws IOException {
-      volume.unreference();
+      if (volume != null) {
+        volume.unreference();
+        volume = null;
+      }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
index d87595c..a73e129 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
@@ -276,10 +276,11 @@ class FsVolumeList {
    * @param ref       a reference to the new FsVolumeImpl instance.
    */
   void addVolume(FsVolumeReference ref) {
+    FsVolumeImpl volume = (FsVolumeImpl) ref.getVolume();
     while (true) {
       final FsVolumeImpl[] curVolumes = volumes.get();
       final List<FsVolumeImpl> volumeList = Lists.newArrayList(curVolumes);
-      volumeList.add((FsVolumeImpl)ref.getVolume());
+      volumeList.add(volume);
       if (volumes.compareAndSet(curVolumes,
           volumeList.toArray(new FsVolumeImpl[volumeList.size()]))) {
         break;
@@ -300,9 +301,9 @@ class FsVolumeList {
     }
     // If the volume is used to replace a failed volume, it needs to reset the
     // volume failure info for this volume.
-    removeVolumeFailureInfo(new File(ref.getVolume().getBasePath()));
+    removeVolumeFailureInfo(new File(volume.getBasePath()));
     FsDatasetImpl.LOG.info("Added new volume: " +
-        ref.getVolume().getStorageID());
+        volume.getStorageID());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
index 8658e47..8aeaef8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
@@ -1493,15 +1493,20 @@ public class MiniDFSCluster {
     if (storageCapacities != null) {
       for (int i = curDatanodesNum; i < curDatanodesNum+numDataNodes; ++i) {
         final int index = i - curDatanodesNum;
-        List<? extends FsVolumeSpi> volumes = dns[index].getFSDataset().getVolumes();
-        assert storageCapacities[index].length == storagesPerDatanode;
-        assert volumes.size() == storagesPerDatanode;
-
-        for (int j = 0; j < volumes.size(); ++j) {
-          FsVolumeImpl volume = (FsVolumeImpl) volumes.get(j);
-          LOG.info("setCapacityForTesting "  + storageCapacities[index][j]
-              + " for [" + volume.getStorageType() + "]" + volume.getStorageID());
-          volume.setCapacityForTesting(storageCapacities[index][j]);
+        try (FsDatasetSpi.FsVolumeReferences volumes =
+            dns[index].getFSDataset().getFsVolumeReferences()) {
+          assert storageCapacities[index].length == storagesPerDatanode;
+          assert volumes.size() == storagesPerDatanode;
+
+          int j = 0;
+          for (FsVolumeSpi fvs : volumes) {
+            FsVolumeImpl volume = (FsVolumeImpl) fvs;
+            LOG.info("setCapacityForTesting " + storageCapacities[index][j]
+                + " for [" + volume.getStorageType() + "]" + volume
+                .getStorageID());
+            volume.setCapacityForTesting(storageCapacities[index][j]);
+            j++;
+          }
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java
index ca2d8d6..b9682a1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY;
 
 import java.io.IOException;
-import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -31,7 +30,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.StaticMapping;
@@ -195,12 +194,14 @@ public class MiniDFSClusterWithNodeGroup extends MiniDFSCluster {
 
     if (storageCapacities != null) {
       for (int i = curDatanodesNum; i < curDatanodesNum+numDataNodes; ++i) {
-        List<? extends FsVolumeSpi> volumes = dns[i].getFSDataset().getVolumes();
-        assert volumes.size() == storagesPerDatanode;
-
-        for (int j = 0; j < volumes.size(); ++j) {
-          FsVolumeImpl volume = (FsVolumeImpl) volumes.get(j);
-          volume.setCapacityForTesting(storageCapacities[i][j]);
+        try (FsDatasetSpi.FsVolumeReferences volumes =
+            dns[i].getFSDataset().getFsVolumeReferences()) {
+          assert volumes.size() == storagesPerDatanode;
+
+          for (int j = 0; j < volumes.size(); ++j) {
+            FsVolumeImpl volume = (FsVolumeImpl) volumes.get(j);
+            volume.setCapacityForTesting(storageCapacities[i][j]);
+          }
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
index 4b97d01..78c1d27 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
@@ -43,9 +44,7 @@ import org.junit.Test;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
-import java.util.Arrays;
 import java.util.Iterator;
-import java.util.List;
 
 import static org.hamcrest.core.Is.is;
 import static org.junit.Assert.assertThat;
@@ -199,15 +198,16 @@ public class TestNameNodePrunesMissingStorages {
         datanodeToRemoveStorageFromIdx++;
       }
       // Find the volume within the datanode which holds that first storage.
-      List<? extends FsVolumeSpi> volumes =
-          datanodeToRemoveStorageFrom.getFSDataset().getVolumes();
-      assertEquals(NUM_STORAGES_PER_DN, volumes.size());
       String volumeDirectoryToRemove = null;
-      for (FsVolumeSpi volume : volumes) {
-        if (volume.getStorageID().equals(storageIdToRemove)) {
-          volumeDirectoryToRemove = volume.getBasePath();
+      try (FsDatasetSpi.FsVolumeReferences volumes =
+          datanodeToRemoveStorageFrom.getFSDataset().getFsVolumeReferences()) {
+        assertEquals(NUM_STORAGES_PER_DN, volumes.size());
+        for (FsVolumeSpi volume : volumes) {
+          if (volume.getStorageID().equals(storageIdToRemove)) {
+            volumeDirectoryToRemove = volume.getBasePath();
+          }
         }
-      }
+      };
       // Shut down the datanode and remove the volume.
       // Replace the volume directory with a regular file, which will
       // cause a volume failure.  (If we merely removed the directory,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index bc24237..2ac9416 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -1270,7 +1270,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   }
 
   @Override
-  public List<FsVolumeSpi> getVolumes() {
+  public FsVolumeReferences getFsVolumeReferences() {
     throw new UnsupportedOperationException();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java
index c47209e..c65ef85 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java
@@ -27,8 +27,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.*;
 import org.apache.hadoop.hdfs.protocol.*;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
@@ -115,11 +115,13 @@ public class TestBlockHasMultipleReplicasOnSameDN {
       blocks.add(new FinalizedReplica(localBlock, null, null));
     }
 
-    BlockListAsLongs bll = BlockListAsLongs.encode(blocks);
-    for (int i = 0; i < cluster.getStoragesPerDatanode(); ++i) {
-      FsVolumeSpi v = dn.getFSDataset().getVolumes().get(i);
-      DatanodeStorage dns = new DatanodeStorage(v.getStorageID());
-      reports[i] = new StorageBlockReport(dns, bll);
+    try (FsDatasetSpi.FsVolumeReferences volumes =
+      dn.getFSDataset().getFsVolumeReferences()) {
+      BlockListAsLongs bll = BlockListAsLongs.encode(blocks);
+      for (int i = 0; i < cluster.getStoragesPerDatanode(); ++i) {
+        DatanodeStorage dns = new DatanodeStorage(volumes.get(i).getStorageID());
+        reports[i] = new StorageBlockReport(dns, bll);
+      }
     }
 
     // Should not assert!

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java
index 14e29e4..b01597a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java
@@ -82,7 +82,7 @@ public class TestBlockScanner {
     final DataNode datanode;
     final BlockScanner blockScanner;
     final FsDatasetSpi<? extends FsVolumeSpi> data;
-    final List<? extends FsVolumeSpi> volumes;
+    final FsDatasetSpi.FsVolumeReferences volumes;
 
     TestContext(Configuration conf, int numNameServices) throws Exception {
       this.numNameServices = numNameServices;
@@ -109,11 +109,12 @@ public class TestBlockScanner {
         dfs[i].mkdirs(new Path("/test"));
       }
       data = datanode.getFSDataset();
-      volumes = data.getVolumes();
+      volumes = data.getFsVolumeReferences();
     }
 
     @Override
     public void close() throws IOException {
+      volumes.close();
       if (cluster != null) {
         for (int i = 0; i < numNameServices; i++) {
           dfs[i].delete(new Path("/test"), true);
@@ -713,8 +714,7 @@ public class TestBlockScanner {
     ctx.createFiles(0, NUM_EXPECTED_BLOCKS, 1);
     final TestScanResultHandler.Info info =
         TestScanResultHandler.getInfo(ctx.volumes.get(0));
-    String storageID = ctx.datanode.getFSDataset().
-        getVolumes().get(0).getStorageID();
+    String storageID = ctx.volumes.get(0).getStorageID();
     synchronized (info) {
       info.sem = new Semaphore(4);
       info.shouldRun = true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
index 315529c..a396b0e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
@@ -517,9 +517,12 @@ public class TestDataNodeHotSwapVolumes {
 
     // Make sure that vol0 and vol2's metadata are not left in memory.
     FsDatasetSpi<?> dataset = dn.getFSDataset();
-    for (FsVolumeSpi volume : dataset.getVolumes()) {
-      assertThat(volume.getBasePath(), is(not(anyOf(
-          is(newDirs.get(0)), is(newDirs.get(2))))));
+    try (FsDatasetSpi.FsVolumeReferences volumes =
+        dataset.getFsVolumeReferences()) {
+      for (FsVolumeSpi volume : volumes) {
+        assertThat(volume.getBasePath(), is(not(anyOf(
+            is(newDirs.get(0)), is(newDirs.get(2))))));
+      }
     }
     DataStorage storage = dn.getStorage();
     for (int i = 0; i < storage.getNumStorageDirs(); i++) {
@@ -688,10 +691,14 @@ public class TestDataNodeHotSwapVolumes {
   }
 
   /** Get the FsVolume on the given basePath */
-  private FsVolumeImpl getVolume(DataNode dn, File basePath) {
-    for (FsVolumeSpi vol : dn.getFSDataset().getVolumes()) {
-      if (vol.getBasePath().equals(basePath.getPath())) {
-        return (FsVolumeImpl)vol;
+  private FsVolumeImpl getVolume(DataNode dn, File basePath)
+      throws IOException {
+    try (FsDatasetSpi.FsVolumeReferences volumes =
+      dn.getFSDataset().getFsVolumeReferences()) {
+      for (FsVolumeSpi vol : volumes) {
+        if (vol.getBasePath().equals(basePath.getPath())) {
+          return (FsVolumeImpl) vol;
+        }
       }
     }
     return null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
index 0d158c9..d38892c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
@@ -63,7 +63,6 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
@@ -249,9 +248,11 @@ public class TestDataNodeVolumeFailure {
 
     // 2. dn0Vol1 is removed from FsDataset
     FsDatasetSpi<? extends FsVolumeSpi> data = dn0.getFSDataset();
-    for (FsVolumeSpi volume : data.getVolumes()) {
-      assertNotEquals(new File(volume.getBasePath()).getAbsoluteFile(),
-          dn0Vol1.getAbsoluteFile());
+    try (FsDatasetSpi.FsVolumeReferences vols = data.getFsVolumeReferences()) {
+      for (FsVolumeSpi volume : vols) {
+        assertNotEquals(new File(volume.getBasePath()).getAbsoluteFile(),
+            dn0Vol1.getAbsoluteFile());
+      }
     }
 
     // 3. all blocks on dn0Vol1 have been removed.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
index 150b5e4..b225e35 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
@@ -157,30 +157,37 @@ public class TestDirectoryScanner {
   private void duplicateBlock(long blockId) throws IOException {
     synchronized (fds) {
       ReplicaInfo b = FsDatasetTestUtil.fetchReplicaInfo(fds, bpid, blockId);
-      for (FsVolumeSpi v : fds.getVolumes()) {
-        if (v.getStorageID().equals(b.getVolume().getStorageID())) {
-          continue;
-        }
-
-        // Volume without a copy of the block. Make a copy now.
-        File sourceBlock = b.getBlockFile();
-        File sourceMeta = b.getMetaFile();
-        String sourceRoot = b.getVolume().getBasePath();
-        String destRoot = v.getBasePath();
-
-        String relativeBlockPath = new File(sourceRoot).toURI().relativize(sourceBlock.toURI()).getPath();
-        String relativeMetaPath = new File(sourceRoot).toURI().relativize(sourceMeta.toURI()).getPath();
-
-        File destBlock = new File(destRoot, relativeBlockPath);
-        File destMeta = new File(destRoot, relativeMetaPath);
-
-        destBlock.getParentFile().mkdirs();
-        FileUtils.copyFile(sourceBlock, destBlock);
-        FileUtils.copyFile(sourceMeta, destMeta);
+      try (FsDatasetSpi.FsVolumeReferences volumes =
+          fds.getFsVolumeReferences()) {
+        for (FsVolumeSpi v : volumes) {
+          if (v.getStorageID().equals(b.getVolume().getStorageID())) {
+            continue;
+          }
 
-        if (destBlock.exists() && destMeta.exists()) {
-          LOG.info("Copied " + sourceBlock + " ==> " + destBlock);
-          LOG.info("Copied " + sourceMeta + " ==> " + destMeta);
+          // Volume without a copy of the block. Make a copy now.
+          File sourceBlock = b.getBlockFile();
+          File sourceMeta = b.getMetaFile();
+          String sourceRoot = b.getVolume().getBasePath();
+          String destRoot = v.getBasePath();
+
+          String relativeBlockPath =
+              new File(sourceRoot).toURI().relativize(sourceBlock.toURI())
+                  .getPath();
+          String relativeMetaPath =
+              new File(sourceRoot).toURI().relativize(sourceMeta.toURI())
+                  .getPath();
+
+          File destBlock = new File(destRoot, relativeBlockPath);
+          File destMeta = new File(destRoot, relativeMetaPath);
+
+          destBlock.getParentFile().mkdirs();
+          FileUtils.copyFile(sourceBlock, destBlock);
+          FileUtils.copyFile(sourceMeta, destMeta);
+
+          if (destBlock.exists() && destMeta.exists()) {
+            LOG.info("Copied " + sourceBlock + " ==> " + destBlock);
+            LOG.info("Copied " + sourceMeta + " ==> " + destMeta);
+          }
         }
       }
     }
@@ -209,58 +216,67 @@ public class TestDirectoryScanner {
 
   /** Create a block file in a random volume*/
   private long createBlockFile() throws IOException {
-    List<? extends FsVolumeSpi> volumes = fds.getVolumes();
-    int index = rand.nextInt(volumes.size() - 1);
     long id = getFreeBlockId();
-    File finalizedDir = volumes.get(index).getFinalizedDir(bpid);
-    File file = new File(finalizedDir, getBlockFile(id));
-    if (file.createNewFile()) {
-      LOG.info("Created block file " + file.getName());
+    try (FsDatasetSpi.FsVolumeReferences volumes = fds.getFsVolumeReferences()) {
+      int numVolumes = volumes.size();
+      int index = rand.nextInt(numVolumes - 1);
+      File finalizedDir = volumes.get(index).getFinalizedDir(bpid);
+      File file = new File(finalizedDir, getBlockFile(id));
+      if (file.createNewFile()) {
+        LOG.info("Created block file " + file.getName());
+      }
     }
     return id;
   }
 
   /** Create a metafile in a random volume*/
   private long createMetaFile() throws IOException {
-    List<? extends FsVolumeSpi> volumes = fds.getVolumes();
-    int index = rand.nextInt(volumes.size() - 1);
     long id = getFreeBlockId();
-    File finalizedDir = volumes.get(index).getFinalizedDir(bpid);
-    File file = new File(finalizedDir, getMetaFile(id));
-    if (file.createNewFile()) {
-      LOG.info("Created metafile " + file.getName());
+    try (FsDatasetSpi.FsVolumeReferences refs = fds.getFsVolumeReferences()) {
+      int numVolumes = refs.size();
+      int index = rand.nextInt(numVolumes - 1);
+
+      File finalizedDir = refs.get(index).getFinalizedDir(bpid);
+      File file = new File(finalizedDir, getMetaFile(id));
+      if (file.createNewFile()) {
+        LOG.info("Created metafile " + file.getName());
+      }
     }
     return id;
   }
 
   /** Create block file and corresponding metafile in a rondom volume */
   private long createBlockMetaFile() throws IOException {
-    List<? extends FsVolumeSpi> volumes = fds.getVolumes();
-    int index = rand.nextInt(volumes.size() - 1);
     long id = getFreeBlockId();
-    File finalizedDir = volumes.get(index).getFinalizedDir(bpid);
-    File file = new File(finalizedDir, getBlockFile(id));
-    if (file.createNewFile()) {
-      LOG.info("Created block file " + file.getName());
-
-      // Create files with same prefix as block file but extension names
-      // such that during sorting, these files appear around meta file
-      // to test how DirectoryScanner handles extraneous files
-      String name1 = file.getAbsolutePath() + ".l";
-      String name2 = file.getAbsolutePath() + ".n";
-      file = new File(name1);
-      if (file.createNewFile()) {
-        LOG.info("Created extraneous file " + name1);
-      }
 
-      file = new File(name2);
-      if (file.createNewFile()) {
-        LOG.info("Created extraneous file " + name2);
-      }
+    try (FsDatasetSpi.FsVolumeReferences refs = fds.getFsVolumeReferences()) {
+      int numVolumes = refs.size();
+      int index = rand.nextInt(numVolumes - 1);
 
-      file = new File(finalizedDir, getMetaFile(id));
+      File finalizedDir = refs.get(index).getFinalizedDir(bpid);
+      File file = new File(finalizedDir, getBlockFile(id));
       if (file.createNewFile()) {
-        LOG.info("Created metafile " + file.getName());
+        LOG.info("Created block file " + file.getName());
+
+        // Create files with same prefix as block file but extension names
+        // such that during sorting, these files appear around meta file
+        // to test how DirectoryScanner handles extraneous files
+        String name1 = file.getAbsolutePath() + ".l";
+        String name2 = file.getAbsolutePath() + ".n";
+        file = new File(name1);
+        if (file.createNewFile()) {
+          LOG.info("Created extraneous file " + name1);
+        }
+
+        file = new File(name2);
+        if (file.createNewFile()) {
+          LOG.info("Created extraneous file " + name2);
+        }
+
+        file = new File(finalizedDir, getMetaFile(id));
+        if (file.createNewFile()) {
+          LOG.info("Created metafile " + file.getName());
+        }
       }
     }
     return id;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java
index 3ec76db..cc8566c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.util.DataChecksum;
@@ -190,12 +191,15 @@ public class TestDiskError {
     // Check permissions on directories in 'dfs.datanode.data.dir'
     FileSystem localFS = FileSystem.getLocal(conf);
     for (DataNode dn : cluster.getDataNodes()) {
-      for (FsVolumeSpi v : dn.getFSDataset().getVolumes()) {
-        String dir = v.getBasePath();
-        Path dataDir = new Path(dir);
-        FsPermission actual = localFS.getFileStatus(dataDir).getPermission();
+      try (FsDatasetSpi.FsVolumeReferences volumes =
+          dn.getFSDataset().getFsVolumeReferences()) {
+        for (FsVolumeSpi vol : volumes) {
+          String dir = vol.getBasePath();
+          Path dataDir = new Path(dir);
+          FsPermission actual = localFS.getFileStatus(dataDir).getPermission();
           assertEquals("Permission for dir: " + dataDir + ", is " + actual +
               ", while expected is " + expected, expected, actual);
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBlockReports.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBlockReports.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBlockReports.java
index f27a78e..de73dcf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBlockReports.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBlockReports.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.*;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
@@ -71,7 +72,10 @@ public class TestIncrementalBlockReports {
     singletonDn = cluster.getDataNodes().get(0);
     bpos = singletonDn.getAllBpOs().get(0);
     actor = bpos.getBPServiceActors().get(0);
-    storageUuid = singletonDn.getFSDataset().getVolumes().get(0).getStorageID();
+    try (FsDatasetSpi.FsVolumeReferences volumes =
+        singletonDn.getFSDataset().getFsVolumeReferences()) {
+      storageUuid = volumes.get(0).getStorageID();
+    }
   }
 
   private static Block getDummyBlock() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java
index 4e73e6e..989e216 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java
@@ -29,7 +29,6 @@ import java.util.UUID;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSClient;
@@ -39,6 +38,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -142,48 +142,55 @@ public class TestIncrementalBrVariations {
     // Get the block list for the file with the block locations.
     LocatedBlocks blocks = createFileGetBlocks(GenericTestUtils.getMethodName());
 
-    // We will send 'fake' incremental block reports to the NN that look
-    // like they originated from DN 0.
-    StorageReceivedDeletedBlocks reports[] =
-        new StorageReceivedDeletedBlocks[dn0.getFSDataset().getVolumes().size()];
-
-    // Lie to the NN that one block on each storage has been deleted.
-    for (int i = 0; i < reports.length; ++i) {
-      FsVolumeSpi volume = dn0.getFSDataset().getVolumes().get(i);
-
-      boolean foundBlockOnStorage = false;
-      ReceivedDeletedBlockInfo rdbi[] = new ReceivedDeletedBlockInfo[1];
-
-      // Find the first block on this storage and mark it as deleted for the
-      // report.
-      for (LocatedBlock block : blocks.getLocatedBlocks()) {
-        if (block.getStorageIDs()[0].equals(volume.getStorageID())) {
-          rdbi[0] = new ReceivedDeletedBlockInfo(block.getBlock().getLocalBlock(),
-              ReceivedDeletedBlockInfo.BlockStatus.DELETED_BLOCK, null);
-          foundBlockOnStorage = true;
-          break;
+    try (FsDatasetSpi.FsVolumeReferences volumes
+        = dn0.getFSDataset().getFsVolumeReferences()) {
+      // We will send 'fake' incremental block reports to the NN that look
+      // like they originated from DN 0.
+      StorageReceivedDeletedBlocks reports[] =
+          new StorageReceivedDeletedBlocks[volumes.size()];
+
+      // Lie to the NN that one block on each storage has been deleted.
+      for (int i = 0; i < reports.length; ++i) {
+        FsVolumeSpi volume = volumes.get(i);
+
+        boolean foundBlockOnStorage = false;
+        ReceivedDeletedBlockInfo rdbi[] = new ReceivedDeletedBlockInfo[1];
+
+        // Find the first block on this storage and mark it as deleted for the
+        // report.
+        for (LocatedBlock block : blocks.getLocatedBlocks()) {
+          if (block.getStorageIDs()[0].equals(volume.getStorageID())) {
+            rdbi[0] =
+                new ReceivedDeletedBlockInfo(block.getBlock().getLocalBlock(),
+                    ReceivedDeletedBlockInfo.BlockStatus.DELETED_BLOCK, null);
+            foundBlockOnStorage = true;
+            break;
+          }
         }
-      }
 
-      assertTrue(foundBlockOnStorage);
-      reports[i] = new StorageReceivedDeletedBlocks(volume.getStorageID(), rdbi);
+        assertTrue(foundBlockOnStorage);
+        reports[i] =
+            new StorageReceivedDeletedBlocks(volume.getStorageID(), rdbi);
 
-      if (splitReports) {
-        // If we are splitting reports then send the report for this storage now.
-        StorageReceivedDeletedBlocks singletonReport[] = { reports[i] };
-        cluster.getNameNodeRpc().blockReceivedAndDeleted(
-            dn0Reg, poolId, singletonReport);
+        if (splitReports) {
+          // If we are splitting reports then send the report for this storage now.
+          StorageReceivedDeletedBlocks singletonReport[] = { reports[i] };
+          cluster.getNameNodeRpc().blockReceivedAndDeleted(
+              dn0Reg, poolId, singletonReport);
+        }
       }
-    }
 
-    if (!splitReports) {
-      // Send a combined report.
-      cluster.getNameNodeRpc().blockReceivedAndDeleted(dn0Reg, poolId, reports);
-    }
+      if (!splitReports) {
+        // Send a combined report.
+        cluster.getNameNodeRpc()
+            .blockReceivedAndDeleted(dn0Reg, poolId, reports);
+      }
 
-    // Make sure that the deleted block from each storage was picked up
-    // by the NameNode.
-    assertThat(cluster.getNamesystem().getMissingBlocksCount(), is((long) reports.length));
+      // Make sure that the deleted block from each storage was picked up
+      // by the NameNode.
+      assertThat(cluster.getNamesystem().getMissingBlocksCount(),
+          is((long) reports.length));
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTriggerBlockReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTriggerBlockReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTriggerBlockReport.java
index c2348e3..00c0f22 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTriggerBlockReport.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTriggerBlockReport.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.client.BlockReportOptions;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
@@ -92,8 +93,11 @@ public final class TestTriggerBlockReport {
     DataNode datanode = cluster.getDataNodes().get(0);
     BPServiceActor actor =
         datanode.getAllBpOs().get(0).getBPServiceActors().get(0);
-    String storageUuid =
-        datanode.getFSDataset().getVolumes().get(0).getStorageID();
+    String storageUuid;
+    try (FsDatasetSpi.FsVolumeReferences volumes =
+        datanode.getFSDataset().getFsVolumeReferences()) {
+      storageUuid = volumes.get(0).getStorageID();
+    }
     actor.notifyNamenodeDeletedBlock(rdbi, storageUuid);
 
     // Manually trigger a block report.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
index b7c2028..28666a0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
@@ -51,7 +51,7 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
       StorageType.DEFAULT);
 
   @Override
-  public List<ExternalVolumeImpl> getVolumes() {
+  public FsVolumeReferences getFsVolumeReferences() {
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
index 7e1aa81..5dc86f7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
@@ -43,7 +43,6 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.HashSet;
-import java.util.List;
 import java.util.Set;
 import java.util.UUID;
 
@@ -62,11 +61,11 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.tools.JMXGet;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
@@ -155,30 +154,34 @@ public abstract class LazyPersistTestCase {
   protected final void ensureLazyPersistBlocksAreSaved(
       LocatedBlocks locatedBlocks) throws IOException, InterruptedException {
     final String bpid = cluster.getNamesystem().getBlockPoolId();
-    List<? extends FsVolumeSpi> volumes =
-      cluster.getDataNodes().get(0).getFSDataset().getVolumes();
-    final Set<Long> persistedBlockIds = new HashSet<Long>();
 
-    while (persistedBlockIds.size() < locatedBlocks.getLocatedBlocks().size()) {
-      // Take 1 second sleep before each verification iteration
-      Thread.sleep(1000);
-
-      for (LocatedBlock lb : locatedBlocks.getLocatedBlocks()) {
-        for (FsVolumeSpi v : volumes) {
-          if (v.isTransientStorage()) {
-            continue;
-          }
-
-          FsVolumeImpl volume = (FsVolumeImpl) v;
-          File lazyPersistDir = volume.getBlockPoolSlice(bpid).getLazypersistDir();
+    final Set<Long> persistedBlockIds = new HashSet<Long>();
 
-          long blockId = lb.getBlock().getBlockId();
-          File targetDir =
-            DatanodeUtil.idToBlockDir(lazyPersistDir, blockId);
-          File blockFile = new File(targetDir, lb.getBlock().getBlockName());
-          if (blockFile.exists()) {
-            // Found a persisted copy for this block and added to the Set
-            persistedBlockIds.add(blockId);
+    try (FsDatasetSpi.FsVolumeReferences volumes =
+        cluster.getDataNodes().get(0).getFSDataset().getFsVolumeReferences()) {
+      while (persistedBlockIds.size() < locatedBlocks.getLocatedBlocks()
+          .size()) {
+        // Take 1 second sleep before each verification iteration
+        Thread.sleep(1000);
+
+        for (LocatedBlock lb : locatedBlocks.getLocatedBlocks()) {
+          for (FsVolumeSpi v : volumes) {
+            if (v.isTransientStorage()) {
+              continue;
+            }
+
+            FsVolumeImpl volume = (FsVolumeImpl) v;
+            File lazyPersistDir =
+                volume.getBlockPoolSlice(bpid).getLazypersistDir();
+
+            long blockId = lb.getBlock().getBlockId();
+            File targetDir =
+                DatanodeUtil.idToBlockDir(lazyPersistDir, blockId);
+            File blockFile = new File(targetDir, lb.getBlock().getBlockName());
+            if (blockFile.exists()) {
+              // Found a persisted copy for this block and added to the Set
+              persistedBlockIds.add(blockId);
+            }
           }
         }
       }
@@ -432,18 +435,21 @@ public abstract class LazyPersistTestCase {
     }
 
     final String bpid = cluster.getNamesystem().getBlockPoolId();
-    List<? extends FsVolumeSpi> volumes =
-      cluster.getDataNodes().get(0).getFSDataset().getVolumes();
+    final FsDatasetSpi<?> dataset =
+        cluster.getDataNodes().get(0).getFSDataset();
 
     // Make sure deleted replica does not have a copy on either finalized dir of
     // transient volume or finalized dir of non-transient volume
-    for (FsVolumeSpi v : volumes) {
-      FsVolumeImpl volume = (FsVolumeImpl) v;
-      File targetDir = (v.isTransientStorage()) ?
-          volume.getBlockPoolSlice(bpid).getFinalizedDir() :
-          volume.getBlockPoolSlice(bpid).getLazypersistDir();
-      if (verifyBlockDeletedFromDir(targetDir, locatedBlocks) == false) {
-        return false;
+    try (FsDatasetSpi.FsVolumeReferences volumes =
+        dataset.getFsVolumeReferences()) {
+      for (FsVolumeSpi vol : volumes) {
+        FsVolumeImpl volume = (FsVolumeImpl) vol;
+        File targetDir = (volume.isTransientStorage()) ?
+            volume.getBlockPoolSlice(bpid).getFinalizedDir() :
+            volume.getBlockPoolSlice(bpid).getLazypersistDir();
+        if (verifyBlockDeletedFromDir(targetDir, locatedBlocks) == false) {
+          return false;
+        }
       }
     }
     return true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestDatanodeRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestDatanodeRestart.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestDatanodeRestart.java
index a91baec..a5a09fa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestDatanodeRestart.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestDatanodeRestart.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.io.IOUtils;
 import org.junit.Assert;
@@ -101,13 +102,18 @@ public class TestDatanodeRestart {
       out.write(writeBuf);
       out.hflush();
       DataNode dn = cluster.getDataNodes().get(0);
-      for (FsVolumeSpi v : dataset(dn).getVolumes()) {
-        final FsVolumeImpl volume = (FsVolumeImpl)v;
-        File currentDir = volume.getCurrentDir().getParentFile().getParentFile();
-        File rbwDir = new File(currentDir, "rbw");
-        for (File file : rbwDir.listFiles()) {
-          if (isCorrupt && Block.isBlockFilename(file)) {
-            new RandomAccessFile(file, "rw").setLength(fileLen-1); // corrupt
+      try (FsDatasetSpi.FsVolumeReferences volumes =
+          dataset(dn).getFsVolumeReferences()) {
+        for (FsVolumeSpi vol : volumes) {
+          final FsVolumeImpl volume = (FsVolumeImpl) vol;
+          File currentDir =
+              volume.getCurrentDir().getParentFile().getParentFile();
+          File rbwDir = new File(currentDir, "rbw");
+          for (File file : rbwDir.listFiles()) {
+            if (isCorrupt && Block.isBlockFilename(file)) {
+              new RandomAccessFile(file, "rw")
+                  .setLength(fileLen - 1); // corrupt
+            }
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/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 56a4287..9f4f700 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
@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaHandler;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.RoundRobinVolumeChoosingPolicy;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
@@ -56,7 +57,6 @@ import org.mockito.stubbing.Answer;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
@@ -68,7 +68,6 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyList;
 import static org.mockito.Matchers.anyListOf;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Matchers.eq;
@@ -124,6 +123,15 @@ public class TestFsDatasetImpl {
     when(storage.getNumStorageDirs()).thenReturn(numDirs);
   }
 
+  private int getNumVolumes() {
+    try (FsDatasetSpi.FsVolumeReferences volumes =
+        dataset.getFsVolumeReferences()) {
+      return volumes.size();
+    } catch (IOException e) {
+      return 0;
+    }
+  }
+
   @Before
   public void setUp() throws IOException {
     datanode = mock(DataNode.class);
@@ -143,14 +151,14 @@ public class TestFsDatasetImpl {
       dataset.addBlockPool(bpid, conf);
     }
 
-    assertEquals(NUM_INIT_VOLUMES, dataset.getVolumes().size());
+    assertEquals(NUM_INIT_VOLUMES, getNumVolumes());
     assertEquals(0, dataset.getNumFailedVolumes());
   }
 
   @Test
   public void testAddVolumes() throws IOException {
     final int numNewVolumes = 3;
-    final int numExistingVolumes = dataset.getVolumes().size();
+    final int numExistingVolumes = getNumVolumes();
     final int totalVolumes = numNewVolumes + numExistingVolumes;
     Set<String> expectedVolumes = new HashSet<String>();
     List<NamespaceInfo> nsInfos = Lists.newArrayList();
@@ -172,13 +180,15 @@ public class TestFsDatasetImpl {
       dataset.addVolume(loc, nsInfos);
     }
 
-    assertEquals(totalVolumes, dataset.getVolumes().size());
+    assertEquals(totalVolumes, getNumVolumes());
     assertEquals(totalVolumes, dataset.storageMap.size());
 
     Set<String> actualVolumes = new HashSet<String>();
-    for (int i = 0; i < numNewVolumes; i++) {
-      actualVolumes.add(
-          dataset.getVolumes().get(numExistingVolumes + i).getBasePath());
+    try (FsDatasetSpi.FsVolumeReferences volumes =
+        dataset.getFsVolumeReferences()) {
+      for (int i = 0; i < numNewVolumes; i++) {
+        actualVolumes.add(volumes.get(numExistingVolumes + i).getBasePath());
+      }
     }
     assertEquals(actualVolumes.size(), expectedVolumes.size());
     assertTrue(actualVolumes.containsAll(expectedVolumes));
@@ -204,7 +214,7 @@ public class TestFsDatasetImpl {
     dataset.removeVolumes(volumesToRemove, true);
     int expectedNumVolumes = dataDirs.length - 1;
     assertEquals("The volume has been removed from the volumeList.",
-        expectedNumVolumes, dataset.getVolumes().size());
+        expectedNumVolumes, getNumVolumes());
     assertEquals("The volume has been removed from the storageMap.",
         expectedNumVolumes, dataset.storageMap.size());
 
@@ -231,7 +241,7 @@ public class TestFsDatasetImpl {
 
   @Test(timeout = 5000)
   public void testRemoveNewlyAddedVolume() throws IOException {
-    final int numExistingVolumes = dataset.getVolumes().size();
+    final int numExistingVolumes = getNumVolumes();
     List<NamespaceInfo> nsInfos = new ArrayList<>();
     for (String bpid : BLOCK_POOL_IDS) {
       nsInfos.add(new NamespaceInfo(0, CLUSTER_ID, bpid, 1));
@@ -247,14 +257,14 @@ public class TestFsDatasetImpl {
         .thenReturn(builder);
 
     dataset.addVolume(loc, nsInfos);
-    assertEquals(numExistingVolumes + 1, dataset.getVolumes().size());
+    assertEquals(numExistingVolumes + 1, getNumVolumes());
 
     when(storage.getNumStorageDirs()).thenReturn(numExistingVolumes + 1);
     when(storage.getStorageDir(numExistingVolumes)).thenReturn(sd);
     Set<File> volumesToRemove = new HashSet<>();
     volumesToRemove.add(loc.getFile());
     dataset.removeVolumes(volumesToRemove, true);
-    assertEquals(numExistingVolumes, dataset.getVolumes().size());
+    assertEquals(numExistingVolumes, getNumVolumes());
   }
 
   @Test(timeout = 5000)
@@ -357,7 +367,10 @@ public class TestFsDatasetImpl {
       DataNode dn = cluster.getDataNodes().get(0);
       
       FsDatasetImpl ds = (FsDatasetImpl) DataNodeTestUtils.getFSDataset(dn);
-      FsVolumeImpl vol = ds.getVolumes().get(0);
+      FsVolumeImpl vol;
+      try (FsDatasetSpi.FsVolumeReferences volumes = ds.getFsVolumeReferences()) {
+        vol = (FsVolumeImpl)volumes.get(0);
+      }
 
       ExtendedBlock eb;
       ReplicaInfo info;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsVolumeList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsVolumeList.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsVolumeList.java
index eccff89..9b9b692 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsVolumeList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsVolumeList.java
@@ -35,7 +35,7 @@ import java.util.Collections;
 import java.util.List;
 
 import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.assertNull;
 import static org.mockito.Mockito.mock;
 
 public class TestFsVolumeList {
@@ -113,11 +113,6 @@ public class TestFsVolumeList {
         conf, StorageType.DEFAULT);
     FsVolumeReference ref = volume.obtainReference();
     volumeList.addVolume(ref);
-    try {
-      ref.close();
-      fail("Should throw exception because the reference is closed in "
-          + "VolumeList#addVolume().");
-    } catch (IllegalStateException e) {
-    }
+    assertNull(ref.getVolume());
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/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 ebf2f3b..9b83b39 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
@@ -32,7 +32,8 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.*;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Daemon;
@@ -44,7 +45,6 @@ import org.junit.rules.ExpectedException;
 
 import java.io.IOException;
 import java.io.OutputStream;
-import java.util.List;
 import java.util.Random;
 import java.util.concurrent.TimeoutException;
 
@@ -64,6 +64,7 @@ public class TestRbwSpaceReservation {
   private Configuration conf;
   private DistributedFileSystem fs = null;
   private DFSClient client = null;
+  FsVolumeReference singletonVolumeRef = null;
   FsVolumeImpl singletonVolume = null;
 
   private static Random rand = new Random();
@@ -104,23 +105,22 @@ public class TestRbwSpaceReservation {
     cluster.waitActive();
 
     if (perVolumeCapacity >= 0) {
-      for (DataNode dn : cluster.getDataNodes()) {
-        for (FsVolumeSpi volume : dn.getFSDataset().getVolumes()) {
-          ((FsVolumeImpl) volume).setCapacityForTesting(perVolumeCapacity);
-        }
+      try (FsDatasetSpi.FsVolumeReferences volumes =
+          cluster.getDataNodes().get(0).getFSDataset().getFsVolumeReferences()) {
+        singletonVolumeRef = volumes.get(0).obtainReference();
       }
-    }
-
-    if (numDatanodes == 1) {
-      List<? extends FsVolumeSpi> volumes =
-          cluster.getDataNodes().get(0).getFSDataset().getVolumes();
-      assertThat(volumes.size(), is(1));
-      singletonVolume = ((FsVolumeImpl) volumes.get(0));
+      singletonVolume = ((FsVolumeImpl) singletonVolumeRef.getVolume());
+      singletonVolume.setCapacityForTesting(perVolumeCapacity);
     }
   }
 
   @After
   public void shutdownCluster() throws IOException {
+    if (singletonVolumeRef != null) {
+      singletonVolumeRef.close();
+      singletonVolumeRef = null;
+    }
+
     if (client != null) {
       client.close();
       client = null;
@@ -266,13 +266,16 @@ public class TestRbwSpaceReservation {
     // Ensure all space reserved for the replica was released on each
     // DataNode.
     for (DataNode dn : cluster.getDataNodes()) {
-      final FsVolumeImpl volume = (FsVolumeImpl) dn.getFSDataset().getVolumes().get(0);
-      GenericTestUtils.waitFor(new Supplier<Boolean>() {
-        @Override
-        public Boolean get() {
-          return (volume.getReservedForRbw() == 0);
-        }
-      }, 500, Integer.MAX_VALUE); // Wait until the test times out.
+      try (FsDatasetSpi.FsVolumeReferences volumes =
+          dn.getFSDataset().getFsVolumeReferences()) {
+        final FsVolumeImpl volume = (FsVolumeImpl) volumes.get(0);
+        GenericTestUtils.waitFor(new Supplier<Boolean>() {
+          @Override
+          public Boolean get() {
+            return (volume.getReservedForRbw() == 0);
+          }
+        }, 500, Integer.MAX_VALUE); // Wait until the test times out.
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java
index 96a73c6..d5664cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java
@@ -45,6 +45,8 @@ import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaUnderRecovery;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaWaitingToBeRecovered;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 import org.junit.Assert;
@@ -161,31 +163,37 @@ public class TestWriteToReplica {
     };
     
     ReplicaMap replicasMap = dataSet.volumeMap;
-    FsVolumeImpl vol = (FsVolumeImpl) dataSet.volumes
-        .getNextVolume(StorageType.DEFAULT, 0).getVolume();
-    ReplicaInfo replicaInfo = new FinalizedReplica(
-        blocks[FINALIZED].getLocalBlock(), vol, vol.getCurrentDir().getParentFile());
-    replicasMap.add(bpid, replicaInfo);
-    replicaInfo.getBlockFile().createNewFile();
-    replicaInfo.getMetaFile().createNewFile();
-    
-    replicasMap.add(bpid, new ReplicaInPipeline(
-        blocks[TEMPORARY].getBlockId(),
-        blocks[TEMPORARY].getGenerationStamp(), vol,
-        vol.createTmpFile(bpid, blocks[TEMPORARY].getLocalBlock()).getParentFile(), 0));
-    
-    replicaInfo = new ReplicaBeingWritten(blocks[RBW].getLocalBlock(), vol,
-        vol.createRbwFile(bpid, blocks[RBW].getLocalBlock()).getParentFile(), null);
-    replicasMap.add(bpid, replicaInfo);
-    replicaInfo.getBlockFile().createNewFile();
-    replicaInfo.getMetaFile().createNewFile();
-    
-    replicasMap.add(bpid, new ReplicaWaitingToBeRecovered(
-        blocks[RWR].getLocalBlock(), vol, vol.createRbwFile(bpid,
-            blocks[RWR].getLocalBlock()).getParentFile()));
-    replicasMap.add(bpid, new ReplicaUnderRecovery(new FinalizedReplica(blocks[RUR]
-        .getLocalBlock(), vol, vol.getCurrentDir().getParentFile()), 2007));    
-    
+    try (FsDatasetSpi.FsVolumeReferences references =
+        dataSet.getFsVolumeReferences()) {
+      FsVolumeImpl vol = (FsVolumeImpl) references.get(0);
+      ReplicaInfo replicaInfo = new FinalizedReplica(
+          blocks[FINALIZED].getLocalBlock(), vol,
+          vol.getCurrentDir().getParentFile());
+      replicasMap.add(bpid, replicaInfo);
+      replicaInfo.getBlockFile().createNewFile();
+      replicaInfo.getMetaFile().createNewFile();
+
+      replicasMap.add(bpid, new ReplicaInPipeline(
+          blocks[TEMPORARY].getBlockId(),
+          blocks[TEMPORARY].getGenerationStamp(), vol,
+          vol.createTmpFile(bpid, blocks[TEMPORARY].getLocalBlock())
+              .getParentFile(), 0));
+
+      replicaInfo = new ReplicaBeingWritten(blocks[RBW].getLocalBlock(), vol,
+          vol.createRbwFile(bpid, blocks[RBW].getLocalBlock()).getParentFile(),
+          null);
+      replicasMap.add(bpid, replicaInfo);
+      replicaInfo.getBlockFile().createNewFile();
+      replicaInfo.getMetaFile().createNewFile();
+
+      replicasMap.add(bpid, new ReplicaWaitingToBeRecovered(
+          blocks[RWR].getLocalBlock(), vol, vol.createRbwFile(bpid,
+          blocks[RWR].getLocalBlock()).getParentFile()));
+      replicasMap
+          .add(bpid, new ReplicaUnderRecovery(new FinalizedReplica(blocks[RUR]
+              .getLocalBlock(), vol, vol.getCurrentDir().getParentFile()),
+              2007));
+    }
     return blocks;
   }
   
@@ -538,9 +546,15 @@ public class TestWriteToReplica {
           getFSDataset(dn);
       ReplicaMap replicaMap = dataSet.volumeMap;
       
-      List<FsVolumeImpl> volumes = dataSet.getVolumes();
-      // number of volumes should be 2 - [data1, data2]
-      assertEquals("number of volumes is wrong", 2, volumes.size());
+      List<FsVolumeImpl> volumes = null;
+      try (FsDatasetSpi.FsVolumeReferences referredVols = dataSet.getFsVolumeReferences()) {
+        // number of volumes should be 2 - [data1, data2]
+        assertEquals("number of volumes is wrong", 2, referredVols.size());
+        volumes = new ArrayList<>(referredVols.size());
+        for (FsVolumeSpi vol : referredVols) {
+          volumes.add((FsVolumeImpl) vol);
+        }
+      }
       ArrayList<String> bpList = new ArrayList<String>(Arrays.asList(
           cluster.getNamesystem(0).getBlockPoolId(), 
           cluster.getNamesystem(1).getBlockPoolId()));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24d3a2d4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
index a5d5848..d8b40d4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
@@ -627,14 +628,18 @@ public class TestStorageMover {
   }
 
   private void setVolumeFull(DataNode dn, StorageType type) {
-    List<? extends FsVolumeSpi> volumes = dn.getFSDataset().getVolumes();
-    for (FsVolumeSpi v : volumes) {
-      FsVolumeImpl volume = (FsVolumeImpl) v;
-      if (volume.getStorageType() == type) {
-        LOG.info("setCapacity to 0 for [" + volume.getStorageType() + "]"
-            + volume.getStorageID());
-        volume.setCapacityForTesting(0);
+    try (FsDatasetSpi.FsVolumeReferences refs = dn.getFSDataset()
+        .getFsVolumeReferences()) {
+      for (FsVolumeSpi fvs : refs) {
+        FsVolumeImpl volume = (FsVolumeImpl) fvs;
+        if (volume.getStorageType() == type) {
+          LOG.info("setCapacity to 0 for [" + volume.getStorageType() + "]"
+              + volume.getStorageID());
+          volume.setCapacityForTesting(0);
+        }
       }
+    } catch (IOException e) {
+      LOG.error("Unexpected exception by closing FsVolumeReference", e);
     }
   }
 


[06/17] hadoop git commit: HADOOP-11911. test-patch should allow configuration of default branch (Sean Busbey via aw)

Posted by ji...@apache.org.
HADOOP-11911. test-patch should allow configuration of default branch (Sean Busbey via aw)


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

Branch: refs/heads/HDFS-7240
Commit: 9b01f81eb874cd63e6b9ae2d09d94fc8bf4fcd7d
Parents: 3810242
Author: Allen Wittenauer <aw...@apache.org>
Authored: Tue May 5 09:59:20 2015 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Tue May 5 09:59:31 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9b01f81e/dev-support/test-patch.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.sh b/dev-support/test-patch.sh
index b6e1b03..5107718 100755
--- a/dev-support/test-patch.sh
+++ b/dev-support/test-patch.sh
@@ -43,6 +43,7 @@ function setup_defaults
   ECLIPSE_HOME=${ECLIPSE_HOME:-}
   BUILD_NATIVE=${BUILD_NATIVE:-true}
   PATCH_BRANCH=""
+  PATCH_BRANCH_DEFAULT="trunk"
   CHANGED_MODULES=""
   USER_MODULE_LIST=""
   OFFLINE=false
@@ -551,7 +552,8 @@ function hadoop_usage
   echo
   echo "Options:"
   echo "--basedir=<dir>        The directory to apply the patch to (default current directory)"
-  echo "--branch=<dir>         Forcibly set the branch"
+  echo "--branch=<ref>         Forcibly set the branch"
+  echo "--branch-default=<ref> If the branch isn't forced and we don't detect one in the patch name, use this branch (default 'trunk')"
   echo "--build-native=<bool>  If true, then build native components (default 'true')"
   echo "--debug                If set, then output some extra stuff to stderr"
   echo "--dirty-workspace      Allow the local git workspace to have uncommitted changes"
@@ -604,6 +606,9 @@ function parse_args
       --branch=*)
         PATCH_BRANCH=${i#*=}
       ;;
+      --branch-default=*)
+        PATCH_BRANCH_DEFAULT=${i#*=}
+      ;;
       --build-native=*)
         BUILD_NATIVE=${i#*=}
       ;;
@@ -832,9 +837,9 @@ function git_checkout
       cleanup_and_exit 1
     fi
 
-    ${GIT} checkout --force trunk
+    ${GIT} checkout --force "${PATCH_BRANCH_DEFAULT}"
     if [[ $? != 0 ]]; then
-      hadoop_error "ERROR: git checkout --force trunk is failing"
+      hadoop_error "ERROR: git checkout --force ${PATCH_BRANCH_DEFAULT} is failing"
       cleanup_and_exit 1
     fi
 
@@ -859,8 +864,8 @@ function git_checkout
       cleanup_and_exit 1
     fi
 
-    # we need to explicitly fetch in case the
-    # git ref hasn't been brought in tree yet
+    # if we've selected a feature branch that has new changes
+    # since our last build, we'll need to rebase to see those changes.
     if [[ ${OFFLINE} == false ]]; then
       ${GIT} pull --rebase
       if [[ $? != 0 ]]; then
@@ -1011,7 +1016,7 @@ function verify_valid_branch
 ## @stability    evolving
 ## @replaceable  no
 ## @return       0 on success, with PATCH_BRANCH updated appropriately
-## @return       1 on failure, with PATCH_BRANCH updated to "trunk"
+## @return       1 on failure, with PATCH_BRANCH updated to PATCH_BRANCH_DEFAULT
 function determine_branch
 {
   local allbranches
@@ -1075,7 +1080,7 @@ function determine_branch
     fi
   done
 
-  PATCH_BRANCH=trunk
+  PATCH_BRANCH="${PATCH_BRANCH_DEFAULT}"
 
   popd >/dev/null
 }
@@ -1365,7 +1370,7 @@ function check_reexec
 
   exec "${PATCH_DIR}/dev-support-test/test-patch.sh" \
     --reexec \
-    --branch ${PATCH_BRANCH} \
+    --branch "${PATCH_BRANCH}" \
     --patch-dir="${PATCH_DIR}" \
       "${USER_PARAMS[@]}"
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9b01f81e/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 8b0e67c..49106ae 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -530,6 +530,9 @@ Release 2.8.0 - UNRELEASED
 
     HADOOP-11120. hadoop fs -rmr gives wrong advice. (Juliet Hougland via wang)
 
+    HADOOP-11911. test-patch should allow configuration of default branch
+    (Sean Busbey via aw)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp


[07/17] hadoop git commit: YARN-3396. Handle URISyntaxException in ResourceLocalizationService. (Contributed by Brahma Reddy Battula)

Posted by ji...@apache.org.
YARN-3396. Handle URISyntaxException in ResourceLocalizationService. (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/38102420
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/38102420
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/38102420

Branch: refs/heads/HDFS-7240
Commit: 38102420621308f5ba91cdeb6a18a63aa5acf640
Parents: 05adc76
Author: Junping Du <ju...@apache.org>
Authored: Tue May 5 10:18:23 2015 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Tue May 5 10:18:23 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                              | 2 ++
 .../localizer/ResourceLocalizationService.java               | 8 ++++++--
 2 files changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/38102420/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 97d3208..a6b7f17 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -180,6 +180,8 @@ Release 2.8.0 - UNRELEASED
     YARN-3363. add localization and container launch time to ContainerMetrics
     at NM to show these timing information for each active container.
     (zxu via rkanter)
+    YARN-3396. Handle URISyntaxException in ResourceLocalizationService. 
+    (Brahma Reddy Battula via junping_du)
 
   OPTIMIZATIONS
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38102420/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.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/localizer/ResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
index e9c45f3..17ea1a9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
@@ -995,7 +995,9 @@ public class ResourceLocalizationService extends CompositeService
         try {
           req = new LocalResourceRequest(rsrc);
         } catch (URISyntaxException e) {
-          // TODO fail? Already translated several times...
+          LOG.error(
+              "Got exception in parsing URL of LocalResource:"
+                  + rsrc.getResource(), e);
         }
         LocalizerResourceRequestEvent assoc = scheduled.get(req);
         if (assoc == null) {
@@ -1069,7 +1071,9 @@ public class ResourceLocalizationService extends CompositeService
           LOG.error("Inorrect path for PRIVATE localization."
               + next.getResource().getFile(), e);
         } catch (URISyntaxException e) {
-            //TODO fail? Already translated several times...
+          LOG.error(
+              "Got exception in parsing URL of LocalResource:"
+                  + next.getResource(), e);
         }
       }
 


[16/17] hadoop git commit: MAPREDUCE-6192. Create unit test to automatically compare MR related classes and mapred-default.xml (rchiang via rkanter)

Posted by ji...@apache.org.
MAPREDUCE-6192. Create unit test to automatically compare MR related classes and mapred-default.xml (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/9809a16d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9809a16d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9809a16d

Branch: refs/heads/HDFS-7240
Commit: 9809a16d3c8068beccbf0106e99c7ede6ba11e0f
Parents: 0100b15
Author: Robert Kanter <rk...@apache.org>
Authored: Mon May 4 17:48:10 2015 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Tue May 5 14:43:28 2015 -0700

----------------------------------------------------------------------
 .../conf/TestConfigurationFieldsBase.java       | 58 ++++++++++++++-
 hadoop-mapreduce-project/CHANGES.txt            |  3 +
 .../mapred/TestMapreduceConfigFields.java       | 76 ++++++++++++++++++++
 3 files changed, 135 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9809a16d/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationFieldsBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationFieldsBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationFieldsBase.java
index c3fe3a3..2e4d8b1 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationFieldsBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationFieldsBase.java
@@ -147,6 +147,12 @@ public abstract class TestConfigurationFieldsBase {
   private Set<String> xmlFieldsMissingInConfiguration = null;
 
   /**
+   * Member variable for debugging base class operation
+   */
+  protected boolean configDebug = false;
+  protected boolean xmlDebug = false;
+
+  /**
    * Abstract method to be used by subclasses for initializing base
    * members.
    */
@@ -168,13 +174,16 @@ public abstract class TestConfigurationFieldsBase {
     HashMap<String,String> retVal = new HashMap<String,String>();
 
     // Setup regexp for valid properties
-    String propRegex = "^[A-Za-z_-]+(\\.[A-Za-z_-]+)+$";
+    String propRegex = "^[A-Za-z][A-Za-z0-9_-]+(\\.[A-Za-z0-9_-]+)+$";
     Pattern p = Pattern.compile(propRegex);
 
     // Iterate through class member variables
     int totalFields = 0;
     String value;
     for (Field f : fields) {
+      if (configDebug) {
+        System.out.println("Field: " + f);
+      }
       // Filter out anything that isn't "public static final"
       if (!Modifier.isStatic(f.getModifiers()) ||
           !Modifier.isPublic(f.getModifiers()) ||
@@ -192,6 +201,9 @@ public abstract class TestConfigurationFieldsBase {
       } catch (IllegalAccessException iaException) {
         continue;
       }
+      if (configDebug) {
+        System.out.println("  Value: " + value);
+      }
       // Special Case: Detect and ignore partial properties (ending in x)
       //               or file properties (ending in .xml)
       if (value.endsWith(".xml") ||
@@ -221,11 +233,23 @@ public abstract class TestConfigurationFieldsBase {
       //                  something like: blah.blah2(.blah3.blah4...)
       Matcher m = p.matcher(value);
       if (!m.find()) {
+        if (configDebug) {
+          System.out.println("  Passes Regex: false");
+        }
         continue;
       }
+      if (configDebug) {
+        System.out.println("  Passes Regex: true");
+      }
 
       // Save member variable/value as hash
-      retVal.put(value,f.getName());
+      if (!retVal.containsKey(value)) {
+        retVal.put(value,f.getName());
+      } else {
+        if (configDebug) {
+          System.out.println("ERROR: Already found key for property " + value);
+        }
+      }
     }
 
     return retVal;
@@ -256,6 +280,9 @@ public abstract class TestConfigurationFieldsBase {
       // Ignore known xml props
       if (xmlPropsToSkipCompare != null) {
         if (xmlPropsToSkipCompare.contains(key)) {
+          if (xmlDebug) {
+            System.out.println("  Skipping Full Key: " + key);
+          }
           continue;
         }
       }
@@ -270,14 +297,23 @@ public abstract class TestConfigurationFieldsBase {
 	}
       }
       if (skipPrefix) {
+        if (xmlDebug) {
+          System.out.println("  Skipping Prefix Key: " + key);
+        }
         continue;
       }
       if (conf.onlyKeyExists(key)) {
         retVal.put(key,null);
+        if (xmlDebug) {
+          System.out.println("  XML Key,Null Value: " + key);
+        }
       } else {
         String value = conf.get(key);
         if (value!=null) {
           retVal.put(key,entry.getValue());
+          if (xmlDebug) {
+            System.out.println("  XML Key,Valid Value: " + key);
+          }
         }
       }
       kvItr.remove();
@@ -312,6 +348,10 @@ public abstract class TestConfigurationFieldsBase {
 
     // Create class member/value map
     configurationMemberVariables = new HashMap<String,String>();
+    if (configDebug) {
+      System.out.println("Reading configuration classes");
+      System.out.println("");
+    }
     for (Class c : configurationClasses) {
       Field[] fields = c.getDeclaredFields();
       Map<String,String> memberMap =
@@ -320,9 +360,23 @@ public abstract class TestConfigurationFieldsBase {
         configurationMemberVariables.putAll(memberMap);
       }
     }
+    if (configDebug) {
+      System.out.println("");
+      System.out.println("=====");
+      System.out.println("");
+    }
 
     // Create XML key/value map
+    if (xmlDebug) {
+      System.out.println("Reading XML property files");
+      System.out.println("");
+    }
     xmlKeyValueMap = extractPropertiesFromXml(xmlFilename);
+    if (xmlDebug) {
+      System.out.println("");
+      System.out.println("=====");
+      System.out.println("");
+    }
 
     // Find class members not in the XML file
     configurationFieldsMissingInXmlFile = compareConfigurationToXmlFields

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9809a16d/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 002fbe6..99621cb 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -304,6 +304,9 @@ Release 2.8.0 - UNRELEASED
     mapreduce.tasktracker.taskmemorymanager.monitoringinterval.
     (J.Andreina via aajisaka)
 
+    MAPREDUCE-6192. Create unit test to automatically compare
+    MR related classes and mapred-default.xml (rchiang via rkanter)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9809a16d/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestMapreduceConfigFields.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestMapreduceConfigFields.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestMapreduceConfigFields.java
new file mode 100644
index 0000000..7f18714
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestMapreduceConfigFields.java
@@ -0,0 +1,76 @@
+/**
+ * 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;
+
+import java.util.HashSet;
+
+import org.apache.hadoop.conf.TestConfigurationFieldsBase;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.ShuffleHandler;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
+
+/**
+ * Unit test class to compare the following MR Configuration classes:
+ * <p></p>
+ * {@link org.apache.hadoop.mapreduce.MRJobConfig}
+ * {@link org.apache.hadoop.mapreduce.MRConfig}
+ * {@link org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig}
+ * {@link org.apache.hadoop.mapred.ShuffleHandler}
+ * {@link org.apache.hadoop.mapreduce.lib.output.FileOutputFormat}
+ * {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat}
+ * {@link org.apache.hadoop.mapreduce.Job}
+ * {@link org.apache.hadoop.mapreduce.lib.input.NLineInputFormat}
+ * {@link org.apache.hadoop.mapred.JobConf}
+ * <p></p>
+ * against mapred-default.xml for missing properties.  Currently only
+ * throws an error if the class is missing a property.
+ * <p></p>
+ * Refer to {@link org.apache.hadoop.conf.TestConfigurationFieldsBase}
+ * for how this class works.
+ */
+public class TestMapreduceConfigFields extends TestConfigurationFieldsBase {
+
+  @SuppressWarnings("deprecation")
+  @Override
+  public void initializeMemberVariables() {
+    xmlFilename = new String("mapred-default.xml");
+    configurationClasses = new Class[] { MRJobConfig.class, MRConfig.class,
+        JHAdminConfig.class, ShuffleHandler.class, FileOutputFormat.class,
+	FileInputFormat.class, Job.class, NLineInputFormat.class,
+	JobConf.class, FileOutputCommitter.class };
+
+    // Initialize used variables
+    configurationPropsToSkipCompare = new HashSet<String>();
+
+    // Set error modes
+    errorIfMissingConfigProps = true;
+    errorIfMissingXmlProps = false;
+
+    // Ignore deprecated MR1 properties in JobConf
+    configurationPropsToSkipCompare
+            .add(JobConf.MAPRED_JOB_MAP_MEMORY_MB_PROPERTY);
+    configurationPropsToSkipCompare
+            .add(JobConf.MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY);
+  }
+
+}


[12/17] hadoop git commit: HADOOP-11917. test-patch.sh should work with ${BASEDIR}/patchprocess setups (aw)

Posted by ji...@apache.org.
HADOOP-11917. test-patch.sh should work with ${BASEDIR}/patchprocess setups (aw)


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

Branch: refs/heads/HDFS-7240
Commit: d33419ae01c528073f9f00ef1aadf153fed41222
Parents: 24d3a2d
Author: Allen Wittenauer <aw...@apache.org>
Authored: Tue May 5 11:26:31 2015 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Tue May 5 11:26:31 2015 -0700

----------------------------------------------------------------------
 .gitignore                                      |  1 +
 dev-support/test-patch.sh                       | 78 +++++++++++++++++---
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +
 pom.xml                                         | 23 +++---
 4 files changed, 84 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d33419ae/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index a49ad4b..779f507 100644
--- a/.gitignore
+++ b/.gitignore
@@ -23,3 +23,4 @@ hadoop-tools/hadoop-openstack/src/test/resources/contract-test-options.xml
 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/tla/yarnregistry.toolbox
 yarnregistry.pdf
 hadoop-tools/hadoop-aws/src/test/resources/contract-test-options.xml
+patchprocess/

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d33419ae/dev-support/test-patch.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.sh b/dev-support/test-patch.sh
index 9f48c64..3759e9f 100755
--- a/dev-support/test-patch.sh
+++ b/dev-support/test-patch.sh
@@ -535,6 +535,26 @@ function echo_and_redirect
   "${@}" > "${logfile}" 2>&1
 }
 
+## @description is PATCH_DIR relative to BASEDIR?
+## @audience    public
+## @stability   stable
+## @replaceable yes
+## @returns     1 - no, PATCH_DIR
+## @returns     0 - yes, PATCH_DIR - BASEDIR
+function relative_patchdir
+{
+  local p=${PATCH_DIR#${BASEDIR}}
+
+  if [[ ${#p} -eq ${#PATCH_DIR} ]]; then
+    echo ${p}
+    return 1
+  fi
+  p=${p#/}
+  echo ${p}
+  return 0
+}
+
+
 ## @description  Print the usage information
 ## @audience     public
 ## @stability    stable
@@ -697,7 +717,8 @@ function parse_args
     esac
   done
 
-  # if we get a relative path, turn it absolute
+  # we need absolute dir for ${BASEDIR}
+  cd "${CWD}"
   BASEDIR=$(cd -P -- "${BASEDIR}" >/dev/null && pwd -P)
 
   if [[ ${BUILD_NATIVE} == "true" ]] ; then
@@ -723,6 +744,7 @@ function parse_args
     JENKINS=false
   fi
 
+  cd "${CWD}"
   if [[ ! -d ${PATCH_DIR} ]]; then
     mkdir -p "${PATCH_DIR}"
     if [[ $? == 0 ]] ; then
@@ -733,6 +755,9 @@ function parse_args
     fi
   fi
 
+  # we need absolute dir for PATCH_DIR
+  PATCH_DIR=$(cd -P -- "${PATCH_DIR}" >/dev/null && pwd -P)
+
   GITDIFFLINES=${PATCH_DIR}/gitdifflines.txt
 }
 
@@ -821,17 +846,36 @@ function find_changed_modules
 function git_checkout
 {
   local currentbranch
+  local exemptdir
 
   big_console_header "Confirming git environment"
 
+  cd "${BASEDIR}"
+  if [[ ! -d .git ]]; then
+    hadoop_error "ERROR: ${BASEDIR} is not a git repo."
+    cleanup_and_exit 1
+  fi
+
   if [[ ${RESETREPO} == "true" ]] ; then
-    cd "${BASEDIR}"
     ${GIT} reset --hard
     if [[ $? != 0 ]]; then
       hadoop_error "ERROR: git reset is failing"
       cleanup_and_exit 1
     fi
-    ${GIT} clean -xdf
+
+    # if PATCH_DIR is in BASEDIR, then we don't want
+    # git wiping it out.
+    exemptdir=$(relative_patchdir)
+    if [[ $? == 1 ]]; then
+      ${GIT} clean -xdf
+    else
+      # we do, however, want it emptied of all _files_.
+      # we need to leave _directories_ in case we are in
+      # re-exec mode (which places a directory full of stuff in it)
+      hadoop_debug "Exempting ${exemptdir} from clean"
+      rm "${PATCH_DIR}/*" 2>/dev/null
+      ${GIT} clean -xdf -e "${exemptdir}"
+    fi
     if [[ $? != 0 ]]; then
       hadoop_error "ERROR: git clean is failing"
       cleanup_and_exit 1
@@ -875,11 +919,6 @@ function git_checkout
     fi
 
   else
-    cd "${BASEDIR}"
-    if [[ ! -d .git ]]; then
-      hadoop_error "ERROR: ${BASEDIR} is not a git repo."
-      cleanup_and_exit 1
-    fi
 
     status=$(${GIT} status --porcelain)
     if [[ "${status}" != "" && -z ${DIRTY_WORKSPACE} ]] ; then
@@ -1000,6 +1039,16 @@ function verify_valid_branch
   local check=$2
   local i
 
+  # shortcut some common
+  # non-resolvable names
+  if [[ -z ${check} ]]; then
+    return 1
+  fi
+
+  if [[ ${check} == patch ]]; then
+    return 1
+  fi
+
   if [[ ${check} =~ ^git ]]; then
     ref=$(echo "${check}" | cut -f2 -dt)
     count=$(echo "${ref}" | wc -c | tr -d ' ')
@@ -2207,9 +2256,16 @@ function cleanup_and_exit
 
   if [[ ${JENKINS} == "true" ]] ; then
     if [[ -e "${PATCH_DIR}" ]] ; then
-      hadoop_debug "mv ${PATCH_DIR} ${BASEDIR} "
       if [[ -d "${PATCH_DIR}" ]]; then
-        mv "${PATCH_DIR}" "${BASEDIR}"
+        # if PATCH_DIR is already inside BASEDIR, then
+        # there is no need to move it since we assume that
+        # Jenkins or whatever already knows where it is at
+        # since it told us to put it there!
+        relative_patchdir >/dev/null
+        if [[ $? == 0 ]]; then
+          hadoop_debug "mv ${PATCH_DIR} ${BASEDIR}"
+          mv "${PATCH_DIR}" "${BASEDIR}"
+        fi
       fi
     fi
   fi
@@ -2442,6 +2498,8 @@ find_changed_files
 
 determine_needed_tests
 
+# from here on out, we'll be in ${BASEDIR} for cwd
+# routines need to pushd/popd if they change.
 git_checkout
 RESULT=$?
 if [[ ${JENKINS} == "true" ]] ; then

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d33419ae/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 5b2654a..1b33b37 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -533,6 +533,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11911. test-patch should allow configuration of default branch
     (Sean Busbey via aw)
 
+    HADOOP-11917. test-patch.sh should work with ${BASEDIR}/patchprocess
+    setups (aw)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d33419ae/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1ea52eb..8394324f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -178,7 +178,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
             </dependency>
           </dependencies>
         </plugin>
-        <!--This plugin's configuration is used to store Eclipse m2e settings only. 
+        <!--This plugin's configuration is used to store Eclipse m2e settings only.
         It has no influence on the Maven build itself.-->
         <plugin>
           <groupId>org.eclipse.m2e</groupId>
@@ -361,6 +361,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
             <exclude>.git/**</exclude>
             <exclude>.idea/**</exclude>
 	    <exclude>**/build/**</exclude>
+            <exclude>**/patchprocess/**</exclude>
          </excludes>
        </configuration>
       </plugin>
@@ -391,7 +392,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
       </plugin>
     </plugins>
   </build>
-  
+
   <reporting>
     <excludeDefaults>true</excludeDefaults>
     <plugins>
@@ -441,15 +442,15 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
                 </docletArtifact>
               </docletArtifacts>
               <useStandardDocletOptions>true</useStandardDocletOptions>
-    
+
               <!-- switch on dependency-driven aggregation -->
               <includeDependencySources>false</includeDependencySources>
-    
+
               <dependencySourceIncludes>
                 <!-- include ONLY dependencies I control -->
                 <dependencySourceInclude>org.apache.hadoop:hadoop-annotations</dependencySourceInclude>
               </dependencySourceIncludes>
-    
+
             </configuration>
             <reports>
               <report>aggregate</report>
@@ -472,7 +473,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
       </plugin>
     </plugins>
   </reporting>
-  
+
   <profiles>
     <profile>
       <id>src</id>
@@ -587,12 +588,12 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
       <properties>
         <cloverLicenseLocation>${user.home}/.clover.license</cloverLicenseLocation>
         <cloverDatabase>${project.build.directory}/clover/hadoop-coverage.db</cloverDatabase>
-        <!-- NB: This additional parametrization is made in order 
+        <!-- NB: This additional parametrization is made in order
              to be able to re-define these properties with "-Dk=v" maven options.
-             By some reason the expressions declared in clover 
-             docs like "${maven.clover.generateHtml}" do not work in that way. 
-             However, the below properties are confirmed to work: e.g. 
-             -DcloverGenHtml=false switches off the Html generation.  
+             By some reason the expressions declared in clover
+             docs like "${maven.clover.generateHtml}" do not work in that way.
+             However, the below properties are confirmed to work: e.g.
+             -DcloverGenHtml=false switches off the Html generation.
              The default values provided here exactly correspond to Clover defaults, so
              the behavior is 100% backwards compatible. -->
         <cloverAlwaysReport>true</cloverAlwaysReport>


[02/17] hadoop git commit: HADOOP-11916. TestStringUtils#testLowerAndUpperStrings failed on MAC due to a JVM bug. Contributed by Ming Ma.

Posted by ji...@apache.org.
HADOOP-11916. TestStringUtils#testLowerAndUpperStrings failed on MAC due to a JVM bug. Contributed by Ming Ma.


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

Branch: refs/heads/HDFS-7240
Commit: 338e88a19eeb01364c7f5bcdc5f4b5c35d53852d
Parents: 551615f
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Tue May 5 12:39:24 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Tue May 5 12:39:24 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                  | 3 +++
 .../src/test/java/org/apache/hadoop/util/TestStringUtils.java    | 4 ++++
 2 files changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/338e88a1/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 2bf790a..0b3c971 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -596,6 +596,9 @@ Release 2.8.0 - UNRELEASED
 
     HADOOP-11900. Add failIfNoTests=false to hadoop-build-tools pom. (gera)
 
+    HADOOP-11916. TestStringUtils#testLowerAndUpperStrings failed on MAC
+    due to a JVM bug. (Ming Ma via ozawa)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/338e88a1/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java
index 515c3e0..5b0715f 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java
@@ -37,6 +37,7 @@ import java.util.regex.Pattern;
 
 import org.apache.hadoop.test.UnitTestcaseTimeLimit;
 import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
+import org.junit.Assume;
 import org.junit.Test;
 
 public class TestStringUtils extends UnitTestcaseTimeLimit {
@@ -416,6 +417,9 @@ public class TestStringUtils extends UnitTestcaseTimeLimit {
 
   @Test
   public void testLowerAndUpperStrings() {
+    // Due to java bug http://bugs.java.com/bugdatabase/view_bug.do?bug_id=8047340,
+    // The test will fail with Turkish locality on Mac OS.
+    Assume.assumeTrue(Shell.LINUX);
     Locale defaultLocale = Locale.getDefault();
     try {
       Locale.setDefault(new Locale("tr", "TR"));