Browse Source

HADOOP-8562. Merge r1480883 for HADOOP-9553, r1480880 for HADOOP-9556, r1478633, r1478592 for HADOOP-9483, r1478577 for HADOOP-9043

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1486244 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 12 years ago
parent
commit
5f496310b6

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

@@ -88,6 +88,9 @@ Release 2.0.5-beta - UNRELEASED
     HADOOP-9150. Avoid unnecessary DNS resolution attempts for logical URIs
     (todd)
 
+    HADOOP-9540. Expose the InMemoryS3 and S3N FilesystemStores implementations
+    for Unit testing. (Hari via stevel)
+
   BUG FIXES
 
     HADOOP-9294. GetGroupsTestBase fails on Windows. (Chris Nauroth via suresh)
@@ -347,11 +350,23 @@ Release 2.0.5-beta - UNRELEASED
 
     HADOOP-9413. Add common utils for File#setReadable/Writable/Executable &
     File#canRead/Write/Execute that work cross-platform. (Ivan Mitic via suresh)
+
+    HADOOP-9043. Disallow in winutils creating symlinks with forwards slashes.
+    (Chris Nauroth via suresh)
+
+    HADOOP-9483. winutils support for readlink command.
+    (Arpit Agarwal via suresh)
     
     HADOOP-9488. FileUtil#createJarWithClassPath only substitutes environment
     variables from current process environment/does not support overriding
     when launching new process (Chris Nauroth via bikas)
 
+    HADOOP-9556. disable HA tests on Windows that fail due to ZooKeeper client
+    connection management bug. (Chris Nauroth via suresh)
+
+    HADOOP-9553. TestAuthenticationToken fails on Windows.
+    (Arpit Agarwal via suresh)
+
 Release 2.0.4-beta - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 7 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.fs.local;
 
 import java.io.IOException;
+import java.io.File;
 import java.io.FileNotFoundException;
 import java.net.URI;
 import java.net.URISyntaxException;
@@ -90,8 +91,8 @@ public class RawLocalFs extends DelegateToFileSystem {
     // NB: Use createSymbolicLink in java.nio.file.Path once available
     try {
       Shell.execCommand(Shell.getSymlinkCommand(
-        getPathWithoutSchemeAndAuthority(target),
-        getPathWithoutSchemeAndAuthority(link)));
+        getPathWithoutSchemeAndAuthority(target).getPath(),
+        getPathWithoutSchemeAndAuthority(link).getPath()));
     } catch (IOException x) {
       throw new IOException("Unable to create symlink: "+x.getMessage());
     }
@@ -167,12 +168,12 @@ public class RawLocalFs extends DelegateToFileSystem {
     throw new AssertionError();
   }
 
-  private static String getPathWithoutSchemeAndAuthority(Path path) {
-    // This code depends on Path.toString() to remove the leading slash before
-    // the drive specification on Windows.
+  private static File getPathWithoutSchemeAndAuthority(Path path) {
     Path newPath = path.isUriPathAbsolute() ?
       new Path(null, null, path.toUri().getPath()) :
       path;
-    return newPath.toString();
+
+    // Path.toString() removes leading slash before drive spec on Windows.
+    return new File(newPath.toString());
   }
 }

+ 3 - 0
hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h

@@ -104,6 +104,9 @@ void TaskUsage();
 int Symlink(__in int argc, __in_ecount(argc) wchar_t *argv[]);
 void SymlinkUsage();
 
+int Readlink(__in int argc, __in_ecount(argc) wchar_t *argv[]);
+void ReadlinkUsage();
+
 int SystemInfo();
 void SystemInfoUsage();
 

+ 8 - 0
hadoop-common-project/hadoop-common/src/main/winutils/main.c

@@ -55,6 +55,10 @@ int wmain(__in int argc, __in_ecount(argc) wchar_t* argv[])
   {
     return Symlink(argc - 1, argv + 1);
   }
