Modified: 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/errno_enum.c
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/errno_enum.c?rev=1446832&r1=1446831&r2=1446832&view=diff
==============================================================================
--- 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/errno_enum.c
 (original)
+++ 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/errno_enum.c
 Sat Feb 16 01:12:07 2013
@@ -63,6 +63,9 @@ static errno_mapping_t ERRNO_MAPPINGS[] 
   MAPPING(EPIPE),
   MAPPING(EDOM),
   MAPPING(ERANGE),
+  MAPPING(ELOOP),
+  MAPPING(ENAMETOOLONG),
+  MAPPING(ENOTEMPTY),
   {-1, NULL}
 };
 

Modified: 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/site/apt/ClusterSetup.apt.vm
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/site/apt/ClusterSetup.apt.vm?rev=1446832&r1=1446831&r2=1446832&view=diff
==============================================================================
--- 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/site/apt/ClusterSetup.apt.vm
 (original)
+++ 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/site/apt/ClusterSetup.apt.vm
 Sat Feb 16 01:12:07 2013
@@ -325,6 +325,13 @@ Hadoop MapReduce Next Generation - Clust
 | | | How long to keep aggregation logs before deleting them. -1 disables. |
 | | | Be careful, set this too small and you will spam the name node. |
 *-------------------------+-------------------------+------------------------+
+| <<<yarn.log-aggregation.retain-check-interval-seconds>>> | | |
+| | <-1> | |
+| | | Time between checks for aggregated log retention. If set to 0 or a |
+| | | negative value then the value is computed as one-tenth of the |
+| | | aggregated log retention time. |
+| | | Be careful, set this too small and you will spam the name node. |
+*-------------------------+-------------------------+------------------------+
 
 
 

Modified: 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/site/apt/CommandsManual.apt.vm
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/site/apt/CommandsManual.apt.vm?rev=1446832&r1=1446831&r2=1446832&view=diff
==============================================================================
--- 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/site/apt/CommandsManual.apt.vm
 (original)
+++ 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/site/apt/CommandsManual.apt.vm
 Sat Feb 16 01:12:07 2013
@@ -350,10 +350,11 @@ Administration Commands
 
    Runs a HDFS dfsadmin client.
 
-   Usage: <<<hadoop dfsadmin [GENERIC_OPTIONS] [-report] [-safemode enter | 
leave | get | wait] [-refreshNodes] [-finalizeUpgrade] [-upgradeProgress status 
| details | force] [-metasave filename] [-setQuota <quota> 
<dirname>...<dirname>] [-clrQuota <dirname>...<dirname>] [-help [cmd]]>>>
+   Usage: <<<hadoop dfsadmin [GENERIC_OPTIONS] [-report] [-safemode enter | 
leave | get | wait] [-refreshNodes] [-finalizeUpgrade] [-upgradeProgress status 
| details | force] [-metasave filename] [-setQuota <quota> 
<dirname>...<dirname>] [-clrQuota <dirname>...<dirname>] [-restoreFailedStorage 
true|false|check] [-help [cmd]]>>>
 
 *-----------------+-----------------------------------------------------------+
 || COMMAND_OPTION || Description
+*-----------------+-----------------------------------------------------------+
 | -report         | Reports basic filesystem information and statistics.
 *-----------------+-----------------------------------------------------------+
 | -safemode enter / leave / get / wait | Safe mode maintenance command. Safe
@@ -403,6 +404,10 @@ Administration Commands
                   | 2. user is not an administrator.  It does not fault if the
                   | directory has no quota.
 *-----------------+-----------------------------------------------------------+
+| -restoreFailedStorage true / false / check | This option will turn on/off 
automatic attempt to restore failed storage replicas.
+                  | If a failed storage becomes available again the system 
will attempt to restore
+                  | edits and/or fsimage during checkpoint. 'check' option 
will return current setting.
+*-----------------+-----------------------------------------------------------+
 | -help [cmd]     | Displays help for the given command or all commands if none
                   | is specified.
 *-----------------+-----------------------------------------------------------+

Propchange: 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/core/
------------------------------------------------------------------------------
  Merged 
/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/core:r1440578-1446830

Modified: 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java?rev=1446832&r1=1446831&r2=1446832&view=diff
==============================================================================
--- 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java
 (original)
+++ 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java
 Sat Feb 16 01:12:07 2013
@@ -546,4 +546,44 @@ public class TestFileUtil {
     long expected = 2 * (3 + System.getProperty("line.separator").length());
     Assert.assertEquals(expected, du);
   }
