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 sz...@apache.org on 2013/03/07 03:57:46 UTC

svn commit: r1453669 [5/5] - in /hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common: ./ src/ src/main/bin/ src/main/conf/ src/main/docs/ src/main/docs/src/documentation/content/xdocs/ src/main/java/ src/main/java/org/apache/hadoop/fs/...

Modified: hadoop/common/branches/HDFS-2802/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-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java?rev=1453669&r1=1453668&r2=1453669&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java Thu Mar  7 02:57:40 2013
@@ -21,6 +21,8 @@ import java.io.File;
 import java.io.FileDescriptor;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.FileWriter;
 import java.io.IOException;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.ArrayList;
@@ -60,11 +62,15 @@ public class TestNativeIO {
     TEST_DIR.mkdirs();
   }
 
-  @Test
+  @Test (timeout = 30000)
   public void testFstat() throws Exception {
+    if (Path.WINDOWS) {
+      return;
+    }
+
     FileOutputStream fos = new FileOutputStream(
       new File(TEST_DIR, "testfstat"));
-    NativeIO.Stat stat = NativeIO.getFstat(fos.getFD());
+    NativeIO.POSIX.Stat stat = NativeIO.POSIX.getFstat(fos.getFD());
     fos.close();
     LOG.info("Stat: " + String.valueOf(stat));
 
@@ -72,7 +78,8 @@ public class TestNativeIO {
     assertNotNull(stat.getGroup());
     assertTrue(!stat.getGroup().isEmpty());
     assertEquals("Stat mode field should indicate a regular file",
-      NativeIO.Stat.S_IFREG, stat.getMode() & NativeIO.Stat.S_IFMT);
+      NativeIO.POSIX.Stat.S_IFREG,
+      stat.getMode() & NativeIO.POSIX.Stat.S_IFMT);
   }
 
   /**
@@ -81,8 +88,12 @@ public class TestNativeIO {
    * NOTE: this test is likely to fail on RHEL 6.0 which has a non-threadsafe
    * implementation of getpwuid_r.
    */