+  else if (wcscmp(L"readlink", cmd) == 0)
+  {
+    return Readlink(argc - 1, argv + 1);
+  }
   else if (wcscmp(L"task", cmd) == 0)
   {
     return Task(argc - 1, argv + 1);
@@ -105,6 +109,10 @@ The available commands and their usages are:\n\n", program);
   SymlinkUsage();
   fwprintf(stdout, L"\n\n");
 
+  fwprintf(stdout, L"%-10s%s\n\n", L"readlink", L"Print the target of a symbolic link.");
+  ReadlinkUsage();
+  fwprintf(stdout, L"\n\n");
+
   fwprintf(stdout, L"%-15s%s\n\n", L"systeminfo", L"System information.");
   SystemInfoUsage();
   fwprintf(stdout, L"\n\n");

+ 224 - 0
hadoop-common-project/hadoop-common/src/main/winutils/readlink.c

@@ -0,0 +1,224 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+#include "winutils.h"
+
+
+//----------------------------------------------------------------------------
+// The Windows SDK does not include the definition of REPARSE_DATA_BUFFER. To
+// avoid adding a dependency on the WDK we define the structure here.
+// Reference: http://msdn.microsoft.com/en-us/library/ff552012.aspx
+//
+#pragma warning(push)
+#pragma warning(disable: 4201)  // nonstandard extension: nameless struct/union
+#pragma pack(push, 1)
+typedef struct _REPARSE_DATA_BUFFER {
+  ULONG  ReparseTag;
+  USHORT ReparseDataLength;
+  USHORT Reserved;
+  union {
+    struct {
+      USHORT SubstituteNameOffset;
+      USHORT SubstituteNameLength;
+      USHORT PrintNameOffset;
+      USHORT PrintNameLength;
+      ULONG  Flags;
+      WCHAR  PathBuffer[1];
+    } SymbolicLinkReparseBuffer;
+    struct {
+      USHORT SubstituteNameOffset;
+      USHORT SubstituteNameLength;
+      USHORT PrintNameOffset;
+      USHORT PrintNameLength;
+      WCHAR  PathBuffer[1];
+    } MountPointReparseBuffer;
+    struct {
+      UCHAR DataBuffer[1];
+    } GenericReparseBuffer;
+  };
+} REPARSE_DATA_BUFFER, *PREPARSE_DATA_BUFFER;
+#pragma pack(pop)
+#pragma warning(pop)
+
+
+//----------------------------------------------------------------------------
+// Function: Readlink
+//
+// Description:
+//  Prints the target of a symbolic link to stdout.
+//
+//  The return codes and output are modeled after the UNIX readlink command. 
+//  Hence no error messages are printed. Unlike the UNIX readlink, no options
+//  are accepted.
+//
+// Returns:
+//  0: on success
+//  1: on all errors
+//
+// Notes:
+//
+int Readlink(__in int argc, __in_ecount(argc) wchar_t *argv[])
+{
+  DWORD bytesReturned;
+  DWORD bufferSize = 1024;                  // Start off with a 1KB buffer.
+  HANDLE hFile = INVALID_HANDLE_VALUE;
+  PWSTR longLinkName = NULL;
+  PWCHAR printName = NULL;
+  PREPARSE_DATA_BUFFER pReparseData = NULL;
+  USHORT printNameLength;
+  USHORT printNameOffset;
+  DWORD result;
+  BOOLEAN succeeded = FALSE;
+  
+  if (argc != 2)
+  {
+    ReadlinkUsage();
+    goto Cleanup;
+  }
+
+  if (ConvertToLongPath(argv[1], &longLinkName) != ERROR_SUCCESS)
+  {
+    goto Cleanup;
+  }
+
+  // Get a handle to the link to issue the FSCTL.
+  // FILE_FLAG_BACKUP_SEMANTICS is needed to open directories.
+  // FILE_FLAG_OPEN_REPARSE_POINT disables normal reparse point processing
+  // so we can query the symlink.
+  //
+  hFile = CreateFileW(longLinkName,
+                      0,        // no rights needed to issue the FSCTL.
+                      FILE_SHARE_READ | FILE_SHARE_WRITE | FILE_SHARE_DELETE,
+                      NULL,
+                      OPEN_EXISTING,
+                      FILE_FLAG_BACKUP_SEMANTICS | FILE_FLAG_OPEN_REPARSE_POINT,
+                      NULL);
+
+  if (hFile == INVALID_HANDLE_VALUE) 
+  {
+    goto Cleanup;
+  }
+
+  for (;;)
+  {
+    pReparseData = (PREPARSE_DATA_BUFFER) LocalAlloc(LMEM_FIXED, bufferSize);
+
+    if (pReparseData == NULL) 
+    {
+      goto Cleanup;
+    }
+
+    // Issue the FSCTL to query the link information.
+    //
+    result = DeviceIoControl(hFile,
+                             FSCTL_GET_REPARSE_POINT,
+                             NULL,
+                             0,
+                             pReparseData,
+                             bufferSize,
+                             &bytesReturned,
+                             NULL);
+
+    if (result != 0)
+    {
+      // Success!
+      //
+      break;
+    }
+    else if ((GetLastError() == ERROR_INSUFFICIENT_BUFFER) ||
+             (GetLastError() == ERROR_MORE_DATA))
+    {
+      // Retry with a larger buffer.
+      //
+      LocalFree(pReparseData);
+      bufferSize *= 2;
+    }
+    else
+    {
+      // Unrecoverable error.
+      //
+      goto Cleanup;
+    }
+  }
+
+  if (pReparseData->ReparseTag != IO_REPARSE_TAG_SYMLINK) 
+  {
+    // Doesn't look like a symlink.
+    //
+    goto Cleanup;
+  }
+
+  // MSDN does not guarantee that the embedded paths in REPARSE_DATA_BUFFER
+  // will be NULL terminated. So we copy the string to a separate buffer and
+  // NULL terminate it before printing.
+  //
+  printNameLength = pReparseData->SymbolicLinkReparseBuffer.PrintNameLength;
+  printNameOffset = pReparseData->SymbolicLinkReparseBuffer.PrintNameOffset;
+  printName = (PWCHAR) LocalAlloc(LMEM_FIXED, printNameLength + 1);
+
+  if (printName == NULL) 
+  {
+    goto Cleanup;
+  }
+
+  memcpy(
+      printName,
+      pReparseData->SymbolicLinkReparseBuffer.PathBuffer + printNameOffset,
+      printNameLength);
+
+  printName[printNameLength / sizeof(WCHAR)] = L'\0';
+
+  fwprintf(stdout, L"%ls", printName);
+  succeeded = TRUE;
+
+Cleanup:
+  if (hFile != INVALID_HANDLE_VALUE) 
+  {
+    CloseHandle(hFile);
+  }
+
+  if (printName != NULL) 
+  {
+    LocalFree(printName);
+  }
+
+  if (pReparseData != NULL)
+  {
+    LocalFree(pReparseData);
+  }
+
+  if (longLinkName != NULL)
+  {
+    LocalFree(longLinkName);
+  }
+
+  return (succeeded ? EXIT_SUCCESS : EXIT_FAILURE);
+}
+
+void ReadlinkUsage()
+{
+    fwprintf(stdout, L"\
+Usage: readlink [LINKNAME]\n\
+Prints the target of a symbolic link\n\
+The output and returned error codes are similar to the UNIX\n\
+readlink command. However no options are accepted.\n\
+\n\
+0 is returned on success.\n\
+1 is returned for all errors.\n\
+\n");
+}
+

+ 11 - 0
hadoop-common-project/hadoop-common/src/main/winutils/symlink.c

@@ -60,6 +60,17 @@ int Symlink(__in int argc, __in_ecount(argc) wchar_t *argv[])
     goto SymlinkEnd;
   }
 
