Browse Source

Merging r1532877 through r1532910 from trunk to branch HDFS-2832

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2832@1532913 13f79535-47bb-0310-9956-ffa450edef68
Arpit Agarwal 12 năm trước cách đây
mục cha
commit
a6076adbef

+ 6 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -357,6 +357,9 @@ Release 2.3.0 - UNRELEASED
     HADOOP-9494. Excluded auto-generated and examples code from clover reports
     (Andrey Klochkov via jeagles)
 
+    HADOOP-9897. Add method to get path start position without drive specifier in
+    o.a.h.fs.Path. (Binglin Chang via cnauroth)
+
   OPTIMIZATIONS
 
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
@@ -582,6 +585,9 @@ Release 2.1.1-beta - 2013-09-23
     HADOOP-9977. Hadoop services won't start with different keypass and
     keystorepass when https is enabled. (cnauroth)
 
+    HADOOP-10005. No need to check INFO severity level is enabled or not.
+    (Jackie Chang via suresh)
+
 Release 2.1.0-beta - 2013-08-22
 
   INCOMPATIBLE CHANGES

+ 25 - 17
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java

@@ -218,10 +218,13 @@ public class Path implements Comparable {
    */
   public static Path mergePaths(Path path1, Path path2) {
     String path2Str = path2.toUri().getPath();
-    if(hasWindowsDrive(path2Str)) {
-      path2Str = path2Str.substring(path2Str.indexOf(':')+1);
-    }
-    return new Path(path1 + path2Str);
+    path2Str = path2Str.substring(startPositionWithoutWindowsDrive(path2Str));
+    // Add path components explicitly, because simply concatenating two path
+    // string is not safe, for example:
+    // "/" + "/foo" yields "//foo", which will be parsed as authority in Path
+    return new Path(path1.toUri().getScheme(), 
+        path1.toUri().getAuthority(), 
+        path1.toUri().getPath() + path2Str);
   }
 
   /**
@@ -247,8 +250,8 @@ public class Path implements Comparable {
     }
     
     // trim trailing slash from non-root path (ignoring windows drive)
-    int minLength = hasWindowsDrive(path) ? 4 : 1;
-    if (path.length() > minLength && path.endsWith("/")) {
+    int minLength = startPositionWithoutWindowsDrive(path) + 1;
+    if (path.length() > minLength && path.endsWith(SEPARATOR)) {
       path = path.substring(0, path.length()-1);
     }
     
@@ -259,6 +262,14 @@ public class Path implements Comparable {
     return (WINDOWS && hasDriveLetterSpecifier.matcher(path).find());
   }
 
+  private static int startPositionWithoutWindowsDrive(String path) {
+    if (hasWindowsDrive(path)) {
+      return path.charAt(0) ==  SEPARATOR_CHAR ? 3 : 2;
+    } else {
+      return 0;
+    }
+  }
+  
   /**
    * Determine whether a given path string represents an absolute path on
    * Windows. e.g. "C:/a/b" is an absolute path. "C:a/b" is not.
@@ -270,13 +281,11 @@ public class Path implements Comparable {
    */
   public static boolean isWindowsAbsolutePath(final String pathString,
                                               final boolean slashed) {
-    int start = (slashed ? 1 : 0);
-
-    return
-        hasWindowsDrive(pathString) &&
-        pathString.length() >= (start + 3) &&
-        ((pathString.charAt(start + 2) == SEPARATOR_CHAR) ||
-          (pathString.charAt(start + 2) == '\\'));
+    int start = startPositionWithoutWindowsDrive(pathString);
+    return start > 0
+        && pathString.length() > start
+        && ((pathString.charAt(start) == SEPARATOR_CHAR) ||
+            (pathString.charAt(start) == '\\'));
   }
 
   /** Convert this to a URI. */
@@ -300,7 +309,7 @@ public class Path implements Comparable {
    *  True if the path component (i.e. directory) of this URI is absolute.
    */
   public boolean isUriPathAbsolute() {
-    int start = hasWindowsDrive(uri.getPath()) ? 3 : 0;
+    int start = startPositionWithoutWindowsDrive(uri.getPath());
     return uri.getPath().startsWith(SEPARATOR, start);
    }
   
@@ -334,7 +343,7 @@ public class Path implements Comparable {
   public Path getParent() {
     String path = uri.getPath();
     int lastSlash = path.lastIndexOf('/');
-    int start = hasWindowsDrive(path) ? 3 : 0;
+    int start = startPositionWithoutWindowsDrive(path);
     if ((path.length() == start) ||               // empty path
         (lastSlash == start && path.length() == start+1)) { // at root
       return null;
@@ -343,8 +352,7 @@ public class Path implements Comparable {
     if (lastSlash==-1) {
       parent = CUR_DIR;
     } else {
-      int end = hasWindowsDrive(path) ? 3 : 0;
-      parent = path.substring(0, lastSlash==end?end+1:lastSlash);
+      parent = path.substring(0, lastSlash==start?start+1:lastSlash);
     }
     return new Path(uri.getScheme(), uri.getAuthority(), parent);
   }

+ 19 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestPath.java

@@ -460,6 +460,13 @@ public class TestPath extends TestCase {
       Path.mergePaths(new Path("/C:/foo"),
         new Path("/C:/bar")));
 
+    assertEquals(new Path(Shell.WINDOWS ? "/C:/bar" : "/C:/C:/bar"),
+        Path.mergePaths(new Path("/C:/"),
+          new Path("/C:/bar")));
+
+    assertEquals(new Path("/bar"),
+        Path.mergePaths(new Path("/"), new Path("/bar")));
+
     assertEquals(new Path("viewfs:///foo/bar"),
       Path.mergePaths(new Path("viewfs:///foo"),
         new Path("file:///bar")));
@@ -468,4 +475,16 @@ public class TestPath extends TestCase {
       Path.mergePaths(new Path("viewfs://vfsauthority/foo"),
         new Path("file://fileauthority/bar")));
   }
+
+  @Test (timeout = 30000)
+  public void testIsWindowsAbsolutePath() {
+    if (!Shell.WINDOWS) return;
+    assertTrue(Path.isWindowsAbsolutePath("C:\\test", false));
+    assertTrue(Path.isWindowsAbsolutePath("C:/test", false));
+    assertTrue(Path.isWindowsAbsolutePath("/C:/test", true));
+    assertFalse(Path.isWindowsAbsolutePath("/test", false));
+    assertFalse(Path.isWindowsAbsolutePath("/test", true));
+    assertFalse(Path.isWindowsAbsolutePath("C:test", false));
+    assertFalse(Path.isWindowsAbsolutePath("/C:test", true));
+  }
 }

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -392,6 +392,9 @@ Release 2.2.1 - UNRELEASED
 
     HDFS-5330. fix readdir and readdirplus for large directories (brandonli)
 
+    HDFS-5370. Typo in Error Message: different between range in condition
+    and range in error message. (Kousuke Saruta via suresh)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -403,7 +403,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
 
     //check offset
     if (offset < 0 || offset >= getFileLength()) {
-      throw new IOException("offset < 0 || offset > getFileLength(), offset="
+      throw new IOException("offset < 0 || offset >= getFileLength(), offset="
           + offset
           + ", updatePosition=" + updatePosition
           + ", locatedBlocks=" + locatedBlocks);

+ 6 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -1273,15 +1273,13 @@ public class DataNode extends Configured
     
     int numTargets = xferTargets.length;
     if (numTargets > 0) {
-      if (LOG.isInfoEnabled()) {
-        StringBuilder xfersBuilder = new StringBuilder();
-        for (int i = 0; i < numTargets; i++) {
-          xfersBuilder.append(xferTargets[i]);
-          xfersBuilder.append(" ");
-        }
-        LOG.info(bpReg + " Starting thread to transfer " + 
-                 block + " to " + xfersBuilder);                       
+      StringBuilder xfersBuilder = new StringBuilder();
+      for (int i = 0; i < numTargets; i++) {
+        xfersBuilder.append(xferTargets[i]);
+        xfersBuilder.append(" ");
       }
+      LOG.info(bpReg + " Starting thread to transfer " + 
+               block + " to " + xfersBuilder);                       
 
       new Daemon(new DataTransfer(xferTargets, block,
           BlockConstructionStage.PIPELINE_SETUP_CREATE, "")).start();

+ 3 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -1494,6 +1494,9 @@ Release 0.23.10 - UNRELEASED
     MAPREDUCE-5513. ConcurrentModificationException in JobControl (Robert
     Parker via jlowe)
 
+    MAPREDUCE-5586. TestCopyMapper#testCopyFailOnBlockSizeDifference fails when
+    run from hadoop-tools/hadoop-distcp directory (jeagles)
+
 Release 0.23.9 - 2013-07-08
 
   INCOMPATIBLE CHANGES

+ 22 - 32
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java

@@ -972,12 +972,10 @@ public class MapTask extends Task {
       maxRec = kvmeta.capacity() / NMETA;
       softLimit = (int)(kvbuffer.length * spillper);
       bufferRemaining = softLimit;
-      if (LOG.isInfoEnabled()) {
-        LOG.info(JobContext.IO_SORT_MB + ": " + sortmb);
-        LOG.info("soft limit at " + softLimit);
-        LOG.info("bufstart = " + bufstart + "; bufvoid = " + bufvoid);
-        LOG.info("kvstart = " + kvstart + "; length = " + maxRec);
-      }
+      LOG.info(JobContext.IO_SORT_MB + ": " + sortmb);
+      LOG.info("soft limit at " + softLimit);
+      LOG.info("bufstart = " + bufstart + "; bufvoid = " + bufvoid);
+      LOG.info("kvstart = " + kvstart + "; length = " + maxRec);
 
       // k/v serialization
       comparator = job.getOutputKeyComparator();
@@ -1180,10 +1178,8 @@ public class MapTask extends Task {
       final int aligned = pos - (pos % METASIZE);
       kvindex =
         ((aligned - METASIZE + kvbuffer.length) % kvbuffer.length) / 4;
-      if (LOG.isInfoEnabled()) {
-        LOG.info("(EQUATOR) " + pos + " kvi " + kvindex +
-            "(" + (kvindex * 4) + ")");
-      }
+      LOG.info("(EQUATOR) " + pos + " kvi " + kvindex +
+          "(" + (kvindex * 4) + ")");
     }
 
     /**
@@ -1198,10 +1194,8 @@ public class MapTask extends Task {
       // set start/end to point to first meta record
       kvstart = kvend =
         ((aligned - METASIZE + kvbuffer.length) % kvbuffer.length) / 4;
-      if (LOG.isInfoEnabled()) {
-        LOG.info("(RESET) equator " + e + " kv " + kvstart + "(" +
-          (kvstart * 4) + ")" + " kvi " + kvindex + "(" + (kvindex * 4) + ")");
-      }
+      LOG.info("(RESET) equator " + e + " kv " + kvstart + "(" +
+        (kvstart * 4) + ")" + " kvi " + kvindex + "(" + (kvindex * 4) + ")");
     }
 
     /**
@@ -1456,15 +1450,13 @@ public class MapTask extends Task {
         if (kvindex != kvend) {
           kvend = (kvindex + NMETA) % kvmeta.capacity();
           bufend = bufmark;
-          if (LOG.isInfoEnabled()) {
-            LOG.info("Spilling map output");
-            LOG.info("bufstart = " + bufstart + "; bufend = " + bufmark +
-                     "; bufvoid = " + bufvoid);
-            LOG.info("kvstart = " + kvstart + "(" + (kvstart * 4) +
-                     "); kvend = " + kvend + "(" + (kvend * 4) +
-                     "); length = " + (distanceTo(kvend, kvstart,
-                           kvmeta.capacity()) + 1) + "/" + maxRec);
-          }
+          LOG.info("Spilling map output");
+          LOG.info("bufstart = " + bufstart + "; bufend = " + bufmark +
+                   "; bufvoid = " + bufvoid);
+          LOG.info("kvstart = " + kvstart + "(" + (kvstart * 4) +
+                   "); kvend = " + kvend + "(" + (kvend * 4) +
+                   "); length = " + (distanceTo(kvend, kvstart,
+                         kvmeta.capacity()) + 1) + "/" + maxRec);
           sortAndSpill();
         }
       } catch (InterruptedException e) {
@@ -1547,15 +1539,13 @@ public class MapTask extends Task {
       kvend = (kvindex + NMETA) % kvmeta.capacity();
       bufend = bufmark;
       spillInProgress = true;
-      if (LOG.isInfoEnabled()) {
-        LOG.info("Spilling map output");
-        LOG.info("bufstart = " + bufstart + "; bufend = " + bufmark +
-                 "; bufvoid = " + bufvoid);
-        LOG.info("kvstart = " + kvstart + "(" + (kvstart * 4) +
-                 "); kvend = " + kvend + "(" + (kvend * 4) +
-                 "); length = " + (distanceTo(kvend, kvstart,
-                       kvmeta.capacity()) + 1) + "/" + maxRec);
-      }
+      LOG.info("Spilling map output");
+      LOG.info("bufstart = " + bufstart + "; bufend = " + bufmark +
+               "; bufvoid = " + bufvoid);
+      LOG.info("kvstart = " + kvstart + "(" + (kvstart * 4) +
+               "); kvend = " + kvend + "(" + (kvend * 4) +
+               "); length = " + (distanceTo(kvend, kvstart,
+                     kvmeta.capacity()) + 1) + "/" + maxRec);
       spillReady.signal();
     }
 

+ 6 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileInputFormat.java

@@ -81,14 +81,12 @@ public class TestMultiFileInputFormat extends TestCase{
   }
   
   public void testFormat() throws IOException {
-    if(LOG.isInfoEnabled()) {
-      LOG.info("Test started");
-      LOG.info("Max split count           = " + MAX_SPLIT_COUNT);
-      LOG.info("Split count increment     = " + SPLIT_COUNT_INCR);
-      LOG.info("Max bytes per file        = " + MAX_BYTES);
-      LOG.info("Max number of files       = " + MAX_NUM_FILES);
-      LOG.info("Number of files increment = " + NUM_FILES_INCR);
-    }
+    LOG.info("Test started");
+    LOG.info("Max split count           = " + MAX_SPLIT_COUNT);
+    LOG.info("Split count increment     = " + SPLIT_COUNT_INCR);
+    LOG.info("Max bytes per file        = " + MAX_BYTES);
+    LOG.info("Max number of files       = " + MAX_NUM_FILES);
+    LOG.info("Number of files increment = " + NUM_FILES_INCR);
     
     MultiFileInputFormat<Text,Text> format = new DummyMultiFileInputFormat();
     FileSystem fs = FileSystem.getLocal(job);

+ 1 - 0
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java

@@ -75,6 +75,7 @@ public class TestCopyMapper {
     Configuration configuration = new Configuration();
     System.setProperty("test.build.data", "target/tmp/build/TEST_COPY_MAPPER/data");
     configuration.set("hadoop.log.dir", "target/tmp");
+    configuration.set("dfs.namenode.fs-limits.min-block-size", "0");
     LOG.debug("fs.default.name  == " + configuration.get("fs.default.name"));
     LOG.debug("dfs.http.address == " + configuration.get("dfs.http.address"));
     return configuration;

+ 3 - 5
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/DebugJobProducer.java

@@ -122,11 +122,9 @@ public class DebugJobProducer implements JobStoryProducer {
       // Add/remove excess
       recs[0] += totalrecs - tot_recs;
       bytes[0] += totalbytes - tot_bytes;
-      if (LOG.isInfoEnabled()) {
-        LOG.info(
-          "DIST: " + Arrays.toString(recs) + " " + tot_recs + "/" + totalrecs +
-            " " + Arrays.toString(bytes) + " " + tot_bytes + "/" + totalbytes);
-      }
+      LOG.info(
+        "DIST: " + Arrays.toString(recs) + " " + tot_recs + "/" + totalrecs +
+          " " + Arrays.toString(bytes) + " " + tot_bytes + "/" + totalbytes);
     }
 
     private static final AtomicInteger seq = new AtomicInteger(0);