+
+  private void doUntarAndVerify(File tarFile, File untarDir) 
+                                 throws IOException {
+    if (untarDir.exists() && !FileUtil.fullyDelete(untarDir)) {
+      throw new IOException("Could not delete directory '" + untarDir + "'");
+    }
+    FileUtil.unTar(tarFile, untarDir);
+
+    String parentDir = untarDir.getCanonicalPath() + Path.SEPARATOR + "name";
+    File testFile = new File(parentDir + Path.SEPARATOR + "version");
+    Assert.assertTrue(testFile.exists());
+    Assert.assertTrue(testFile.length() == 0);
+    String imageDir = parentDir + Path.SEPARATOR + "image";
+    testFile = new File(imageDir + Path.SEPARATOR + "fsimage");
+    Assert.assertTrue(testFile.exists());
+    Assert.assertTrue(testFile.length() == 157);
+    String currentDir = parentDir + Path.SEPARATOR + "current";
+    testFile = new File(currentDir + Path.SEPARATOR + "fsimage");
+    Assert.assertTrue(testFile.exists());
+    Assert.assertTrue(testFile.length() == 4331);
+    testFile = new File(currentDir + Path.SEPARATOR + "edits");
+    Assert.assertTrue(testFile.exists());
+    Assert.assertTrue(testFile.length() == 1033);
+    testFile = new File(currentDir + Path.SEPARATOR + "fstime");
+    Assert.assertTrue(testFile.exists());
+    Assert.assertTrue(testFile.length() == 8);
+  }
+
+  @Test
+  public void testUntar() throws IOException {
+    String tarGzFileName = System.getProperty("test.cache.data",
+        "build/test/cache") + "/test-untar.tgz";
+    String tarFileName = System.getProperty("test.cache.data",
+        "build/test/cache") + "/test-untar.tar";
+    String dataDir = System.getProperty("test.build.data", "build/test/data");
+    File untarDir = new File(dataDir, "untarDir");
+
+    doUntarAndVerify(new File(tarGzFileName), untarDir);
+    doUntarAndVerify(new File(tarFileName), untarDir);
+  }
 }

Modified: 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java?rev=1446832&r1=1446831&r2=1446832&view=diff
==============================================================================
--- 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java
 (original)
+++ 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java
 Sat Feb 16 01:12:07 2013
@@ -304,6 +304,46 @@ public class TestFsShellReturnCode {
   }
   
   @Test