+  if (wcschr(longLinkName, L'/') != NULL || wcschr(longFileName, L'/') != NULL)
+  {
+    // Reject forward-slash separated paths as they result in unusable symlinks.
+    //
+    fwprintf(stderr,
+      L"Rejecting forward-slash separated path which would result in an "
+      L"unusable symlink: link = %s, target = %s\n", longLinkName, longFileName);
+    ret = FAILURE;
+    goto SymlinkEnd;
+  }
+
   // Check if the the process's access token has the privilege to create
   // symbolic links. Without this step, the call to CreateSymbolicLink() from
   // users have the privilege to create symbolic links will still succeed.

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/winutils/winutils.vcxproj

@@ -160,6 +160,7 @@
     </Link>
   </ItemDefinitionGroup>
   <ItemGroup>
+    <ClCompile Include="readlink.c" />
     <ClCompile Include="symlink.c" />
     <ClCompile Include="systeminfo.c" />
     <ClCompile Include="chmod.c" />
@@ -178,4 +179,4 @@
   <Import Project="$(VCTargetsPath)\Microsoft.Cpp.targets" />
   <ImportGroup Label="ExtensionTargets">
   </ImportGroup>
-</Project>
+</Project>

+ 40 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFSFileContextSymlink.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileUtil;
 import static org.apache.hadoop.fs.FileContextTestHelper.*;
 import static org.junit.Assert.*;