-  @Test
+  @Test (timeout = 30000)
   public void testMultiThreadedFstat() throws Exception {
+    if (Path.WINDOWS) {
+      return;
+    }
+
     final FileOutputStream fos = new FileOutputStream(
       new File(TEST_DIR, "testfstat"));
 
@@ -96,12 +107,13 @@ public class TestNativeIO {
           long et = Time.now() + 5000;
           while (Time.now() < et) {
             try {
-              NativeIO.Stat stat = NativeIO.getFstat(fos.getFD());
+              NativeIO.POSIX.Stat stat = NativeIO.POSIX.getFstat(fos.getFD());
               assertEquals(System.getProperty("user.name"), stat.getOwner());
               assertNotNull(stat.getGroup());
               assertTrue(!stat.getGroup().isEmpty());
               assertEquals("Stat mode field should indicate a regular file",
-                NativeIO.Stat.S_IFREG, stat.getMode() & NativeIO.Stat.S_IFMT);
+                NativeIO.POSIX.Stat.S_IFREG,
+                stat.getMode() & NativeIO.POSIX.Stat.S_IFMT);
             } catch (Throwable t) {
               thrown.set(t);
             }
@@ -122,26 +134,123 @@ public class TestNativeIO {
     }
   }
 
-  @Test
+  @Test (timeout = 30000)
   public void testFstatClosedFd() throws Exception {
+    if (Path.WINDOWS) {
+      return;
+    }
+
     FileOutputStream fos = new FileOutputStream(
       new File(TEST_DIR, "testfstat2"));
     fos.close();
     try {
-      NativeIO.Stat stat = NativeIO.getFstat(fos.getFD());
+      NativeIO.POSIX.Stat stat = NativeIO.POSIX.getFstat(fos.getFD());
     } catch (NativeIOException nioe) {
       LOG.info("Got expected exception", nioe);
       assertEquals(Errno.EBADF, nioe.getErrno());
     }
   }
 
-  @Test
+  @Test (timeout = 30000)
+  public void testSetFilePointer() throws Exception {
+    if (!Path.WINDOWS) {
+      return;
+    }
+
+    LOG.info("Set a file pointer on Windows");
+    try {
+      File testfile = new File(TEST_DIR, "testSetFilePointer");
+      assertTrue("Create test subject",
+          testfile.exists() || testfile.createNewFile());
+      FileWriter writer = new FileWriter(testfile);
+      try {
+        for (int i = 0; i < 200; i++)
+          if (i < 100)
+            writer.write('a');
+          else
+            writer.write('b');
+        writer.flush();
+      } catch (Exception writerException) {
+        fail("Got unexpected exception: " + writerException.getMessage());
+      } finally {
+        writer.close();
+      }
+
+      FileDescriptor fd = NativeIO.Windows.createFile(
+          testfile.getCanonicalPath(),
+          NativeIO.Windows.GENERIC_READ,
+          NativeIO.Windows.FILE_SHARE_READ |
+          NativeIO.Windows.FILE_SHARE_WRITE |
+          NativeIO.Windows.FILE_SHARE_DELETE,
+          NativeIO.Windows.OPEN_EXISTING);
+      NativeIO.Windows.setFilePointer(fd, 120, NativeIO.Windows.FILE_BEGIN);
+      FileReader reader = new FileReader(fd);
+      try {
+        int c = reader.read();
+        assertTrue("Unexpected character: " + c, c == 'b');
+      } catch (Exception readerException) {
+        fail("Got unexpected exception: " + readerException.getMessage());
+      } finally {
+        reader.close();
+      }
+    } catch (Exception e) {
+      fail("Got unexpected exception: " + e.getMessage());
+    }
+  }
+
+  @Test (timeout = 30000)
+  public void testCreateFile() throws Exception {
+    if (!Path.WINDOWS) {
+      return;
+    }
+
+    LOG.info("Open a file on Windows with SHARE_DELETE shared mode");
+    try {
+      File testfile = new File(TEST_DIR, "testCreateFile");
+      assertTrue("Create test subject",
+          testfile.exists() || testfile.createNewFile());
+
+      FileDescriptor fd = NativeIO.Windows.createFile(
+          testfile.getCanonicalPath(),
+          NativeIO.Windows.GENERIC_READ,
+          NativeIO.Windows.FILE_SHARE_READ |
+          NativeIO.Windows.FILE_SHARE_WRITE |
+          NativeIO.Windows.FILE_SHARE_DELETE,
+          NativeIO.Windows.OPEN_EXISTING);
+
+      FileInputStream fin = new FileInputStream(fd);
+      try {
+        fin.read();
+
+        File newfile = new File(TEST_DIR, "testRenamedFile");
+
+        boolean renamed = testfile.renameTo(newfile);
+        assertTrue("Rename failed.", renamed);
+
+        fin.read();
+      } catch (Exception e) {
+        fail("Got unexpected exception: " + e.getMessage());
+      }
+      finally {
+        fin.close();
+      }
+    } catch (Exception e) {
+      fail("Got unexpected exception: " + e.getMessage());
+    }
+
+  }
+
+  @Test (timeout = 30000)
   public void testOpenMissingWithoutCreate() throws Exception {
+    if (Path.WINDOWS) {
+      return;
+    }
+
     LOG.info("Open a missing file without O_CREAT and it should fail");
     try {
-      FileDescriptor fd = NativeIO.open(
+      FileDescriptor fd = NativeIO.POSIX.open(
         new File(TEST_DIR, "doesntexist").getAbsolutePath(),
-        NativeIO.O_WRONLY, 0700);
+        NativeIO.POSIX.O_WRONLY, 0700);
       fail("Able to open a new file without O_CREAT");
     } catch (NativeIOException nioe) {
       LOG.info("Got expected exception", nioe);
@@ -149,12 +258,16 @@ public class TestNativeIO {
     }
   }
 
-  @Test
+  @Test (timeout = 30000)
   public void testOpenWithCreate() throws Exception {
+    if (Path.WINDOWS) {
+      return;
+    }
+
     LOG.info("Test creating a file with O_CREAT");
-    FileDescriptor fd = NativeIO.open(
+    FileDescriptor fd = NativeIO.POSIX.open(
       new File(TEST_DIR, "testWorkingOpen").getAbsolutePath(),
-      NativeIO.O_WRONLY | NativeIO.O_CREAT, 0700);
+      NativeIO.POSIX.O_WRONLY | NativeIO.POSIX.O_CREAT, 0700);
     assertNotNull(true);
     assertTrue(fd.valid());
     FileOutputStream fos = new FileOutputStream(fd);
@@ -165,9 +278,9 @@ public class TestNativeIO {
 
     LOG.info("Test exclusive create");
     try {
-      fd = NativeIO.open(
+      fd = NativeIO.POSIX.open(
         new File(TEST_DIR, "testWorkingOpen").getAbsolutePath(),
-        NativeIO.O_WRONLY | NativeIO.O_CREAT | NativeIO.O_EXCL, 0700);
+        NativeIO.POSIX.O_WRONLY | NativeIO.POSIX.O_CREAT | NativeIO.POSIX.O_EXCL, 0700);
       fail("Was able to create existing file with O_EXCL");
     } catch (NativeIOException nioe) {
       LOG.info("Got expected exception for failed exclusive create", nioe);
@@ -179,12 +292,16 @@ public class TestNativeIO {
    * Test that opens and closes a file 10000 times - this would crash with
    * "Too many open files" if we leaked fds using this access pattern.
    */
-  @Test
+  @Test (timeout = 30000)
   public void testFDDoesntLeak() throws IOException {
+    if (Path.WINDOWS) {
+      return;
+    }
+
     for (int i = 0; i < 10000; i++) {
-      FileDescriptor fd = NativeIO.open(
+      FileDescriptor fd = NativeIO.POSIX.open(
         new File(TEST_DIR, "testNoFdLeak").getAbsolutePath(),
-        NativeIO.O_WRONLY | NativeIO.O_CREAT, 0700);
+        NativeIO.POSIX.O_WRONLY | NativeIO.POSIX.O_CREAT, 0700);
       assertNotNull(true);
       assertTrue(fd.valid());
       FileOutputStream fos = new FileOutputStream(fd);
@@ -196,10 +313,14 @@ public class TestNativeIO {
   /**
    * Test basic chmod operation
    */
-  @Test
+  @Test (timeout = 30000)
   public void testChmod() throws Exception {
+    if (Path.WINDOWS) {
+      return;
+    }
+
     try {
-      NativeIO.chmod("/this/file/doesnt/exist", 777);
+      NativeIO.POSIX.chmod("/this/file/doesnt/exist", 777);
       fail("Chmod of non-existent file didn't fail");
     } catch (NativeIOException nioe) {
       assertEquals(Errno.ENOENT, nioe.getErrno());
@@ -208,21 +329,26 @@ public class TestNativeIO {
     File toChmod = new File(TEST_DIR, "testChmod");
     assertTrue("Create test subject",
                toChmod.exists() || toChmod.mkdir());
-    NativeIO.chmod(toChmod.getAbsolutePath(), 0777);
+    NativeIO.POSIX.chmod(toChmod.getAbsolutePath(), 0777);
     assertPermissions(toChmod, 0777);
-    NativeIO.chmod(toChmod.getAbsolutePath(), 0000);
+    NativeIO.POSIX.chmod(toChmod.getAbsolutePath(), 0000);
     assertPermissions(toChmod, 0000);
-    NativeIO.chmod(toChmod.getAbsolutePath(), 0644);
+    NativeIO.POSIX.chmod(toChmod.getAbsolutePath(), 0644);
     assertPermissions(toChmod, 0644);
   }
 
 
-  @Test
+  @Test (timeout = 30000)
   public void testPosixFadvise() throws Exception {
+    if (Path.WINDOWS) {
+      return;
+    }
+
     FileInputStream fis = new FileInputStream("/dev/zero");
     try {
-      NativeIO.posix_fadvise(fis.getFD(), 0, 0,
-                             NativeIO.POSIX_FADV_SEQUENTIAL);
+      NativeIO.POSIX.posix_fadvise(
+          fis.getFD(), 0, 0,
+          NativeIO.POSIX.POSIX_FADV_SEQUENTIAL);
     } catch (UnsupportedOperationException uoe) {
       // we should just skip the unit test on machines where we don't
       // have fadvise support
@@ -235,8 +361,9 @@ public class TestNativeIO {
     }
 
     try {
-      NativeIO.posix_fadvise(fis.getFD(), 0, 1024,
-                             NativeIO.POSIX_FADV_SEQUENTIAL);
+      NativeIO.POSIX.posix_fadvise(
+          fis.getFD(), 0, 1024,
+          NativeIO.POSIX.POSIX_FADV_SEQUENTIAL);
 
       fail("Did not throw on bad file");
     } catch (NativeIOException nioe) {
@@ -244,8 +371,9 @@ public class TestNativeIO {
     }
     
     try {
-      NativeIO.posix_fadvise(null, 0, 1024,
-                             NativeIO.POSIX_FADV_SEQUENTIAL);
+      NativeIO.POSIX.posix_fadvise(
+          null, 0, 1024,
+          NativeIO.POSIX.POSIX_FADV_SEQUENTIAL);
 
       fail("Did not throw on null file");
     } catch (NullPointerException npe) {
@@ -253,14 +381,15 @@ public class TestNativeIO {
     }
   }
 
-  @Test
+  @Test (timeout = 30000)
   public void testSyncFileRange() throws Exception {
     FileOutputStream fos = new FileOutputStream(
       new File(TEST_DIR, "testSyncFileRange"));
     try {
       fos.write("foo".getBytes());
-      NativeIO.sync_file_range(fos.getFD(), 0, 1024,
-                               NativeIO.SYNC_FILE_RANGE_WRITE);
+      NativeIO.POSIX.sync_file_range(
+          fos.getFD(), 0, 1024,
+          NativeIO.POSIX.SYNC_FILE_RANGE_WRITE);
       // no way to verify that this actually has synced,
       // but if it doesn't throw, we can assume it worked
     } catch (UnsupportedOperationException uoe) {
@@ -271,8 +400,9 @@ public class TestNativeIO {
       fos.close();
     }
     try {
-      NativeIO.sync_file_range(fos.getFD(), 0, 1024,
-                               NativeIO.SYNC_FILE_RANGE_WRITE);
+      NativeIO.POSIX.sync_file_range(
+          fos.getFD(), 0, 1024,
+          NativeIO.POSIX.SYNC_FILE_RANGE_WRITE);
       fail("Did not throw on bad file");
     } catch (NativeIOException nioe) {
       assertEquals(Errno.EBADF, nioe.getErrno());
@@ -286,17 +416,25 @@ public class TestNativeIO {
     assertEquals(expected, perms.toShort());
   }
 
-  @Test
+  @Test (timeout = 30000)
   public void testGetUserName() throws IOException {
-    assertFalse(NativeIO.getUserName(0).isEmpty());
+    if (Path.WINDOWS) {
+      return;
+    }
+
+    assertFalse(NativeIO.POSIX.getUserName(0).isEmpty());
   }
 
-  @Test
+  @Test (timeout = 30000)
   public void testGetGroupName() throws IOException {
-    assertFalse(NativeIO.getGroupName(0).isEmpty());
+    if (Path.WINDOWS) {
+      return;
+    }
+
+    assertFalse(NativeIO.POSIX.getGroupName(0).isEmpty());
   }
 
-  @Test
+  @Test (timeout = 30000)
   public void testRenameTo() throws Exception {
     final File TEST_DIR = new File(new File(
         System.getProperty("test.build.data","build/test/data")), "renameTest");

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java?rev=1453669&r1=1453668&r2=1453669&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java Thu Mar  7 02:57:40 2013
@@ -22,6 +22,7 @@ import static org.apache.hadoop.test.Met
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.net.URISyntaxException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto;
@@ -83,6 +84,13 @@ public class TestProtoBufRpc {
         EmptyRequestProto request) throws ServiceException {
       throw new ServiceException("error", new RpcServerException("error"));
     }
+    
+    @Override
+    public EmptyResponseProto error2(RpcController unused,
+        EmptyRequestProto request) throws ServiceException {
+      throw new ServiceException("error", new URISyntaxException("",
+          "testException"));
+    }
   }
   
   public static class PBServer2Impl implements TestRpcService2 {
@@ -149,7 +157,7 @@ public class TestProtoBufRpc {
         conf);
   }
 
-  @Test
+  @Test (timeout=5000)
   public void testProtoBufRpc() throws Exception {
     TestRpcService client = getClient();
     testProtoBufRpc(client);
@@ -178,7 +186,7 @@ public class TestProtoBufRpc {
     }
   }
   
-  @Test
+  @Test (timeout=5000)
   public void testProtoBufRpc2() throws Exception {
     TestRpcService2 client = getClient2();
     
@@ -201,4 +209,20 @@ public class TestProtoBufRpc {
         getMetrics(server.getRpcDetailedMetrics().name());
     assertCounterGt("Echo2NumOps", 0L, rpcDetailedMetrics);
   }
+
+  @Test (timeout=5000)
+  public void testProtoBufRandomException() throws Exception {
+    TestRpcService client = getClient();
+    EmptyRequestProto emptyRequest = EmptyRequestProto.newBuilder().build();
+
+    try {
+      client.error2(null, emptyRequest);
+    } catch (ServiceException se) {
+      Assert.assertTrue(se.getCause() instanceof RemoteException);
+      RemoteException re = (RemoteException) se.getCause();
+      Assert.assertTrue(re.getClassName().equals(
+          URISyntaxException.class.getName()));
+      Assert.assertTrue(re.getMessage().contains("testException"));
+    }
+  }
 }
\ No newline at end of file

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java?rev=1453669&r1=1453668&r2=1453669&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java Thu Mar  7 02:57:40 2013
@@ -127,6 +127,7 @@ public class TestDoAsEffectiveUser {
     public static final long versionID = 1L;
 
     String aMethod() throws IOException;
+    String getServerRemoteUser() throws IOException;
   }
 
   public class TestImpl implements TestProtocol {
@@ -137,6 +138,11 @@ public class TestDoAsEffectiveUser {
     }
 
     @Override
+    public String getServerRemoteUser() throws IOException {
+      return Server.getRemoteUser().toString();
+    }
+    
+    @Override
     public long getProtocolVersion(String protocol, long clientVersion)
         throws IOException {
       return TestProtocol.versionID;
@@ -149,7 +155,23 @@ public class TestDoAsEffectiveUser {
     }
   }
 
-  @Test
+  private void checkRemoteUgi(final Server server,
+      final UserGroupInformation ugi, final Configuration conf)
+          throws Exception {
+    ugi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws IOException {
+        proxy = RPC.getProxy(
+            TestProtocol.class, TestProtocol.versionID,
+            NetUtils.getConnectAddress(server), conf);
+        Assert.assertEquals(ugi.toString(), proxy.aMethod());
+        Assert.assertEquals(ugi.toString(), proxy.getServerRemoteUser());
+        return null;
+      }
+    });    
+  }
+  
+  @Test(timeout=4000)
   public void testRealUserSetup() throws IOException {
     final Configuration conf = new Configuration();
     conf.setStrings(ProxyUsers
@@ -163,24 +185,13 @@ public class TestDoAsEffectiveUser {
     try {
       server.start();
 
-      final InetSocketAddress addr = NetUtils.getConnectAddress(server);
-
       UserGroupInformation realUserUgi = UserGroupInformation
           .createRemoteUser(REAL_USER_NAME);
+      checkRemoteUgi(server, realUserUgi, conf);
+      
       UserGroupInformation proxyUserUgi = UserGroupInformation.createProxyUserForTesting(
           PROXY_USER_NAME, realUserUgi, GROUP_NAMES);
-      String retVal = proxyUserUgi
-          .doAs(new PrivilegedExceptionAction<String>() {
-            @Override
-            public String run() throws IOException {
-              proxy = RPC.getProxy(TestProtocol.class,
-                  TestProtocol.versionID, addr, conf);
-              String ret = proxy.aMethod();
-              return ret;
-            }
-          });
-
-      Assert.assertEquals(PROXY_USER_NAME + " (auth:SIMPLE) via " + REAL_USER_NAME + " (auth:SIMPLE)", retVal);
+      checkRemoteUgi(server, proxyUserUgi, conf);
     } catch (Exception e) {
       e.printStackTrace();
       Assert.fail();
@@ -192,7 +203,7 @@ public class TestDoAsEffectiveUser {
     }
   }
 
-  @Test
+  @Test(timeout=4000)
   public void testRealUserAuthorizationSuccess() throws IOException {
     final Configuration conf = new Configuration();
     configureSuperUserIPAddresses(conf, REAL_USER_SHORT_NAME);
@@ -206,25 +217,13 @@ public class TestDoAsEffectiveUser {
     try {
       server.start();
 
-      final InetSocketAddress addr = NetUtils.getConnectAddress(server);
-
       UserGroupInformation realUserUgi = UserGroupInformation
           .createRemoteUser(REAL_USER_NAME);
+      checkRemoteUgi(server, realUserUgi, conf);
 
       UserGroupInformation proxyUserUgi = UserGroupInformation
           .createProxyUserForTesting(PROXY_USER_NAME, realUserUgi, GROUP_NAMES);
-      String retVal = proxyUserUgi
-          .doAs(new PrivilegedExceptionAction<String>() {
-            @Override
-            public String run() throws IOException {
-              proxy = RPC.getProxy(TestProtocol.class,
-                  TestProtocol.versionID, addr, conf);
-              String ret = proxy.aMethod();
-              return ret;
-            }
-          });
-
-      Assert.assertEquals(PROXY_USER_NAME + " (auth:SIMPLE) via " + REAL_USER_NAME + " (auth:SIMPLE)", retVal);
+      checkRemoteUgi(server, proxyUserUgi, conf);
     } catch (Exception e) {
       e.printStackTrace();
       Assert.fail();

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java?rev=1453669&r1=1453668&r2=1453669&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java Thu Mar  7 02:57:40 2013
@@ -42,6 +42,7 @@ import org.apache.hadoop.security.token.
 import org.apache.hadoop.security.token.TokenIdentifier;
 import static org.apache.hadoop.test.MetricsAsserts.*;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.util.Shell;
 
 public class TestUserGroupInformation {
   final private static String USER_NAME = "user1@HADOOP.APACHE.ORG";
@@ -90,17 +91,17 @@ public class TestUserGroupInformation {
     UserGroupInformation.setLoginUser(null);
   }
 
-  @Test
+  @Test (timeout = 30000)
   public void testSimpleLogin() throws IOException {
     tryLoginAuthenticationMethod(AuthenticationMethod.SIMPLE, true);
   }
 
-  @Test
+  @Test (timeout = 30000)
   public void testTokenLogin() throws IOException {
     tryLoginAuthenticationMethod(AuthenticationMethod.TOKEN, false);
   }
   
-  @Test
+  @Test (timeout = 30000)
   public void testProxyLogin() throws IOException {
     tryLoginAuthenticationMethod(AuthenticationMethod.PROXY, false);
   }
@@ -129,7 +130,7 @@ public class TestUserGroupInformation {
     }
   }
   
-  @Test
+  @Test (timeout = 30000)
   public void testGetRealAuthenticationMethod() {
     UserGroupInformation ugi = UserGroupInformation.createRemoteUser("user1");
     ugi.setAuthenticationMethod(AuthenticationMethod.SIMPLE);
@@ -140,7 +141,7 @@ public class TestUserGroupInformation {
     assertEquals(AuthenticationMethod.SIMPLE, ugi.getRealAuthenticationMethod());
   }
   /** Test login method */
-  @Test
+  @Test (timeout = 30000)
   public void testLogin() throws Exception {
     // login from unix
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
@@ -167,7 +168,7 @@ public class TestUserGroupInformation {
    * given user name - get all the groups.
    * Needs to happen before creating the test users
    */
-  @Test
+  @Test (timeout = 30000)
   public void testGetServerSideGroups() throws IOException,
                                                InterruptedException {
     // get the user name
@@ -175,19 +176,38 @@ public class TestUserGroupInformation {
     BufferedReader br = new BufferedReader
                           (new InputStreamReader(pp.getInputStream()));
     String userName = br.readLine().trim();
+    // If on windows domain, token format is DOMAIN\\user and we want to
+    // extract only the user name
+    if(Shell.WINDOWS) {
+      int sp = userName.lastIndexOf('\\');
+      if (sp != -1) {
+        userName = userName.substring(sp + 1);
+      }
+      // user names are case insensitive on Windows. Make consistent
+      userName = userName.toLowerCase();
+    }
     // get the groups
-    pp = Runtime.getRuntime().exec("id -Gn " + userName);
+    pp = Runtime.getRuntime().exec(Shell.WINDOWS ?
+      Shell.WINUTILS + " groups -F" : "id -Gn");
     br = new BufferedReader(new InputStreamReader(pp.getInputStream()));
     String line = br.readLine();
+
     System.out.println(userName + ":" + line);
    
     Set<String> groups = new LinkedHashSet<String> ();    
-    for(String s: line.split("[\\s]")) {
+    String[] tokens = line.split(Shell.TOKEN_SEPARATOR_REGEX);
+    for(String s: tokens) {
       groups.add(s);
     }
     
     final UserGroupInformation login = UserGroupInformation.getCurrentUser();
-    assertEquals(userName, login.getShortUserName());
+    String loginUserName = login.getShortUserName();
+    if(Shell.WINDOWS) {
+      // user names are case insensitive on Windows. Make consistent
+      loginUserName = loginUserName.toLowerCase();
+    }
+    assertEquals(userName, loginUserName);
+
     String[] gi = login.getGroupNames();
     assertEquals(groups.size(), gi.length);
     for(int i=0; i < gi.length; i++) {
@@ -208,7 +228,7 @@ public class TestUserGroupInformation {
   }
 
   /** test constructor */
-  @Test
+  @Test (timeout = 30000)
   public void testConstructor() throws Exception {
     UserGroupInformation ugi = 
       UserGroupInformation.createUserForTesting("user2/cron@HADOOP.APACHE.ORG", 
@@ -234,7 +254,7 @@ public class TestUserGroupInformation {
     assertTrue(gotException);
   }
 
-  @Test
+  @Test (timeout = 30000)
   public void testEquals() throws Exception {
     UserGroupInformation uugi = 
       UserGroupInformation.createUserForTesting(USER_NAME, GROUP_NAMES);
@@ -252,7 +272,7 @@ public class TestUserGroupInformation {
     assertEquals(uugi.hashCode(), ugi3.hashCode());
   }
   
-  @Test
+  @Test (timeout = 30000)
   public void testEqualsWithRealUser() throws Exception {
     UserGroupInformation realUgi1 = UserGroupInformation.createUserForTesting(
         "RealUser", GROUP_NAMES);
@@ -265,7 +285,7 @@ public class TestUserGroupInformation {
     assertFalse(remoteUgi.equals(proxyUgi1));
   }
   
-  @Test
+  @Test (timeout = 30000)
   public void testGettingGroups() throws Exception {
     UserGroupInformation uugi = 
       UserGroupInformation.createUserForTesting(USER_NAME, GROUP_NAMES);
@@ -275,7 +295,7 @@ public class TestUserGroupInformation {
   }
 
   @SuppressWarnings("unchecked") // from Mockito mocks
-  @Test
+  @Test (timeout = 30000)
   public <T extends TokenIdentifier> void testAddToken() throws Exception {
     UserGroupInformation ugi = 
         UserGroupInformation.createRemoteUser("someone"); 
@@ -313,7 +333,7 @@ public class TestUserGroupInformation {
   }
 
   @SuppressWarnings("unchecked") // from Mockito mocks
-  @Test
+  @Test (timeout = 30000)
   public <T extends TokenIdentifier> void testGetCreds() throws Exception {
     UserGroupInformation ugi = 
         UserGroupInformation.createRemoteUser("someone"); 
@@ -339,7 +359,7 @@ public class TestUserGroupInformation {
   }
 
   @SuppressWarnings("unchecked") // from Mockito mocks
-  @Test
+  @Test (timeout = 30000)
   public <T extends TokenIdentifier> void testAddCreds() throws Exception {
     UserGroupInformation ugi = 
         UserGroupInformation.createRemoteUser("someone"); 
@@ -364,7 +384,7 @@ public class TestUserGroupInformation {
     assertSame(secret, ugi.getCredentials().getSecretKey(secretKey));
   }
 
-  @Test
+  @Test (timeout = 30000)
   public <T extends TokenIdentifier> void testGetCredsNotSame()
       throws Exception {
     UserGroupInformation ugi = 
@@ -392,7 +412,7 @@ public class TestUserGroupInformation {
   }
 
   @SuppressWarnings("unchecked") // from Mockito mocks
-  @Test
+  @Test (timeout = 30000)
   public <T extends TokenIdentifier> void testAddNamedToken() throws Exception {
     UserGroupInformation ugi = 
         UserGroupInformation.createRemoteUser("someone"); 
@@ -413,7 +433,7 @@ public class TestUserGroupInformation {
   }
 
   @SuppressWarnings("unchecked") // from Mockito mocks
-  @Test
+  @Test (timeout = 30000)
   public <T extends TokenIdentifier> void testUGITokens() throws Exception {
     UserGroupInformation ugi = 
       UserGroupInformation.createUserForTesting("TheDoctor", 
@@ -459,7 +479,7 @@ public class TestUserGroupInformation {
     assertTrue(otherSet.contains(t2));
   }
   
-  @Test
+  @Test (timeout = 30000)
   public void testTokenIdentifiers() throws Exception {
     UserGroupInformation ugi = UserGroupInformation.createUserForTesting(
         "TheDoctor", new String[] { "TheTARDIS" });
@@ -487,7 +507,7 @@ public class TestUserGroupInformation {
     assertEquals(2, otherSet.size());
   }
 
-  @Test
+  @Test (timeout = 30000)
   public void testTestAuthMethod() throws Exception {
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
     // verify the reverse mappings works
@@ -499,7 +519,7 @@ public class TestUserGroupInformation {
     }
   }
   
-  @Test
+  @Test (timeout = 30000)
   public void testUGIAuthMethod() throws Exception {
     final UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
     final AuthenticationMethod am = AuthenticationMethod.KERBEROS;
@@ -515,7 +535,7 @@ public class TestUserGroupInformation {
     });
   }
   
-  @Test
+  @Test (timeout = 30000)
   public void testUGIAuthMethodInRealUser() throws Exception {
     final UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
     UserGroupInformation proxyUgi = UserGroupInformation.createProxyUser(
@@ -550,7 +570,7 @@ public class TestUserGroupInformation {
     Assert.assertEquals(proxyUgi3, proxyUgi4);
   }
   
-  @Test
+  @Test (timeout = 30000)
   public void testLoginObjectInSubject() throws Exception {
     UserGroupInformation loginUgi = UserGroupInformation.getLoginUser();
     UserGroupInformation anotherUgi = new UserGroupInformation(loginUgi
@@ -563,7 +583,7 @@ public class TestUserGroupInformation {
     Assert.assertTrue(login1 == login2);
   }
   
-  @Test
+  @Test (timeout = 30000)
   public void testLoginModuleCommit() throws Exception {
     UserGroupInformation loginUgi = UserGroupInformation.getLoginUser();
     User user1 = loginUgi.getSubject().getPrincipals(User.class).iterator()
@@ -597,7 +617,7 @@ public class TestUserGroupInformation {
    * with it, but that Subject was not created by Hadoop (ie it has no
    * associated User principal)
    */
-  @Test
+  @Test (timeout = 30000)
   public void testUGIUnderNonHadoopContext() throws Exception {
     Subject nonHadoopSubject = new Subject();
     Subject.doAs(nonHadoopSubject, new PrivilegedExceptionAction<Void>() {
@@ -611,7 +631,7 @@ public class TestUserGroupInformation {
   }
 
   /** Test hasSufficientTimeElapsed method */
-  @Test
+  @Test (timeout = 30000)
   public void testHasSufficientTimeElapsed() throws Exception {
     // Make hasSufficientTimeElapsed public
     Method method = UserGroupInformation.class
@@ -644,4 +664,11 @@ public class TestUserGroupInformation {
     // Restore hasSufficientTimElapsed back to private
     method.setAccessible(false);
   }
+  
+  @Test(timeout=1000)
+  public void testSetLoginUser() throws IOException {
+    UserGroupInformation ugi = UserGroupInformation.createRemoteUser("test-user");
+    UserGroupInformation.setLoginUser(ugi);
+    assertEquals(ugi, UserGroupInformation.getLoginUser());
+  }
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDiskChecker.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDiskChecker.java?rev=1453669&r1=1453668&r2=1453669&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDiskChecker.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDiskChecker.java Thu Mar  7 02:57:40 2013
@@ -30,24 +30,29 @@ import org.apache.hadoop.fs.LocalFileSys
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+import org.apache.hadoop.util.Shell;
 
 public class TestDiskChecker {
   final FsPermission defaultPerm = new FsPermission("755");
   final FsPermission invalidPerm = new FsPermission("000");
 
-  @Test public void testMkdirs_dirExists() throws Throwable {
+  @Test (timeout = 30000)
+  public void testMkdirs_dirExists() throws Throwable {
     _mkdirs(true, defaultPerm, defaultPerm);
   }
 
-  @Test public void testMkdirs_noDir() throws Throwable {
+  @Test (timeout = 30000)
+  public void testMkdirs_noDir() throws Throwable {
     _mkdirs(false, defaultPerm, defaultPerm);
   }
 
-  @Test public void testMkdirs_dirExists_badUmask() throws Throwable {
+  @Test (timeout = 30000)
+  public void testMkdirs_dirExists_badUmask() throws Throwable {
     _mkdirs(true, defaultPerm, invalidPerm);
   }
 
-  @Test public void testMkdirs_noDir_badUmask() throws Throwable {
+  @Test (timeout = 30000)
+  public void testMkdirs_noDir_badUmask() throws Throwable {
     _mkdirs(false, defaultPerm, invalidPerm);
   }
 
@@ -78,23 +83,28 @@ public class TestDiskChecker {
     }
   }
 
-  @Test public void testCheckDir_normal() throws Throwable {
+  @Test (timeout = 30000)
+  public void testCheckDir_normal() throws Throwable {
     _checkDirs(true, new FsPermission("755"), true);
   }
 
-  @Test public void testCheckDir_notDir() throws Throwable {
+  @Test (timeout = 30000)
+  public void testCheckDir_notDir() throws Throwable {
     _checkDirs(false, new FsPermission("000"), false);
   }
 
-  @Test public void testCheckDir_notReadable() throws Throwable {
+  @Test (timeout = 30000)
+  public void testCheckDir_notReadable() throws Throwable {
     _checkDirs(true, new FsPermission("000"), false);
   }
 
-  @Test public void testCheckDir_notWritable() throws Throwable {
+  @Test (timeout = 30000)
+  public void testCheckDir_notWritable() throws Throwable {
     _checkDirs(true, new FsPermission("444"), false);
   }
 
-  @Test public void testCheckDir_notListable() throws Throwable {
+  @Test (timeout = 30000)
+  public void testCheckDir_notListable() throws Throwable {
     _checkDirs(true, new FsPermission("666"), false);   // not listable
   }
 
@@ -130,27 +140,27 @@ public class TestDiskChecker {
    * permission for result of mapper.
    */
 
-  @Test
+  @Test (timeout = 30000)
   public void testCheckDir_normal_local() throws Throwable {
     _checkDirs(true, "755", true);
   }
 
-  @Test
+  @Test (timeout = 30000)
   public void testCheckDir_notDir_local() throws Throwable {
     _checkDirs(false, "000", false);
   }
 
-  @Test
+  @Test (timeout = 30000)
   public void testCheckDir_notReadable_local() throws Throwable {
     _checkDirs(true, "000", false);
   }
 
-  @Test
+  @Test (timeout = 30000)
   public void testCheckDir_notWritable_local() throws Throwable {
     _checkDirs(true, "444", false);
   }
 
-  @Test
+  @Test (timeout = 30000)
   public void testCheckDir_notListable_local() throws Throwable {
     _checkDirs(true, "666", false);
   }
@@ -160,8 +170,8 @@ public class TestDiskChecker {
     File localDir = File.createTempFile("test", "tmp");
     localDir.delete();
     localDir.mkdir();
-    Runtime.getRuntime().exec(
-	"chmod " + perm + "  " + localDir.getAbsolutePath()).waitFor();
+    Shell.execCommand(Shell.getSetPermissionCommand(perm, false,
+                                                    localDir.getAbsolutePath()));
     try {
       DiskChecker.checkDir(localDir);
       assertTrue("checkDir success", success);

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java?rev=1453669&r1=1453668&r2=1453669&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java Thu Mar  7 02:57:40 2013
@@ -44,7 +44,9 @@ public class TestGenericOptionsParser ex
     String[] args = new String[2];
     // pass a files option 
     args[0] = "-files";
-    args[1] = tmpFile.toString();
+    // Convert a file to a URI as File.toString() is not a valid URI on
+    // all platforms and GenericOptionsParser accepts only valid URIs
+    args[1] = tmpFile.toURI().toString();
     new GenericOptionsParser(conf, args);
     String files = conf.get("tmpfiles");
     assertNotNull("files is null", files);
@@ -53,7 +55,7 @@ public class TestGenericOptionsParser ex
     
     // pass file as uri
     Configuration conf1 = new Configuration();
-    URI tmpURI = new URI(tmpFile.toString() + "#link");
+    URI tmpURI = new URI(tmpFile.toURI().toString() + "#link");
     args[0] = "-files";
     args[1] = tmpURI.toString();
     new GenericOptionsParser(conf1, args);
@@ -148,7 +150,7 @@ public class TestGenericOptionsParser ex
     String[] args = new String[2];
     // pass a files option 
     args[0] = "-tokenCacheFile";
-    args[1] = tmpFile.toString();
+    args[1] = tmpFile.toURI().toString();
     
     // test non existing file
     Throwable th = null;

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java?rev=1453669&r1=1453668&r2=1453669&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java Thu Mar  7 02:57:40 2013
@@ -81,6 +81,10 @@ public class TestShell extends TestCase 
   }
   
   public void testShellCommandTimeout() throws Throwable {
+    if(Shell.WINDOWS) {
+      // setExecutable does not work on Windows
+      return;
+    }
     String rootDir = new File(System.getProperty(
         "test.build.data", "/tmp")).getAbsolutePath();
     File shellFile = new File(rootDir, "timeout.sh");

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java?rev=1453669&r1=1453668&r2=1453669&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java Thu Mar  7 02:57:40 2013
@@ -25,7 +25,10 @@ import static org.junit.Assert.assertEqu
 import static org.junit.Assert.fail;
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
 
 import org.apache.hadoop.test.UnitTestcaseTimeLimit;
 import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
@@ -43,7 +46,7 @@ public class TestStringUtils extends Uni
   final private static String ESCAPED_STR_WITH_BOTH2 = 
     "\\,A\\\\\\,\\,B\\\\\\\\\\,";
   
-  @Test
+  @Test (timeout = 30000)
   public void testEscapeString() throws Exception {
     assertEquals(NULL_STR, StringUtils.escapeString(NULL_STR));
     assertEquals(EMPTY_STR, StringUtils.escapeString(EMPTY_STR));
@@ -57,7 +60,7 @@ public class TestStringUtils extends Uni
         StringUtils.escapeString(STR_WITH_BOTH2));
   }
   
-  @Test
+  @Test (timeout = 30000)
   public void testSplit() throws Exception {
     assertEquals(NULL_STR, StringUtils.split(NULL_STR));
     String[] splits = StringUtils.split(EMPTY_STR);
@@ -87,7 +90,7 @@ public class TestStringUtils extends Uni
     assertEquals(ESCAPED_STR_WITH_BOTH2, splits[0]);    
   }
   
-  @Test
+  @Test (timeout = 30000)
   public void testSimpleSplit() throws Exception {
     final String[] TO_TEST = {
         "a/b/c",
@@ -103,7 +106,7 @@ public class TestStringUtils extends Uni
     }
   }
 
-  @Test
+  @Test (timeout = 30000)
   public void testUnescapeString() throws Exception {
     assertEquals(NULL_STR, StringUtils.unEscapeString(NULL_STR));
     assertEquals(EMPTY_STR, StringUtils.unEscapeString(EMPTY_STR));
@@ -135,7 +138,7 @@ public class TestStringUtils extends Uni
         StringUtils.unEscapeString(ESCAPED_STR_WITH_BOTH2));
   }
   
-  @Test
+  @Test (timeout = 30000)
   public void testTraditionalBinaryPrefix() throws Exception {
     //test string2long(..)
     String[] symbol = {"k", "m", "g", "t", "p", "e"};
@@ -261,7 +264,7 @@ public class TestStringUtils extends Uni
     assertEquals("0.5430%", StringUtils.formatPercent(0.00543, 4));
   }
 
-  @Test
+  @Test (timeout = 30000)
   public void testJoin() {
     List<String> s = new ArrayList<String>();
     s.add("a");
@@ -273,7 +276,7 @@ public class TestStringUtils extends Uni
     assertEquals("a:b:c", StringUtils.join(":", s.subList(0, 3)));
   }
   
-  @Test
+  @Test (timeout = 30000)
   public void testGetTrimmedStrings() throws Exception {
     String compactDirList = "/spindle1/hdfs,/spindle2/hdfs,/spindle3/hdfs";
     String spacedDirList = "/spindle1/hdfs, /spindle2/hdfs, /spindle3/hdfs";
@@ -295,7 +298,7 @@ public class TestStringUtils extends Uni
     assertArrayEquals(emptyArray, estring);
   } 
 
-  @Test
+  @Test (timeout = 30000)
   public void testCamelize() {
     // common use cases
     assertEquals("Map", StringUtils.camelize("MAP"));
@@ -331,7 +334,7 @@ public class TestStringUtils extends Uni
     assertEquals("Zz", StringUtils.camelize("zZ"));
   }
   
-  @Test
+  @Test (timeout = 30000)
   public void testStringToURI() {
     String[] str = new String[] { "file://" };
     try {
@@ -342,7 +345,7 @@ public class TestStringUtils extends Uni
     }
   }
 
-  @Test
+  @Test (timeout = 30000)
   public void testSimpleHostName() {
     assertEquals("Should return hostname when FQDN is specified",
             "hadoop01",
@@ -355,6 +358,49 @@ public class TestStringUtils extends Uni
             StringUtils.simpleHostname("10.10.5.68"));
   }
 
+  @Test (timeout = 5000)
+  public void testReplaceTokensShellEnvVars() {
+    Pattern pattern = StringUtils.SHELL_ENV_VAR_PATTERN;
+    Map<String, String> replacements = new HashMap<String, String>();
+    replacements.put("FOO", "one");
+    replacements.put("BAZ", "two");
+    replacements.put("NUMBERS123", "one-two-three");
+    replacements.put("UNDER_SCORES", "___");
+
+    assertEquals("one", StringUtils.replaceTokens("$FOO", pattern,
+      replacements));
+    assertEquals("two", StringUtils.replaceTokens("$BAZ", pattern,
+      replacements));
+    assertEquals("", StringUtils.replaceTokens("$BAR", pattern, replacements));
+    assertEquals("", StringUtils.replaceTokens("", pattern, replacements));
+    assertEquals("one-two-three", StringUtils.replaceTokens("$NUMBERS123",
+      pattern, replacements));
+    assertEquals("___", StringUtils.replaceTokens("$UNDER_SCORES", pattern,
+      replacements));
+    assertEquals("//one//two//", StringUtils.replaceTokens("//$FOO/$BAR/$BAZ//",
+      pattern, replacements));
+  }
+
+  @Test (timeout = 5000)
+  public void testReplaceTokensWinEnvVars() {
+    Pattern pattern = StringUtils.WIN_ENV_VAR_PATTERN;
+    Map<String, String> replacements = new HashMap<String, String>();
+    replacements.put("foo", "zoo");
+    replacements.put("baz", "zaz");
+
+    assertEquals("zoo", StringUtils.replaceTokens("%foo%", pattern,
+      replacements));
+    assertEquals("zaz", StringUtils.replaceTokens("%baz%", pattern,
+      replacements));
+    assertEquals("", StringUtils.replaceTokens("%bar%", pattern,
+      replacements));
+    assertEquals("", StringUtils.replaceTokens("", pattern, replacements));
+    assertEquals("zoo__zaz", StringUtils.replaceTokens("%foo%_%bar%_%baz%",
+      pattern, replacements));
+    assertEquals("begin zoo__zaz end", StringUtils.replaceTokens(
+      "begin %foo%_%bar%_%baz% end", pattern, replacements));
+  }
+
   // Benchmark for StringUtils split
   public static void main(String []args) {
     final String TO_SPLIT = "foo,bar,baz,blah,blah";

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto?rev=1453669&r1=1453668&r2=1453669&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto Thu Mar  7 02:57:40 2013
@@ -31,6 +31,7 @@ service TestProtobufRpcProto {
   rpc ping(EmptyRequestProto) returns (EmptyResponseProto);
   rpc echo(EchoRequestProto) returns (EchoResponseProto);
   rpc error(EmptyRequestProto) returns (EmptyResponseProto);
+  rpc error2(EmptyRequestProto) returns (EmptyResponseProto);
 }
 
 service TestProtobufRpc2Proto {

Modified: hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml?rev=1453669&r1=1453668&r2=1453669&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml Thu Mar  7 02:57:40 2013
@@ -487,6 +487,22 @@
     </test>
 
     <test> <!-- TESTED -->
+      <description>help: help for checksum</description>
+      <test-commands>
+        <command>-help checksum</command>
+      </test-commands>
+      <cleanup-commands>
+        <!-- No cleanup -->
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^-checksum &lt;src&gt; \.\.\.:( |\t)*Dump checksum information for files.*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test> <!-- TESTED -->
       <description>help: help for copyToLocal</description>
       <test-commands>
         <command>-help copyToLocal</command>