+  public void testRmWithNonexistentGlob() throws Exception {
+    Configuration conf = new Configuration();
+    FsShell shell = new FsShell();
+    shell.setConf(conf);
+    final ByteArrayOutputStream bytes = new ByteArrayOutputStream();
+    final PrintStream err = new PrintStream(bytes);
+    final PrintStream oldErr = System.err;
+    System.setErr(err);
+    final String results;
+    try {
+      int exit = shell.run(new String[]{"-rm", "nomatch*"});
+      assertEquals(1, exit);
+      results = bytes.toString();
+      assertTrue(results.contains("rm: `nomatch*': No such file or 
directory"));
+    } finally {
+      IOUtils.closeStream(err);
+      System.setErr(oldErr);
+    }
+  }
+
+  @Test
+  public void testRmForceWithNonexistentGlob() throws Exception {
+    Configuration conf = new Configuration();
+    FsShell shell = new FsShell();
+    shell.setConf(conf);
+    final ByteArrayOutputStream bytes = new ByteArrayOutputStream();
+    final PrintStream err = new PrintStream(bytes);
+    final PrintStream oldErr = System.err;
+    System.setErr(err);
+    try {
+      int exit = shell.run(new String[]{"-rm", "-f", "nomatch*"});
+      assertEquals(0, exit);
+      assertTrue(bytes.toString().isEmpty());
+    } finally {
+      IOUtils.closeStream(err);
+      System.setErr(oldErr);
+    }
+  }
+
+  @Test
   public void testInvalidDefaultFS() throws Exception {
     // if default fs doesn't exist or is invalid, but the path provided in 
     // arguments is valid - fsshell should work

Modified: 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java?rev=1446832&r1=1446831&r2=1446832&view=diff
==============================================================================
--- 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java
 (original)
+++ 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java
 Sat Feb 16 01:12:07 2013
@@ -28,6 +28,7 @@ import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Shell;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -46,8 +47,18 @@ public class TestHarFileSystemBasics {
 
   private static final String ROOT_PATH = System.getProperty("test.build.data",
       "build/test/data");
-  private static final Path rootPath = new Path(
-      new File(ROOT_PATH).getAbsolutePath() + "/localfs");
+  private static final Path rootPath;
+  static {
+    String root = new Path(new File(ROOT_PATH).getAbsolutePath(), "localfs")
+      .toUri().getPath();
+    // Strip drive specifier on Windows, which would make the HAR URI invalid 
and
+    // cause tests to fail.
+    if (Shell.WINDOWS) {
+      root = root.substring(root.indexOf(':') + 1);
+    }
+    rootPath = new Path(root);
+  }
+
   // NB: .har suffix is necessary
   private static final Path harPath = new Path(rootPath, "path1/path2/my.har");
 

Modified: 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestListFiles.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestListFiles.java?rev=1446832&r1=1446831&r2=1446832&view=diff
==============================================================================
--- 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestListFiles.java
 (original)
+++ 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestListFiles.java
 Sat Feb 16 01:12:07 2013
@@ -45,19 +45,39 @@ public class TestListFiles {
 
   final protected static Configuration conf = new Configuration();
   protected static FileSystem fs;
-  final protected static Path TEST_DIR = getTestDir();
+  protected static Path TEST_DIR;
   final private static int FILE_LEN = 10;
-  final private static Path FILE1 = new Path(TEST_DIR, "file1");
-  final private static Path DIR1 = new Path(TEST_DIR, "dir1");
-  final private static Path FILE2 = new Path(DIR1, "file2");
-  final private static Path FILE3 = new Path(DIR1, "file3");
+  private static Path FILE1;
+  private static Path DIR1;
+  private static Path FILE2;
+  private static Path FILE3;
+
+  static {
+    setTestPaths(new Path(
+      System.getProperty("test.build.data", 
"build/test/data/work-dir/localfs"),
+      "main_"));
+  }
 
   protected static Path getTestDir() {
-    return new Path(
-      System.getProperty("test.build.data","build/test/data/work-dir/localfs"),
-      "main_");
+    return TEST_DIR;
   }
-  
+
+  /**
+   * Sets the root testing directory and reinitializes any additional test 
paths
+   * that are under the root.  This method is intended to be called from a
+   * subclass's @BeforeClass method if there is a need to override the testing
+   * directory.
+   * 
+   * @param testDir Path root testing directory
+   */
+  protected static void setTestPaths(Path testDir) {
+    TEST_DIR = testDir;
+    FILE1 = new Path(TEST_DIR, "file1");
+    DIR1 = new Path(TEST_DIR, "dir1");
+    FILE2 = new Path(DIR1, "file2");
+    FILE3 = new Path(DIR1, "file3");
+  }
+
   @BeforeClass
   public static void testSetUp() throws Exception {
     fs = FileSystem.getLocal(conf);

Modified: 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java?rev=1446832&r1=1446831&r2=1446832&view=diff
==============================================================================
--- 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java
 (original)
+++ 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java
 Sat Feb 16 01:12:07 2013
@@ -28,6 +28,7 @@ import java.io.*;
 import static org.junit.Assert.*;
 import static org.junit.Assume.assumeTrue;
 
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -38,8 +39,9 @@ public class TestLocalFileSystem {
   private static final String TEST_ROOT_DIR
     = System.getProperty("test.build.data","build/test/data") + 
"/work-dir/localfs";
 
+  private final File base = new File(TEST_ROOT_DIR);
   private Configuration conf;
-  private FileSystem fileSys;
+  private LocalFileSystem fileSys;
 
   private void cleanupFile(FileSystem fs, Path name) throws IOException {
     assertTrue(fs.exists(name));
@@ -53,6 +55,13 @@ public class TestLocalFileSystem {
     fileSys = FileSystem.getLocal(conf);
     fileSys.delete(new Path(TEST_ROOT_DIR), true);
   }
+  
+  @After
+  public void after() throws IOException {
+    base.setWritable(true);
+    FileUtil.fullyDelete(base);
+    assertTrue(!base.exists());
+  }
 
   /**
    * Test the capability of setting the working directory.
@@ -269,10 +278,83 @@ public class TestLocalFileSystem {
     LocalFileSystem fs = FileSystem.getLocal(conf);
     File colonFile = new File(TEST_ROOT_DIR, "foo:bar");
     colonFile.mkdirs();
-    colonFile.createNewFile();
     FileStatus[] stats = fs.listStatus(new Path(TEST_ROOT_DIR));
     assertEquals("Unexpected number of stats", 1, stats.length);
     assertEquals("Bad path from stat", colonFile.getAbsolutePath(),
         stats[0].getPath().toUri().getPath());
   }
+  
+  @Test
+  public void testReportChecksumFailure() throws IOException {
+    base.mkdirs();
+    assertTrue(base.exists() && base.isDirectory());
+    
+    final File dir1 = new File(base, "dir1");
+    final File dir2 = new File(dir1, "dir2");
+    dir2.mkdirs();
+    assertTrue(dir2.exists() && dir2.canWrite());
+    
+    final String dataFileName = "corruptedData";
+    final Path dataPath = new Path(new File(dir2, dataFileName).toURI());
+    final Path checksumPath = fileSys.getChecksumFile(dataPath);
+    final FSDataOutputStream fsdos = fileSys.create(dataPath);
+    try {
+      fsdos.writeUTF("foo");
+    } finally {
+      fsdos.close();
+    }
+    assertTrue(fileSys.pathToFile(dataPath).exists());
+    final long dataFileLength = fileSys.getFileStatus(dataPath).getLen();
+    assertTrue(dataFileLength > 0);
+    
+    // check the the checksum file is created and not empty:
+    assertTrue(fileSys.pathToFile(checksumPath).exists());
+    final long checksumFileLength = 
fileSys.getFileStatus(checksumPath).getLen();
+    assertTrue(checksumFileLength > 0);
+    
+    // this is a hack to force the #reportChecksumFailure() method to stop
+    // climbing up at the 'base' directory and use 'dir1/bad_files' as the 
+    // corrupted files storage:
+    base.setWritable(false);
+    
+    FSDataInputStream dataFsdis = fileSys.open(dataPath);
+    FSDataInputStream checksumFsdis = fileSys.open(checksumPath);
+    
+    boolean retryIsNecessary = fileSys.reportChecksumFailure(dataPath, 
dataFsdis, 0, checksumFsdis, 0);
+    assertTrue(!retryIsNecessary);
+    
+    // the data file should be moved:
+    assertTrue(!fileSys.pathToFile(dataPath).exists());
+    // the checksum file should be moved:
+    assertTrue(!fileSys.pathToFile(checksumPath).exists());
+    
+    // check that the files exist in the new location where they were moved:
+    File[] dir1files = dir1.listFiles(new FileFilter() {
+      @Override
+      public boolean accept(File pathname) {
+        return pathname != null && !pathname.getName().equals("dir2");
+      }
+    });
+    assertTrue(dir1files != null);
+    assertTrue(dir1files.length == 1);
+    File badFilesDir = dir1files[0];
+    
+    File[] badFiles = badFilesDir.listFiles();
+    assertTrue(badFiles != null);
+    assertTrue(badFiles.length == 2);
+    boolean dataFileFound = false;
+    boolean checksumFileFound = false;
+    for (File badFile: badFiles) {
+      if (badFile.getName().startsWith(dataFileName)) {
+        assertTrue(dataFileLength == badFile.length());
+        dataFileFound = true;
+      } else if (badFile.getName().contains(dataFileName + ".crc")) {
+        assertTrue(checksumFileLength == badFile.length());
+        checksumFileFound = true;
+      }
+    }
+    assertTrue(dataFileFound);
+    assertTrue(checksumFileFound);
+  }
+  
 }

Modified: 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestBoundedByteArrayOutputStream.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestBoundedByteArrayOutputStream.java?rev=1446832&r1=1446831&r2=1446832&view=diff
==============================================================================
--- 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestBoundedByteArrayOutputStream.java
 (original)
+++ 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestBoundedByteArrayOutputStream.java
 Sat Feb 16 01:12:07 2013
@@ -88,4 +88,61 @@ public class TestBoundedByteArrayOutputS
     assertTrue("Writing beyond limit did not throw an exception",
         caughtException);
   }
+  
+  
+  static class ResettableBoundedByteArrayOutputStream 
+  extends BoundedByteArrayOutputStream {
+
+    public ResettableBoundedByteArrayOutputStream(int capacity) {
+      super(capacity);
+    }
+
+    public void resetBuffer(byte[] buf, int offset, int length) {
+      super.resetBuffer(buf, offset, length);
+    }
+    
+  }
+  
+  public void testResetBuffer() throws IOException {
+    
+    ResettableBoundedByteArrayOutputStream stream = 
+      new ResettableBoundedByteArrayOutputStream(SIZE);
+
+    // Write to the stream, get the data back and check for contents
+    stream.write(INPUT, 0, SIZE);
+    assertTrue("Array Contents Mismatch",
+        Arrays.equals(INPUT, stream.getBuffer()));
+    
+    // Try writing beyond end of buffer. Should throw an exception
+    boolean caughtException = false;
+    
+    try {
+      stream.write(INPUT[0]);
+    } catch (Exception e) {
+      caughtException = true;
+    }
+    
+    assertTrue("Writing beyond limit did not throw an exception",
+        caughtException);
+    
+    //Reset the stream and try, should succeed
+    byte[] newBuf = new byte[SIZE];
+    stream.resetBuffer(newBuf, 0, newBuf.length);
+    assertTrue("Limit did not get reset correctly", 
+        (stream.getLimit() == SIZE));
+    stream.write(INPUT, 0, SIZE);
+    assertTrue("Array Contents Mismatch",
+        Arrays.equals(INPUT, stream.getBuffer()));
+  
+    // Try writing one more byte, should fail
+    caughtException = false;
+    try {
+      stream.write(INPUT[0]);
+    } catch (Exception e) {
+      caughtException = true;
+    }
+    assertTrue("Writing beyond limit did not throw an exception",
+        caughtException);
+  }
+
 }

Modified: 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSortedMapWritable.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSortedMapWritable.java?rev=1446832&r1=1446831&r2=1446832&view=diff
==============================================================================
--- 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSortedMapWritable.java
 (original)
+++ 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSortedMapWritable.java
 Sat Feb 16 01:12:07 2013
@@ -17,15 +17,20 @@
  */
 package org.apache.hadoop.io;
 
-import java.util.Map;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
-import junit.framework.TestCase;
+import java.util.Map;
+import org.junit.Test;
 
 /**
  * Tests SortedMapWritable
  */
-public class TestSortedMapWritable extends TestCase {
+public class TestSortedMapWritable {
   /** the test */
+  @Test
   @SuppressWarnings("unchecked")
   public void testSortedMapWritable() {
     Text[] keys = {
@@ -90,6 +95,7 @@ public class TestSortedMapWritable exten
   /**
    * Test that number of "unknown" classes is propagated across multiple 
copies.
    */
+  @Test
   @SuppressWarnings("deprecation")
   public void testForeignClass() {
     SortedMapWritable inMap = new SortedMapWritable();
@@ -99,4 +105,77 @@ public class TestSortedMapWritable exten
     SortedMapWritable copyOfCopy = new SortedMapWritable(outMap);
     assertEquals(1, copyOfCopy.getNewClasses());
   }
+  
+  /**
+   * Tests if equal and hashCode method still hold the contract.
+   */
+  @Test
+  public void testEqualsAndHashCode() {
+    String failureReason;
+    SortedMapWritable mapA = new SortedMapWritable();
+    SortedMapWritable mapB = new SortedMapWritable();
+    
+    // Sanity checks
+    failureReason = "SortedMapWritable couldn't be initialized. Got null 
reference";
+    assertNotNull(failureReason, mapA);
+    assertNotNull(failureReason, mapB);
+    
+    // Basic null check
+    assertFalse("equals method returns true when passed null", 
mapA.equals(null));
+    
+    // When entry set is empty, they should be equal
+    assertTrue("Two empty SortedMapWritables are no longer equal", 
mapA.equals(mapB));
+    
+    // Setup
+    Text[] keys = {
+        new Text("key1"),
+        new Text("key2")
+    };
+    
+    BytesWritable[] values = {
+        new BytesWritable("value1".getBytes()),
+        new BytesWritable("value2".getBytes())
+    };
+    
+    mapA.put(keys[0], values[0]);
+    mapB.put(keys[1], values[1]);
+    
+    // entrySets are different
+    failureReason = "Two SortedMapWritables with different data are now equal";
+    assertTrue(failureReason, mapA.hashCode() != mapB.hashCode());
+    assertTrue(failureReason, !mapA.equals(mapB));
+    assertTrue(failureReason, !mapB.equals(mapA));
+    
+    mapA.put(keys[1], values[1]);
+    mapB.put(keys[0], values[0]);
+    
+    // entrySets are now same
+    failureReason = "Two SortedMapWritables with same entry sets formed in 
different order are now different";
+    assertEquals(failureReason, mapA.hashCode(), mapB.hashCode());
+    assertTrue(failureReason, mapA.equals(mapB));
+    assertTrue(failureReason, mapB.equals(mapA));
+    
+    // Let's check if entry sets of same keys but different values
+    mapA.put(keys[0], values[1]);
+    mapA.put(keys[1], values[0]);
+    
+    failureReason = "Two SortedMapWritables with different content are now 
equal";
+    assertTrue(failureReason, mapA.hashCode() != mapB.hashCode());
+    assertTrue(failureReason, !mapA.equals(mapB));
+    assertTrue(failureReason, !mapB.equals(mapA));
+  }
+
+  @Test(timeout = 1000)
+  public void testPutAll() {
+    SortedMapWritable map1 = new SortedMapWritable();
+    SortedMapWritable map2 = new SortedMapWritable();
+    map1.put(new Text("key"), new Text("value"));
+    map2.putAll(map1);
+
+    assertEquals("map1 entries don't match map2 entries", map1, map2);
+    assertTrue(
+        "map2 doesn't have class information from map1",
+        map2.classToIdMap.containsKey(Text.class)
+            && map2.idToClassMap.containsValue(Text.class));
+  }
 }

Modified: 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java?rev=1446832&r1=1446831&r2=1446832&view=diff
==============================================================================
--- 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java
 (original)
+++ 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java
 Sat Feb 16 01:12:07 2013
@@ -25,11 +25,14 @@ import java.io.IOException;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.ArrayList;
 import java.util.List;
+
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import static org.junit.Assume.*;
 import static org.junit.Assert.*;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -293,4 +296,40 @@ public class TestNativeIO {
     assertFalse(NativeIO.getGroupName(0).isEmpty());
   }
 
+  @Test
+  public void testRenameTo() throws Exception {
+    final File TEST_DIR = new File(new File(
+        System.getProperty("test.build.data","build/test/data")), 
"renameTest");
+    assumeTrue(TEST_DIR.mkdirs());
+    File nonExistentFile = new File(TEST_DIR, "nonexistent");
+    File targetFile = new File(TEST_DIR, "target");
+    // Test attempting to rename a nonexistent file.
+    try {
+      NativeIO.renameTo(nonExistentFile, targetFile);
+      Assert.fail();
+    } catch (NativeIOException e) {
+      Assert.assertEquals(e.getErrno(), Errno.ENOENT);
+    }
+    
+    // Test renaming a file to itself.  It should succeed and do nothing.
+    File sourceFile = new File(TEST_DIR, "source");
+    Assert.assertTrue(sourceFile.createNewFile());
+    NativeIO.renameTo(sourceFile, sourceFile);
+
+    // Test renaming a source to a destination.
+    NativeIO.renameTo(sourceFile, targetFile);
+
+    // Test renaming a source to a path which uses a file as a directory.
+    sourceFile = new File(TEST_DIR, "source");
+    Assert.assertTrue(sourceFile.createNewFile());
+    File badTarget = new File(targetFile, "subdir");
+    try {
+      NativeIO.renameTo(sourceFile, badTarget);
+      Assert.fail();
+    } catch (NativeIOException e) {
+      Assert.assertEquals(e.getErrno(), Errno.ENOTDIR);
+    }
+
+    FileUtils.deleteQuietly(TEST_DIR);
+  }
 }

Modified: 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java?rev=1446832&r1=1446831&r2=1446832&view=diff
==============================================================================
--- 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
 (original)
+++ 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
 Sat Feb 16 01:12:07 2013
@@ -26,6 +26,7 @@ import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.net.ConnectTimeoutException;
 import org.apache.hadoop.net.NetUtils;
 
 import java.util.Random;
@@ -586,7 +587,7 @@ public class TestIPC {
   private void assertRetriesOnSocketTimeouts(Configuration conf,
       int maxTimeoutRetries) throws IOException, InterruptedException {
     SocketFactory mockFactory = Mockito.mock(SocketFactory.class);
-    doThrow(new SocketTimeoutException()).when(mockFactory).createSocket();
+    doThrow(new 
ConnectTimeoutException("fake")).when(mockFactory).createSocket();
     Client client = new Client(IntWritable.class, conf, mockFactory);
     InetSocketAddress address = new InetSocketAddress("127.0.0.1", 9090);
     try {

Modified: 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestMetricsSystemImpl.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestMetricsSystemImpl.java?rev=1446832&r1=1446831&r2=1446832&view=diff
==============================================================================
--- 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestMetricsSystemImpl.java
 (original)
+++ 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestMetricsSystemImpl.java
 Sat Feb 16 01:12:07 2013
@@ -56,6 +56,7 @@ import org.apache.hadoop.metrics2.lib.Mu
 import org.apache.hadoop.metrics2.lib.MutableRate;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 
 /**
  * Test the MetricsSystemImpl class
@@ -80,7 +81,7 @@ public class TestMetricsSystemImpl {
     }
   }
 
-  @Test public void testInitFirst() throws Exception {
+  @Test public void testInitFirstVerifyStopInvokedImmediately() throws 
Exception {
     new ConfigBuilder().add("*.period", 8)
         //.add("test.sink.plugin.urls", getPluginUrlsAsString())
         .add("test.sink.test.class", TestSink.class.getName())
@@ -106,14 +107,61 @@ public class TestMetricsSystemImpl {
     ms.stop();
     ms.shutdown();
 
-    verify(sink1, times(2)).putMetrics(r1.capture());
+    //When we call stop, at most two sources will be consumed by each sink 
thread.
+    verify(sink1, atMost(2)).putMetrics(r1.capture());
+    List<MetricsRecord> mr1 = r1.getAllValues();
+    verify(sink2, atMost(2)).putMetrics(r2.capture());
+    List<MetricsRecord> mr2 = r2.getAllValues();
+    if (mr1.size() != 0 && mr2.size() != 0) {
+      checkMetricsRecords(mr1);
+      assertEquals("output", mr1, mr2);
+    } else if (mr1.size() != 0) {
+      checkMetricsRecords(mr1);
+    } else if (mr2.size() != 0) {
+      checkMetricsRecords(mr2);
+    }
+  }
+
+  @Test public void testInitFirstVerifyCallBacks() throws Exception {
+    DefaultMetricsSystem.shutdown(); 
+    new ConfigBuilder().add("*.period", 8)
+        //.add("test.sink.plugin.urls", getPluginUrlsAsString())
+        .add("test.sink.test.class", TestSink.class.getName())
+        .add("test.*.source.filter.exclude", "s0")
+        .add("test.source.s1.metric.filter.exclude", "X*")
+        .add("test.sink.sink1.metric.filter.exclude", "Y*")
+        .add("test.sink.sink2.metric.filter.exclude", "Y*")
+        .save(TestMetricsConfig.getTestFilename("hadoop-metrics2-test"));
+    MetricsSystemImpl ms = new MetricsSystemImpl("Test");
+    ms.start();
+    ms.register("s0", "s0 desc", new TestSource("s0rec"));
+    TestSource s1 = ms.register("s1", "s1 desc", new TestSource("s1rec"));
+    s1.c1.incr();
+    s1.xxx.incr();
+    s1.g1.set(2);
+    s1.yyy.incr(2);
+    s1.s1.add(0);
+    MetricsSink sink1 = mock(MetricsSink.class);
+    MetricsSink sink2 = mock(MetricsSink.class);
+    ms.registerSink("sink1", "sink1 desc", sink1);
+    ms.registerSink("sink2", "sink2 desc", sink2);
+    ms.publishMetricsNow(); // publish the metrics
+
+    try {
+      verify(sink1, timeout(200).times(2)).putMetrics(r1.capture());
+      verify(sink2, timeout(200).times(2)).putMetrics(r2.capture());
+    } finally {
+      ms.stop();
+      ms.shutdown();
+    }
+    //When we call stop, at most two sources will be consumed by each sink 
thread.
     List<MetricsRecord> mr1 = r1.getAllValues();
-    verify(sink2, times(2)).putMetrics(r2.capture());
     List<MetricsRecord> mr2 = r2.getAllValues();
     checkMetricsRecords(mr1);
     assertEquals("output", mr1, mr2);
-  }
 
+  }
+  
   @Test public void testMultiThreadedPublish() throws Exception {
     new ConfigBuilder().add("*.period", 80)
       .add("test.sink.Collector.queue.capacity", "20")

Modified: 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tools/GetGroupsTestBase.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tools/GetGroupsTestBase.java?rev=1446832&r1=1446831&r2=1446832&view=diff
==============================================================================
--- 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tools/GetGroupsTestBase.java
 (original)
+++ 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tools/GetGroupsTestBase.java
 Sat Feb 16 01:12:07 2013
@@ -108,7 +108,7 @@ public abstract class GetGroupsTestBase 
     for (String group : user.getGroupNames()) {
       expectedOutput += " " + group;
     }
-    return expectedOutput + "\n";
+    return expectedOutput + System.getProperty("line.separator");
   }
   
   private String runTool(Configuration conf, String[] args, boolean success)

Modified: 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java?rev=1446832&r1=1446831&r2=1446832&view=diff
==============================================================================
--- 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java
 (original)
+++ 
hadoop/common/branches/HDFS-347/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java
 Sat Feb 16 01:12:07 2013
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.util;
 
+import static 
org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix.long2String;
+import static 
org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix.string2long;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
@@ -26,6 +28,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.test.UnitTestcaseTimeLimit;
+import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
 import org.junit.Test;
 
 public class TestStringUtils extends UnitTestcaseTimeLimit {
@@ -134,45 +137,34 @@ public class TestStringUtils extends Uni
   
   @Test
   public void testTraditionalBinaryPrefix() throws Exception {
+    //test string2long(..)
     String[] symbol = {"k", "m", "g", "t", "p", "e"};
     long m = 1024;
     for(String s : symbol) {
-      assertEquals(0, StringUtils.TraditionalBinaryPrefix.string2long(0 + s));
-      assertEquals(m, StringUtils.TraditionalBinaryPrefix.string2long(1 + s));
+      assertEquals(0, string2long(0 + s));
+      assertEquals(m, string2long(1 + s));
       m *= 1024;
     }
     
-    assertEquals(0L, StringUtils.TraditionalBinaryPrefix.string2long("0"));
-    assertEquals(1024L, StringUtils.TraditionalBinaryPrefix.string2long("1k"));
-    assertEquals(-1024L, 
StringUtils.TraditionalBinaryPrefix.string2long("-1k"));
-    assertEquals(1259520L,
-        StringUtils.TraditionalBinaryPrefix.string2long("1230K"));
-    assertEquals(-1259520L,
-        StringUtils.TraditionalBinaryPrefix.string2long("-1230K"));
-    assertEquals(104857600L,
-        StringUtils.TraditionalBinaryPrefix.string2long("100m"));
-    assertEquals(-104857600L,
-        StringUtils.TraditionalBinaryPrefix.string2long("-100M"));
-    assertEquals(956703965184L,
-        StringUtils.TraditionalBinaryPrefix.string2long("891g"));
-    assertEquals(-956703965184L,
-        StringUtils.TraditionalBinaryPrefix.string2long("-891G"));
-    assertEquals(501377302265856L,
-        StringUtils.TraditionalBinaryPrefix.string2long("456t"));
-    assertEquals(-501377302265856L,
-        StringUtils.TraditionalBinaryPrefix.string2long("-456T"));
-    assertEquals(11258999068426240L,
-        StringUtils.TraditionalBinaryPrefix.string2long("10p"));
-    assertEquals(-11258999068426240L,
-        StringUtils.TraditionalBinaryPrefix.string2long("-10P"));
-    assertEquals(1152921504606846976L,
-        StringUtils.TraditionalBinaryPrefix.string2long("1e"));
-    assertEquals(-1152921504606846976L,
-        StringUtils.TraditionalBinaryPrefix.string2long("-1E"));
+    assertEquals(0L, string2long("0"));
+    assertEquals(1024L, string2long("1k"));
+    assertEquals(-1024L, string2long("-1k"));
+    assertEquals(1259520L, string2long("1230K"));
+    assertEquals(-1259520L, string2long("-1230K"));
+    assertEquals(104857600L, string2long("100m"));
+    assertEquals(-104857600L, string2long("-100M"));
+    assertEquals(956703965184L, string2long("891g"));
+    assertEquals(-956703965184L, string2long("-891G"));
+    assertEquals(501377302265856L, string2long("456t"));
+    assertEquals(-501377302265856L, string2long("-456T"));
+    assertEquals(11258999068426240L, string2long("10p"));
+    assertEquals(-11258999068426240L, string2long("-10P"));
+    assertEquals(1152921504606846976L, string2long("1e"));
+    assertEquals(-1152921504606846976L, string2long("-1E"));
 
     String tooLargeNumStr = "10e";
     try {
-      StringUtils.TraditionalBinaryPrefix.string2long(tooLargeNumStr);
+      string2long(tooLargeNumStr);
       fail("Test passed for a number " + tooLargeNumStr + " too large");
     } catch (IllegalArgumentException e) {
       assertEquals(tooLargeNumStr + " does not fit in a Long", e.getMessage());
@@ -180,7 +172,7 @@ public class TestStringUtils extends Uni
 
     String tooSmallNumStr = "-10e";
     try {
-      StringUtils.TraditionalBinaryPrefix.string2long(tooSmallNumStr);
+      string2long(tooSmallNumStr);
       fail("Test passed for a number " + tooSmallNumStr + " too small");
     } catch (IllegalArgumentException e) {
       assertEquals(tooSmallNumStr + " does not fit in a Long", e.getMessage());
@@ -189,7 +181,7 @@ public class TestStringUtils extends Uni
     String invalidFormatNumStr = "10kb";
     char invalidPrefix = 'b';
     try {
-      StringUtils.TraditionalBinaryPrefix.string2long(invalidFormatNumStr);
+      string2long(invalidFormatNumStr);
       fail("Test passed for a number " + invalidFormatNumStr
           + " has invalid format");
     } catch (IllegalArgumentException e) {
@@ -199,6 +191,74 @@ public class TestStringUtils extends Uni
           e.getMessage());
     }
 
+    //test long2string(..)
+    assertEquals("0", long2String(0, null, 2));
+    for(int decimalPlace = 0; decimalPlace < 2; decimalPlace++) {
+      for(int n = 1; n < TraditionalBinaryPrefix.KILO.value; n++) {
+        assertEquals(n + "", long2String(n, null, decimalPlace));
+        assertEquals(-n + "", long2String(-n, null, decimalPlace));
+      }
+      assertEquals("1 K", long2String(1L << 10, null, decimalPlace));
+      assertEquals("-1 K", long2String(-1L << 10, null, decimalPlace));
+    }
+
+    assertEquals("8.00 E", long2String(Long.MAX_VALUE, null, 2));
+    assertEquals("8.00 E", long2String(Long.MAX_VALUE - 1, null, 2));
+    assertEquals("-8 E", long2String(Long.MIN_VALUE, null, 2));
+    assertEquals("-8.00 E", long2String(Long.MIN_VALUE + 1, null, 2));
+
+    final String[] zeros = {" ", ".0 ", ".00 "};
+    for(int decimalPlace = 0; decimalPlace < zeros.length; decimalPlace++) {
+      final String trailingZeros = zeros[decimalPlace]; 
+
+      for(int e = 11; e < Long.SIZE - 1; e++) {
+        final TraditionalBinaryPrefix p
+            = TraditionalBinaryPrefix.values()[e/10 - 1]; 
+  
+        { // n = 2^e
+          final long n = 1L << e;
+          final String expected = (n/p.value) + " " + p.symbol;
+          assertEquals("n=" + n, expected, long2String(n, null, 2));
+        }
+  
+        { // n = 2^e + 1
+          final long n = (1L << e) + 1;
+          final String expected = (n/p.value) + trailingZeros + p.symbol;
+          assertEquals("n=" + n, expected, long2String(n, null, decimalPlace));
+        }
+  
+        { // n = 2^e - 1
+          final long n = (1L << e) - 1;
+          final String expected = ((n+1)/p.value) + trailingZeros + p.symbol;
+          assertEquals("n=" + n, expected, long2String(n, null, decimalPlace));
+        }
+      }
+    }
+
+    assertEquals("1.50 K", long2String(3L << 9, null, 2));
+    assertEquals("1.5 K", long2String(3L << 9, null, 1));
+    assertEquals("1.50 M", long2String(3L << 19, null, 2));
+    assertEquals("2 M", long2String(3L << 19, null, 0));
+    assertEquals("3 G", long2String(3L << 30, null, 2));
+
+    // test byteDesc(..)
+    assertEquals("0 B", StringUtils.byteDesc(0));
+    assertEquals("-100 B", StringUtils.byteDesc(-100));
+    assertEquals("1 KB", StringUtils.byteDesc(1024));
+    assertEquals("1.50 KB", StringUtils.byteDesc(3L << 9));
+    assertEquals("1.50 MB", StringUtils.byteDesc(3L << 19));
+    assertEquals("3 GB", StringUtils.byteDesc(3L << 30));
+    
+    // test formatPercent(..)
+    assertEquals("10%", StringUtils.formatPercent(0.1, 0));
+    assertEquals("10.0%", StringUtils.formatPercent(0.1, 1));
+    assertEquals("10.00%", StringUtils.formatPercent(0.1, 2));
+
+    assertEquals("1%", StringUtils.formatPercent(0.00543, 0));
+    assertEquals("0.5%", StringUtils.formatPercent(0.00543, 1));
+    assertEquals("0.54%", StringUtils.formatPercent(0.00543, 2));
+    assertEquals("0.543%", StringUtils.formatPercent(0.00543, 3));
+    assertEquals("0.5430%", StringUtils.formatPercent(0.00543, 4));
   }
 
   @Test
@@ -314,10 +374,9 @@ public class TestStringUtils extends Uni
         }
         long et = System.nanoTime();
         if (outer > 3) {
-          System.out.println(
-            (useOurs ? "StringUtils impl" : "Java impl") +
-            " #" + outer + ":" +
-            (et - st)/1000000 + "ms");
+          System.out.println( (useOurs ? "StringUtils impl" : "Java impl")
+              + " #" + outer + ":" + (et - st)/1000000 + "ms, components="
+              + components  );
         }
       }
     }


Reply via email to