+import static org.junit.Assume.assumeTrue;
 import org.junit.Test;
 import org.junit.Before;
 
@@ -65,7 +66,44 @@ public class TestLocalFSFileContextSymlink extends FileContextSymlinkBaseTest {
     fc = FileContext.getLocalFSFileContext();
     super.setUp();
   }
-  
+
+  @Override
+  public void testCreateDanglingLink() throws IOException {
+    // Dangling symlinks are not supported on Windows local file system.
+    assumeTrue(!Path.WINDOWS);
+    super.testCreateDanglingLink();
+  }
+
+  @Override
+  public void testCreateFileViaDanglingLinkParent() throws IOException {
+    assumeTrue(!Path.WINDOWS);
+    super.testCreateFileViaDanglingLinkParent();
+  }
+
+  @Override
+  public void testOpenResolvesLinks() throws IOException {
+    assumeTrue(!Path.WINDOWS);
+    super.testOpenResolvesLinks();
+  }
+
+  @Override
+  public void testRecursiveLinks() throws IOException {
+    assumeTrue(!Path.WINDOWS);
+    super.testRecursiveLinks();
+  }
+
+  @Override
+  public void testRenameDirToDanglingSymlink() throws IOException {
+    assumeTrue(!Path.WINDOWS);
+    super.testRenameDirToDanglingSymlink();
+  }
+
+  @Override  
+  public void testStatDanglingLink() throws IOException {
+    assumeTrue(!Path.WINDOWS);
+    super.testStatDanglingLink();
+  }
+
   @Test
   /** lstat a non-existant file using a partially qualified path */
   public void testDanglingLinkFilePartQual() throws IOException {
@@ -87,6 +125,7 @@ public class TestLocalFSFileContextSymlink extends FileContextSymlinkBaseTest {
   @Test
   /** Stat and lstat a dangling link */
   public void testDanglingLink() throws IOException {
+    assumeTrue(!Path.WINDOWS);
     Path fileAbs  = new Path(testBaseDir1()+"/file");    
     Path fileQual = new Path(testURI().toString(), fileAbs);    
     Path link     = new Path(testBaseDir1()+"/linkToFile");

+ 4 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestActiveStandbyElectorRealZK.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.ha;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeTrue;
 
 import java.util.Collections;
 import java.util.UUID;
@@ -28,6 +29,7 @@ import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.ha.ActiveStandbyElector.ActiveStandbyElectorCallback;
 import org.apache.hadoop.ha.ActiveStandbyElector.State;
 import org.apache.hadoop.ha.HAZKUtil.ZKAuthInfo;
+import org.apache.hadoop.util.Shell;
 import org.apache.log4j.Level;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.server.ZooKeeperServer;
@@ -59,6 +61,8 @@ public class TestActiveStandbyElectorRealZK extends ClientBaseWithFixes {
   
   @Override
   public void setUp() throws Exception {
+    // skip tests on Windows until after resolution of ZooKeeper client bug
+    assumeTrue(!Shell.WINDOWS);
     super.setUp();
     
     zkServer = getServer(serverFactory);

+ 4 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.ha;
 
 import static org.junit.Assert.*;
+import static org.junit.Assume.assumeTrue;
 
 import java.security.NoSuchAlgorithmException;
 
@@ -28,6 +29,7 @@ import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
 import org.apache.hadoop.ha.HealthMonitor.State;
 import org.apache.hadoop.ha.MiniZKFCCluster.DummyZKFC;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
 import org.apache.zookeeper.KeeperException;
@@ -66,6 +68,8 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
   
   @Before
   public void setupConfAndServices() {
+    // skip tests on Windows until after resolution of ZooKeeper client bug
+    assumeTrue(!Shell.WINDOWS);
     conf = new Configuration();
     conf.set(ZKFailoverController.ZK_ACL_KEY, TEST_ACL);
     conf.set(ZKFailoverController.ZK_AUTH_KEY, TEST_AUTH_GOOD);

+ 8 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverControllerStress.java

@@ -17,10 +17,13 @@
  */
 package org.apache.hadoop.ha;
 
+import static org.junit.Assume.assumeTrue;
+
 import java.util.Random;
 
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Time;
 import org.junit.After;
 import org.junit.Before;
@@ -45,6 +48,8 @@ public class TestZKFailoverControllerStress extends ClientBaseWithFixes {
 
   @Before
   public void setupConfAndServices() throws Exception {
+    // skip tests on Windows until after resolution of ZooKeeper client bug
+    assumeTrue(!Shell.WINDOWS);
     conf = new Configuration();
     conf.set(ZKFailoverController.ZK_QUORUM_KEY, hostPort);
     this.cluster = new MiniZKFCCluster(conf, getServer(serverFactory));
@@ -52,7 +57,9 @@ public class TestZKFailoverControllerStress extends ClientBaseWithFixes {
   
   @After
   public void stopCluster() throws Exception {
-    cluster.stop();
+    if (cluster != null) {
+      cluster.stop();
+    }
   }
 
   /**

+ 118 - 25
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestWinUtils.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.util;
 
 import static org.junit.Assert.*;
+import static org.junit.Assume.assumeTrue;
 
 import java.io.File;
 import java.io.FileInputStream;
@@ -32,6 +33,8 @@ import org.apache.hadoop.fs.FileUtil;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import static org.junit.Assume.*;
+import static org.hamcrest.CoreMatchers.*;
 
 /**
  * Test cases for helper Windows winutils.exe utility.
@@ -44,6 +47,8 @@ public class TestWinUtils {
 
   @Before
   public void setUp() {
+    // Not supported on non-Windows platforms
+    assumeTrue(Shell.WINDOWS);
     TEST_DIR.mkdirs();
   }
 
@@ -70,11 +75,6 @@ public class TestWinUtils {
 
   @Test (timeout = 30000)
   public void testLs() throws IOException {
-    if (!Shell.WINDOWS) {
-      // Not supported on non-Windows platforms
-      return;
-    }
-
     final String content = "6bytes";
     final int contentSize = content.length();
     File testFile = new File(TEST_DIR, "file1");
@@ -104,11 +104,6 @@ public class TestWinUtils {
 
   @Test (timeout = 30000)
   public void testGroups() throws IOException {
-    if (!Shell.WINDOWS) {
-      // Not supported on non-Windows platforms
-      return;
-    }
-
     String currentUser = System.getProperty("user.name");
 
     // Verify that groups command returns information about the current user
@@ -229,11 +224,6 @@ public class TestWinUtils {
 
   @Test (timeout = 30000)
   public void testBasicChmod() throws IOException {
-    if (!Shell.WINDOWS) {
-      // Not supported on non-Windows platforms
-      return;
-    }
-
     // - Create a file.
     // - Change mode to 377 so owner does not have read permission.
     // - Verify the owner truly does not have the permissions to read.
@@ -285,11 +275,6 @@ public class TestWinUtils {
 
   @Test (timeout = 30000)
   public void testChmod() throws IOException {
-    if (!Shell.WINDOWS) {
-      // Not supported on non-Windows platforms
-      return;
-    }
-
     testChmodInternal("7", "-------rwx");
     testChmodInternal("70", "----rwx---");
     testChmodInternal("u-x,g+r,o=g", "-rw-r--r--");
@@ -322,11 +307,6 @@ public class TestWinUtils {
 
   @Test (timeout = 30000)
   public void testChown() throws IOException {
-    if (!Shell.WINDOWS) {
-      // Not supported on non-Windows platforms
-      return;
-    }
-
     File a = new File(TEST_DIR, "a");
     assertTrue(a.createNewFile());
     String username = System.getProperty("user.name");
@@ -349,4 +329,117 @@ public class TestWinUtils {
     assertTrue(a.delete());
     assertFalse(a.exists());
   }
+
+  @Test (timeout = 30000)
+  public void testSymlinkRejectsForwardSlashesInLink() throws IOException {
+    File newFile = new File(TEST_DIR, "file");
+    assertTrue(newFile.createNewFile());
+    String target = newFile.getPath();
+    String link = new File(TEST_DIR, "link").getPath().replaceAll("\\\\", "/");
+    try {
+      Shell.execCommand(Shell.WINUTILS, "symlink", link, target);
+      fail(String.format("did not receive expected failure creating symlink "
+        + "with forward slashes in link: link = %s, target = %s", link, target));
+    } catch (IOException e) {
+      LOG.info(
+        "Expected: Failed to create symlink with forward slashes in target");
+    }
+  }
+
+  @Test (timeout = 30000)
+  public void testSymlinkRejectsForwardSlashesInTarget() throws IOException {
+    File newFile = new File(TEST_DIR, "file");
+    assertTrue(newFile.createNewFile());
+    String target = newFile.getPath().replaceAll("\\\\", "/");
+    String link = new File(TEST_DIR, "link").getPath();
+    try {
+      Shell.execCommand(Shell.WINUTILS, "symlink", link, target);
+      fail(String.format("did not receive expected failure creating symlink "
+        + "with forward slashes in target: link = %s, target = %s", link, target));
+    } catch (IOException e) {
+      LOG.info(
+        "Expected: Failed to create symlink with forward slashes in target");
+    }
+  }
+
+  @Test (timeout = 30000)
+  public void testReadLink() throws IOException {
+    // Create TEST_DIR\dir1\file1.txt
+    //
+    File dir1 = new File(TEST_DIR, "dir1");
+    assertTrue(dir1.mkdirs());
+
+    File file1 = new File(dir1, "file1.txt");
+    assertTrue(file1.createNewFile());
+
+    File dirLink = new File(TEST_DIR, "dlink");
+    File fileLink = new File(TEST_DIR, "flink");
+
+    // Next create a directory symlink to dir1 and a file
+    // symlink to file1.txt.
+    //
+    Shell.execCommand(
+        Shell.WINUTILS, "symlink", dirLink.toString(), dir1.toString());
+    Shell.execCommand(
+        Shell.WINUTILS, "symlink", fileLink.toString(), file1.toString());
+
+    // Read back the two links and ensure we get what we expected.
+    //
+    String readLinkOutput = Shell.execCommand(Shell.WINUTILS,
+        "readlink",
+        dirLink.toString());
+    assertThat(readLinkOutput, equalTo(dir1.toString()));
+
+    readLinkOutput = Shell.execCommand(Shell.WINUTILS,
+        "readlink",
+        fileLink.toString());
+    assertThat(readLinkOutput, equalTo(file1.toString()));
+
+    // Try a few invalid inputs and verify we get an ExitCodeException for each.
+    //
+    try {
+      // No link name specified.
+      //
+      Shell.execCommand(Shell.WINUTILS, "readlink", "");
+      fail("Failed to get Shell.ExitCodeException when reading bad symlink");
+    } catch (Shell.ExitCodeException ece) {
+      assertThat(ece.getExitCode(), is(1));
+    }
+
+    try {
+      // Bad link name.
+      //
+      Shell.execCommand(Shell.WINUTILS, "readlink", "ThereIsNoSuchLink");
+      fail("Failed to get Shell.ExitCodeException when reading bad symlink");
+    } catch (Shell.ExitCodeException ece) {
+      assertThat(ece.getExitCode(), is(1));
+    }
+
+    try {
+      // Non-symlink directory target.
+      //
+      Shell.execCommand(Shell.WINUTILS, "readlink", dir1.toString());
+      fail("Failed to get Shell.ExitCodeException when reading bad symlink");
+    } catch (Shell.ExitCodeException ece) {
+      assertThat(ece.getExitCode(), is(1));
+    }
+
+    try {
+      // Non-symlink file target.
+      //
+      Shell.execCommand(Shell.WINUTILS, "readlink", file1.toString());
+      fail("Failed to get Shell.ExitCodeException when reading bad symlink");
+    } catch (Shell.ExitCodeException ece) {
+      assertThat(ece.getExitCode(), is(1));
+    }
+
+    try {
+      // Too many parameters.
+      //
+      Shell.execCommand(Shell.WINUTILS, "readlink", "a", "b");
+      fail("Failed to get Shell.ExitCodeException with bad parameters");
+    } catch (Shell.ExitCodeException ece) {
+      assertThat(ece.getExitCode(), is(1));
+    }
+  }
 }