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 jg...@apache.org on 2011/03/11 00:33:52 UTC

svn commit: r1080396 - in /hadoop/common/trunk: CHANGES.txt src/java/org/apache/hadoop/fs/FileUtil.java src/java/org/apache/hadoop/fs/HardLink.java src/test/core/org/apache/hadoop/fs/TestHardLink.java

Author: jghoman
Date: Thu Mar 10 23:33:52 2011
New Revision: 1080396

URL: http://svn.apache.org/viewvc?rev=1080396&view=rev
Log:
HADOOP-7133. Batch the calls in DataStorage to FileUtil.createHardLink().  Contributed by Matt Foley.

Added:
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/HardLink.java
    hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/TestHardLink.java
Modified:
    hadoop/common/trunk/CHANGES.txt
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/FileUtil.java

Modified: hadoop/common/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/CHANGES.txt?rev=1080396&r1=1080395&r2=1080396&view=diff
==============================================================================
--- hadoop/common/trunk/CHANGES.txt (original)
+++ hadoop/common/trunk/CHANGES.txt Thu Mar 10 23:33:52 2011
@@ -71,6 +71,9 @@ Trunk (unreleased changes)
 
     HADOOP-7167. Allow using a file to exclude certain tests from build. (todd)
 
+    HADOOP-7133. Batch the calls in DataStorage to FileUtil.createHardLink().
+    (Matt Foley via jghoman)
+
   OPTIMIZATIONS
 
   BUG FIXES

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/FileUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/FileUtil.java?rev=1080396&r1=1080395&r2=1080396&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/FileUtil.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/FileUtil.java Thu Mar 10 23:33:52 2011
@@ -585,146 +585,13 @@ public class FileUtil {
   /**
    * Class for creating hardlinks.
    * Supports Unix, Cygwin, WindXP.
-   *  
+   * @deprecated Use {@link org.apache.hadoop.fs.HardLink}
    */
-  public static class HardLink { 
-    enum OSType {
-      OS_TYPE_UNIX, 
-      OS_TYPE_WINXP,
-      OS_TYPE_SOLARIS,
-      OS_TYPE_MAC; 
-    }
-  
-    private static String[] hardLinkCommand;
-    private static String[] getLinkCountCommand;
-    private static OSType osType;
-    
-    static {
-      osType = getOSType();
-      switch(osType) {
-      case OS_TYPE_WINXP:
-        hardLinkCommand = new String[] {"fsutil","hardlink","create", null, null};
-        getLinkCountCommand = new String[] {"stat","-c%h"};
-        break;
-      case OS_TYPE_SOLARIS:
-        hardLinkCommand = new String[] {"ln", null, null};
-        getLinkCountCommand = new String[] {"ls","-l"};
-        break;
-      case OS_TYPE_MAC:
-        hardLinkCommand = new String[] {"ln", null, null};
-        getLinkCountCommand = new String[] {"stat","-f%l"};
-        break;
-      case OS_TYPE_UNIX:
-      default:
-        hardLinkCommand = new String[] {"ln", null, null};
-        getLinkCountCommand = new String[] {"stat","-c%h"};
-      }
-    }
-
-    static private OSType getOSType() {
-      String osName = System.getProperty("os.name");
-      if (osName.indexOf("Windows") >= 0 && 
-          (osName.indexOf("XP") >= 0 || osName.indexOf("2003") >= 0 || osName.indexOf("Vista") >= 0))
-        return OSType.OS_TYPE_WINXP;
-      else if (osName.indexOf("SunOS") >= 0)
-         return OSType.OS_TYPE_SOLARIS;
-      else if (osName.indexOf("Mac") >= 0)
-         return OSType.OS_TYPE_MAC;
-      else
-        return OSType.OS_TYPE_UNIX;
-    }
-    
-    /**
-     * Creates a hardlink 
-     */
-    public static void createHardLink(File target, 
-                                      File linkName) throws IOException {
-      int len = hardLinkCommand.length;
-      if (osType == OSType.OS_TYPE_WINXP) {
-       hardLinkCommand[len-1] = target.getCanonicalPath();
-       hardLinkCommand[len-2] = linkName.getCanonicalPath();
-      } else {
-       hardLinkCommand[len-2] = makeShellPath(target, true);
-       hardLinkCommand[len-1] = makeShellPath(linkName, true);
-      }
-      // execute shell command
-      Process process = Runtime.getRuntime().exec(hardLinkCommand);
-      try {
-        if (process.waitFor() != 0) {
-          String errMsg = new BufferedReader(new InputStreamReader(
-                                                                   process.getInputStream())).readLine();
-          if (errMsg == null)  errMsg = "";
-          String inpMsg = new BufferedReader(new InputStreamReader(
-                                                                   process.getErrorStream())).readLine();
-          if (inpMsg == null)  inpMsg = "";
-          throw new IOException(errMsg + inpMsg);
-        }
-      } catch (InterruptedException e) {
-        throw new IOException(StringUtils.stringifyException(e));
-      } finally {
-        process.destroy();
-      }
-    }
-
-    /**
-     * Retrieves the number of links to the specified file.
-     */
-    public static int getLinkCount(File fileName) throws IOException {
-      if (!fileName.exists()) {
-        throw new FileNotFoundException(fileName + " not found.");
-      }
-
-      int len = getLinkCountCommand.length;
-      String[] cmd = new String[len + 1];
-      for (int i = 0; i < len; i++) {
-        cmd[i] = getLinkCountCommand[i];
-      }
-      cmd[len] = fileName.toString();
-      String inpMsg = null;
-      String errMsg = null;
-      int exitValue = -1;
-      BufferedReader in = null;
-      BufferedReader err = null;
-
-      // execute shell command
-      Process process = Runtime.getRuntime().exec(cmd);
-      try {
-        exitValue = process.waitFor();
-        in = new BufferedReader(new InputStreamReader(
-                                    process.getInputStream()));
-        inpMsg = in.readLine();
-        err = new BufferedReader(new InputStreamReader(
-                                     process.getErrorStream()));
-        errMsg = err.readLine();
-        if (inpMsg == null || exitValue != 0) {
-          throw createIOException(fileName, inpMsg, errMsg, exitValue, null);
-        }
-        if (getOSType() == OSType.OS_TYPE_SOLARIS) {
-          String[] result = inpMsg.split("\\s+");
-          return Integer.parseInt(result[1]);
-        } else {
-          return Integer.parseInt(inpMsg);
-        }
-      } catch (NumberFormatException e) {
-        throw createIOException(fileName, inpMsg, errMsg, exitValue, e);
-      } catch (InterruptedException e) {
-        throw createIOException(fileName, inpMsg, errMsg, exitValue, e);
-      } finally {
-        process.destroy();
-        if (in != null) in.close();
-        if (err != null) err.close();
-      }
-    }
-  }
-
-  /** Create an IOException for failing to get link count. */
-  static private IOException createIOException(File f, String message,
-      String error, int exitvalue, Exception cause) {
-    final String s = "Failed to get link count on file " + f
-        + ": message=" + message
-        + "; error=" + error
-        + "; exit value=" + exitvalue;
-    return cause == null? new IOException(s): new IOException(s, cause);
+  @Deprecated
+  public static class HardLink extends org.apache.hadoop.fs.HardLink { 
+    // This is a stub to assist with coordinated change between
+    // COMMON and HDFS projects.  It will be removed after the
+    // corresponding change is committed to HDFS.
   }
 
   /**

Added: hadoop/common/trunk/src/java/org/apache/hadoop/fs/HardLink.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/HardLink.java?rev=1080396&view=auto
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/HardLink.java (added)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/HardLink.java Thu Mar 10 23:33:52 2011
@@ -0,0 +1,631 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.Arrays;
+
+/**
+ * Class for creating hardlinks.
+ * Supports Unix/Linux, WinXP/2003/Vista via Cygwin, and Mac OS X.
+ * 
+ * The HardLink class was formerly a static inner class of FSUtil,
+ * and the methods provided were blatantly non-thread-safe.
+ * To enable volume-parallel Update snapshots, we now provide static 
+ * threadsafe methods that allocate new buffer string arrays
+ * upon each call.  We also provide an API to hardlink all files in a
+ * directory with a single command, which is up to 128 times more 
+ * efficient - and minimizes the impact of the extra buffer creations.
+ */
+public class HardLink { 
+
+  public enum OSType {
+    OS_TYPE_UNIX,
+    OS_TYPE_WINXP,
+    OS_TYPE_SOLARIS,
+    OS_TYPE_MAC
+  }
+  
+  public static OSType osType;
+  private static HardLinkCommandGetter getHardLinkCommand;
+  
+  public final LinkStats linkStats; //not static
+  
+  //initialize the command "getters" statically, so can use their 
+  //methods without instantiating the HardLink object
+  static { 
+    osType = getOSType();
+    if (osType == OSType.OS_TYPE_WINXP) {
+      // Windows
+      getHardLinkCommand = new HardLinkCGWin();
+    } else {
+      // Unix
+      getHardLinkCommand = new HardLinkCGUnix();
+      //override getLinkCountCommand for the particular Unix variant
+      //Linux is already set as the default - {"stat","-c%h", null}
+      if (osType == OSType.OS_TYPE_MAC) {
+        String[] linkCountCmdTemplate = {"stat","-f%l", null};
+        HardLinkCGUnix.setLinkCountCmdTemplate(linkCountCmdTemplate);
+      } else if (osType == OSType.OS_TYPE_SOLARIS) {
+        String[] linkCountCmdTemplate = {"ls","-l", null};
+        HardLinkCGUnix.setLinkCountCmdTemplate(linkCountCmdTemplate);        
+      }
+    }
+  }
+
+  public HardLink() {
+    linkStats = new LinkStats();
+  }
+  
+  static private OSType getOSType() {
+    String osName = System.getProperty("os.name");
+    if (osName.contains("Windows") &&
+            (osName.contains("XP") 
+            || osName.contains("2003") 
+            || osName.contains("Vista")
+            || osName.contains("Windows_7")
+            || osName.contains("Windows 7") 
+            || osName.contains("Windows7"))) {
+      return OSType.OS_TYPE_WINXP;
+    }
+    else if (osName.contains("SunOS") 
+            || osName.contains("Solaris")) {
+       return OSType.OS_TYPE_SOLARIS;
+    }
+    else if (osName.contains("Mac")) {
+       return OSType.OS_TYPE_MAC;
+    }
+    else {
+      return OSType.OS_TYPE_UNIX;
+    }
+  }
+  
+  /**
+   * This abstract class bridges the OS-dependent implementations of the 
+   * needed functionality for creating hardlinks and querying link counts.
+   * The particular implementation class is chosen during 
+   * static initialization phase of the HardLink class.
+   * The "getter" methods construct shell command strings for various purposes.
+   */
+  private static abstract class HardLinkCommandGetter {
+
+    /**
+     * Get the command string needed to hardlink a bunch of files from
+     * a single source directory into a target directory.  The source directory
+     * is not specified here, but the command will be executed using the source
+     * directory as the "current working directory" of the shell invocation.
+     * 
+     * @param fileBaseNames - array of path-less file names, relative
+     *            to the source directory
+     * @param linkDir - target directory where the hardlinks will be put
+     * @return - an array of Strings suitable for use as a single shell command
+     *            with {@link Runtime.exec()}
+     * @throws IOException - if any of the file or path names misbehave
+     */
+    abstract String[] linkMult(String[] fileBaseNames, File linkDir) 
+                          throws IOException;
+    
+    /**
+     * Get the command string needed to hardlink a single file
+     */
+    abstract String[] linkOne(File file, File linkName) throws IOException;
+    
+    /**
+     * Get the command string to query the hardlink count of a file
+     */
+    abstract String[] linkCount(File file) throws IOException;
+    
+    /**
+     * Calculate the total string length of the shell command
+     * resulting from execution of linkMult, plus the length of the
+     * source directory name (which will also be provided to the shell)
+     * 
+     * @param fileDir - source directory, parent of fileBaseNames
+     * @param fileBaseNames - array of path-less file names, relative
+     *            to the source directory
+     * @param linkDir - target directory where the hardlinks will be put
+     * @return - total data length (must not exceed maxAllowedCmdArgLength)
+     * @throws IOException
+     */
+    abstract int getLinkMultArgLength(
+                     File fileDir, String[] fileBaseNames, File linkDir) 
+                     throws IOException;
+    
+    /**
+     * Get the maximum allowed string length of a shell command on this OS,
+     * which is just the documented minimum guaranteed supported command
+     * length - aprx. 32KB for Unix, and 8KB for Windows.
+     */
+    abstract int getMaxAllowedCmdArgLength(); 
+  }
+  
+  /**
+   * Implementation of HardLinkCommandGetter class for Unix
+   */
+  static class HardLinkCGUnix extends HardLinkCommandGetter {
+    private static String[] hardLinkCommand = {"ln", null, null};
+    private static String[] hardLinkMultPrefix = {"ln"};
+    private static String[] hardLinkMultSuffix = {null};
+    private static String[] getLinkCountCommand = {"stat","-c%h", null};
+    //Unix guarantees at least 32K bytes cmd length.
+    //Subtract another 64b to allow for Java 'exec' overhead
+    private static final int maxAllowedCmdArgLength = 32*1024 - 65;
+    
+    private static synchronized 
+    void setLinkCountCmdTemplate(String[] template) {
+      //May update this for specific unix variants, 
+      //after static initialization phase
+      getLinkCountCommand = template;
+    }
+    
+    /*
+     * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#linkOne(java.io.File, java.io.File)
+     */
+    @Override
+    String[] linkOne(File file, File linkName) 
+    throws IOException {
+      String[] buf = new String[hardLinkCommand.length];
+      System.arraycopy(hardLinkCommand, 0, buf, 0, hardLinkCommand.length);
+      //unix wants argument order: "ln <existing> <new>"
+      buf[1] = FileUtil.makeShellPath(file, true); 
+      buf[2] = FileUtil.makeShellPath(linkName, true);
+      return buf;
+    }
+    
+    /*
+     * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#linkMult(java.lang.String[], java.io.File)
+     */
+    @Override
+    String[] linkMult(String[] fileBaseNames, File linkDir) 
+    throws IOException {
+      String[] buf = new String[fileBaseNames.length 
+                                + hardLinkMultPrefix.length 
+                                + hardLinkMultSuffix.length];
+      int mark=0;
+      System.arraycopy(hardLinkMultPrefix, 0, buf, mark, 
+                       hardLinkMultPrefix.length);
+      mark += hardLinkMultPrefix.length;
+      System.arraycopy(fileBaseNames, 0, buf, mark, fileBaseNames.length);
+      mark += fileBaseNames.length;
+      buf[mark] = FileUtil.makeShellPath(linkDir, true);
+      return buf;
+    }
+    
+    /*
+     * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#linkCount(java.io.File)
+     */
+    @Override
+    String[] linkCount(File file) 
+    throws IOException {
+      String[] buf = new String[getLinkCountCommand.length];
+      System.arraycopy(getLinkCountCommand, 0, buf, 0, 
+                       getLinkCountCommand.length);
+      buf[getLinkCountCommand.length - 1] = FileUtil.makeShellPath(file, true);
+      return buf;
+    }
+    
+    /*
+     * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#getLinkMultArgLength(java.io.File, java.lang.String[], java.io.File)
+     */
+    @Override
+    int getLinkMultArgLength(File fileDir, String[] fileBaseNames, File linkDir) 
+    throws IOException{
+      int sum = 0;
+      for (String x : fileBaseNames) {
+        // add 1 to account for terminal null or delimiter space
+        sum += 1 + ((x == null) ? 0 : x.length());
+      }
+      sum += 2 + FileUtil.makeShellPath(fileDir, true).length()
+             + FileUtil.makeShellPath(linkDir, true).length();
+      //add the fixed overhead of the hardLinkMult prefix and suffix
+      sum += 3; //length("ln") + 1
+      return sum;
+    }
+    
+    /*
+     * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#getMaxAllowedCmdArgLength()
+     */
+    @Override
+    int getMaxAllowedCmdArgLength() {
+      return maxAllowedCmdArgLength;
+    }
+  }
+  
+  
+  /**
+   * Implementation of HardLinkCommandGetter class for Windows
+   * 
+   * Note that the linkCount shell command for Windows is actually
+   * a Cygwin shell command, and depends on ${cygwin}/bin
+   * being in the Windows PATH environment variable, so
+   * stat.exe can be found.
+   */
+  static class HardLinkCGWin extends HardLinkCommandGetter {
+    //The Windows command getter impl class and its member fields are
+    //package-private ("default") access instead of "private" to assist 
+    //unit testing (sort of) on non-Win servers
+
+    static String[] hardLinkCommand = {
+                        "fsutil","hardlink","create", null, null};
+    static String[] hardLinkMultPrefix = {
+                        "cmd","/q","/c","for", "%f", "in", "("};
+    static String   hardLinkMultDir = "\\%f";
+    static String[] hardLinkMultSuffix = {
+                        ")", "do", "fsutil", "hardlink", "create", null, 
+                        "%f", "1>NUL"};
+    static String[] getLinkCountCommand = {"stat","-c%h", null};
+    //Windows guarantees only 8K - 1 bytes cmd length.
+    //Subtract another 64b to allow for Java 'exec' overhead
+    static final int maxAllowedCmdArgLength = 8*1024 - 65;
+
+    /*
+     * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#linkOne(java.io.File, java.io.File)
+     */
+    @Override
+    String[] linkOne(File file, File linkName) 
+    throws IOException {
+      String[] buf = new String[hardLinkCommand.length];
+      System.arraycopy(hardLinkCommand, 0, buf, 0, hardLinkCommand.length);
+      //windows wants argument order: "create <new> <existing>"
+      buf[4] = file.getCanonicalPath(); 
+      buf[3] = linkName.getCanonicalPath();
+      return buf;
+    }
+    
+    /*
+     * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#linkMult(java.lang.String[], java.io.File)
+     */
+    @Override
+    String[] linkMult(String[] fileBaseNames, File linkDir) 
+    throws IOException {
+      String[] buf = new String[fileBaseNames.length 
+                                + hardLinkMultPrefix.length 
+                                + hardLinkMultSuffix.length];
+      String td = linkDir.getCanonicalPath() + hardLinkMultDir;
+      int mark=0;
+      System.arraycopy(hardLinkMultPrefix, 0, buf, mark, 
+                       hardLinkMultPrefix.length);
+      mark += hardLinkMultPrefix.length;
+      System.arraycopy(fileBaseNames, 0, buf, mark, fileBaseNames.length);
+      mark += fileBaseNames.length;
+      System.arraycopy(hardLinkMultSuffix, 0, buf, mark, 
+                       hardLinkMultSuffix.length);
+      mark += hardLinkMultSuffix.length;
+      buf[mark - 3] = td;
+      return buf;
+    }
+    
+    /*
+     * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#linkCount(java.io.File)
+     */
+    @Override
+    String[] linkCount(File file) 
+    throws IOException {
+      String[] buf = new String[getLinkCountCommand.length];
+      System.arraycopy(getLinkCountCommand, 0, buf, 0, 
+                       getLinkCountCommand.length);
+      //The linkCount command is actually a Cygwin shell command,
+      //not a Windows shell command, so we should use "makeShellPath()"
+      //instead of "getCanonicalPath()".  However, that causes another
+      //shell exec to "cygpath.exe", and "stat.exe" actually can handle
+      //DOS-style paths (it just prints a couple hundred bytes of warning
+      //to stderr), so we use the more efficient "getCanonicalPath()".
+      buf[getLinkCountCommand.length - 1] = file.getCanonicalPath();
+      return buf;
+    }
+    
+    /*
+     * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#getLinkMultArgLength(java.io.File, java.lang.String[], java.io.File)
+     */
+    @Override
+    int getLinkMultArgLength(File fileDir, String[] fileBaseNames, File linkDir) 
+    throws IOException {
+      int sum = 0;
+      for (String x : fileBaseNames) {
+        // add 1 to account for terminal null or delimiter space
+        sum += 1 + ((x == null) ? 0 : x.length());
+      }
+      sum += 2 + fileDir.getCanonicalPath().length() +
+               linkDir.getCanonicalPath().length();
+      //add the fixed overhead of the hardLinkMult command 
+      //(prefix, suffix, and Dir suffix)
+      sum += ("cmd.exe /q /c for %f in ( ) do "
+              + "fsutil hardlink create \\%f %f 1>NUL ").length();
+      return sum;
+    }
+    
+    /*
+     * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#getMaxAllowedCmdArgLength()
+     */
+    @Override
+    int getMaxAllowedCmdArgLength() {
+      return maxAllowedCmdArgLength;
+    }
+  }
+  
+  
+  /**
+   * Calculate the nominal length of all contributors to the total 
+   * commandstring length, including fixed overhead of the OS-dependent 
+   * command.  It's protected rather than private, to assist unit testing,
+   * but real clients are not expected to need it -- see the way 
+   * createHardLinkMult() uses it internally so the user doesn't need to worry
+   * about it.
+   * 
+   * @param fileDir - source directory, parent of fileBaseNames
+   * @param fileBaseNames - array of path-less file names, relative
+   *            to the source directory
+   * @param linkDir - target directory where the hardlinks will be put
+   * @return - total data length (must not exceed maxAllowedCmdArgLength)
+   * @throws IOException
+   */
+  protected static int getLinkMultArgLength(
+          File fileDir, String[] fileBaseNames, File linkDir) 
+  throws IOException {
+    return getHardLinkCommand.getLinkMultArgLength(fileDir, 
+          fileBaseNames, linkDir);
+  }
+  
+  /**
+   * Return this private value for use by unit tests.
+   * Shell commands are not allowed to have a total string length
+   * exceeding this size.
+   */
+  protected static int getMaxAllowedCmdArgLength() {
+    return getHardLinkCommand.getMaxAllowedCmdArgLength();
+  }
+  
+  /*
+   * ****************************************************
+   * Complexity is above.  User-visible functionality is below
+   * ****************************************************
+   */
+
+  /**
+   * Creates a hardlink 
+   * @param file - existing source file
+   * @param linkName - desired target link file
+   */
+  public static void createHardLink(File file, File linkName) 
+  throws IOException {
+    if (file == null) {
+      throw new IOException(
+          "invalid arguments to createHardLink: source file is null");
+    }
+    if (linkName == null) {
+      throw new IOException(
+          "invalid arguments to createHardLink: link name is null");
+    }
+	  // construct and execute shell command
+    String[] hardLinkCommand = getHardLinkCommand.linkOne(file, linkName);
+    Process process = Runtime.getRuntime().exec(hardLinkCommand);
+    try {
+      if (process.waitFor() != 0) {
+        String errMsg = new BufferedReader(new InputStreamReader(
+            process.getInputStream())).readLine();
+        if (errMsg == null)  errMsg = "";
+        String inpMsg = new BufferedReader(new InputStreamReader(
+            process.getErrorStream())).readLine();
+        if (inpMsg == null)  inpMsg = "";
+        throw new IOException(errMsg + inpMsg);
+      }
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    } finally {
+      process.destroy();
+    }
+  }
+
+  /**
+   * Creates hardlinks from multiple existing files within one parent
+   * directory, into one target directory.
+   * @param parentDir - directory containing source files
+   * @param fileBaseNames - list of path-less file names, as returned by 
+   *                        parentDir.list()
+   * @param linkDir - where the hardlinks should be put.  It must already exist.
+   * 
+   * If the list of files is too long (overflows maxAllowedCmdArgLength),
+   * we will automatically split it into multiple invocations of the
+   * underlying method.
+   */
+  public static void createHardLinkMult(File parentDir, String[] fileBaseNames, 
+      File linkDir) throws IOException {
+    //This is the public method all non-test clients are expected to use.
+    //Normal case - allow up to maxAllowedCmdArgLength characters in the cmd
+    createHardLinkMult(parentDir, fileBaseNames, linkDir, 
+                       getHardLinkCommand.getMaxAllowedCmdArgLength());
+  }
+
+  /*
+   * Implements {@link createHardLinkMult} with added variable  "maxLength",
+   * to ease unit testing of the auto-splitting feature for long lists.
+   * Likewise why it returns "callCount", the number of sub-arrays that
+   * the file list had to be split into.
+   * Non-test clients are expected to call the public method instead.
+   */
+  protected static int createHardLinkMult(File parentDir, 
+      String[] fileBaseNames, File linkDir, int maxLength) 
+  throws IOException {
+    if (parentDir == null) {
+      throw new IOException(
+          "invalid arguments to createHardLinkMult: parent directory is null");
+    }
+    if (linkDir == null) {
+      throw new IOException(
+          "invalid arguments to createHardLinkMult: link directory is null");
+    }
+    if (fileBaseNames == null) {
+      throw new IOException(
+          "invalid arguments to createHardLinkMult: "
+          + "filename list can be empty but not null");
+    }
+    if (fileBaseNames.length == 0) {
+      //the OS cmds can't handle empty list of filenames, 
+      //but it's legal, so just return.
+      return 0; 
+    }
+    if (!linkDir.exists()) {
+      throw new FileNotFoundException(linkDir + " not found.");
+    }
+
+    //if the list is too long, split into multiple invocations
+    int callCount = 0;
+    if (getLinkMultArgLength(parentDir, fileBaseNames, linkDir) > maxLength
+          && fileBaseNames.length > 1) {
+      String[] list1 = Arrays.copyOf(fileBaseNames, fileBaseNames.length/2);
+      callCount += createHardLinkMult(parentDir, list1, linkDir, maxLength);
+      String[] list2 = Arrays.copyOfRange(fileBaseNames, fileBaseNames.length/2,
+          fileBaseNames.length);
+      callCount += createHardLinkMult(parentDir, list2, linkDir, maxLength);  
+      return callCount;
+    } else {
+      callCount = 1;
+    }
+    
+    // construct and execute shell command
+    String[] hardLinkCommand = getHardLinkCommand.linkMult(fileBaseNames, 
+        linkDir);
+    Process process = Runtime.getRuntime().exec(hardLinkCommand, null, 
+        parentDir);
+    try {
+      if (process.waitFor() != 0) {
+        String errMsg = new BufferedReader(new InputStreamReader(
+            process.getInputStream())).readLine();
+        if (errMsg == null)  errMsg = "";
+        String inpMsg = new BufferedReader(new InputStreamReader(
+            process.getErrorStream())).readLine();
+        if (inpMsg == null)  inpMsg = "";
+        throw new IOException(errMsg + inpMsg);
+      }
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    } finally {
+      process.destroy();
+    }
+    return callCount;
+  }
+
+   /**
+   * Retrieves the number of links to the specified file.
+   */
+  public static int getLinkCount(File fileName) throws IOException {
+    if (fileName == null) {
+      throw new IOException(
+          "invalid argument to getLinkCount: file name is null");
+    }
+    if (!fileName.exists()) {
+      throw new FileNotFoundException(fileName + " not found.");
+    }
+
+    // construct and execute shell command
+    String[] cmd = getHardLinkCommand.linkCount(fileName);
+    String inpMsg = null;
+    String errMsg = null;
+    int exitValue = -1;
+    BufferedReader in = null;
+    BufferedReader err = null;
+
+    Process process = Runtime.getRuntime().exec(cmd);
+    try {
+      exitValue = process.waitFor();
+      in = new BufferedReader(new InputStreamReader(
+                                  process.getInputStream()));
+      inpMsg = in.readLine();
+      err = new BufferedReader(new InputStreamReader(
+                                   process.getErrorStream()));
+      errMsg = err.readLine();
+      if (inpMsg == null || exitValue != 0) {
+        throw createIOException(fileName, inpMsg, errMsg, exitValue, null);
+      }
+      if (osType == OSType.OS_TYPE_SOLARIS) {
+        String[] result = inpMsg.split("\\s+");
+        return Integer.parseInt(result[1]);
+      } else {
+        return Integer.parseInt(inpMsg);
+      }
+    } catch (NumberFormatException e) {
+      throw createIOException(fileName, inpMsg, errMsg, exitValue, e);
+    } catch (InterruptedException e) {
+      throw createIOException(fileName, inpMsg, errMsg, exitValue, e);
+    } finally {
+      process.destroy();
+      if (in != null) in.close();
+      if (err != null) err.close();
+    }
+  }
+  
+  /* Create an IOException for failing to get link count. */
+  private static IOException createIOException(File f, String message,
+      String error, int exitvalue, Exception cause) {
+    
+    final String winErrMsg = "; Windows errors in getLinkCount are often due "
+         + "to Cygwin misconfiguration";
+
+    final String s = "Failed to get link count on file " + f
+        + ": message=" + message
+        + "; error=" + error
+        + ((osType == OSType.OS_TYPE_WINXP) ? winErrMsg : "")
+        + "; exit value=" + exitvalue;
+    return (cause == null) ? new IOException(s) : new IOException(s, cause);
+  }
+  
+  
+  /**
+   * HardLink statistics counters and methods.
+   * Not multi-thread safe, obviously.
+   * Init is called during HardLink instantiation, above.
+   * 
+   * These are intended for use by knowledgeable clients, not internally, 
+   * because many of the internal methods are static and can't update these
+   * per-instance counters.
+   */
+  public static class LinkStats {
+    public int countDirs = 0; 
+    public int countSingleLinks = 0; 
+    public int countMultLinks = 0; 
+    public int countFilesMultLinks = 0; 
+    public int countEmptyDirs = 0; 
+    public int countPhysicalFileCopies = 0;
+  
+    public void clear() {
+      countDirs = 0; 
+      countSingleLinks = 0; 
+      countMultLinks = 0; 
+      countFilesMultLinks = 0; 
+      countEmptyDirs = 0; 
+      countPhysicalFileCopies = 0;
+    }
+    
+    public String report() {
+      return "HardLinkStats: " + countDirs + " Directories, including " 
+      + countEmptyDirs + " Empty Directories, " 
+      + countSingleLinks 
+      + " single Link operations, " + countMultLinks 
+      + " multi-Link operations, linking " + countFilesMultLinks 
+      + " files, total " + (countSingleLinks + countFilesMultLinks) 
+      + " linkable files.  Also physically copied " 
+      + countPhysicalFileCopies + " other files.";
+    }
+  }
+}
+

Added: hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/TestHardLink.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/TestHardLink.java?rev=1080396&view=auto
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/TestHardLink.java (added)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/TestHardLink.java Thu Mar 10 23:33:52 2011
@@ -0,0 +1,423 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.junit.After;
+import static org.junit.Assert.*;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.hadoop.fs.HardLink.*;
+
+/**
+ * This testing is fairly lightweight.  Assumes HardLink routines will
+ * only be called when permissions etc are okay; no negative testing is
+ * provided.
+ * 
+ * These tests all use 
+ * "src" as the source directory, 
+ * "tgt_one" as the target directory for single-file hardlinking, and
+ * "tgt_mult" as the target directory for multi-file hardlinking.
+ * 
+ * Contents of them are/will be:
+ * dir:src: 
+ *   files: x1, x2, x3
+ * dir:tgt_one:
+ *   files: x1 (linked to src/x1), y (linked to src/x2), 
+ *          x3 (linked to src/x3), x11 (also linked to src/x1)
+ * dir:tgt_mult:
+ *   files: x1, x2, x3 (all linked to same name in src/)
+ *   
+ * NOTICE: This test class only tests the functionality of the OS
+ * upon which the test is run! (although you're pretty safe with the
+ * unix-like OS's, unless a typo sneaks in.)
+ * 
+ * Notes about Windows testing:  
+ * (a) In order to create hardlinks, the process must be run with 
+ * administrative privs, in both the account AND the invocation.
+ * For instance, to run within Eclipse, the Eclipse application must be 
+ * launched by right-clicking on it, and selecting "Run as Administrator" 
+ * (and that option will only be available if the current user id does 
+ * in fact have admin privs).
+ * (b) The getLinkCount() test case will fail for Windows, unless Cygwin
+ * is set up properly.  In particular, ${cygwin}/bin must be in
+ * the PATH environment variable, so the cygwin utilities can be found.
+ */
+public class TestHardLink {
+  
+  public static final String TEST_ROOT_DIR = 
+    System.getProperty("test.build.data", "build/test/data") + "/test";
+  final static private File TEST_DIR = new File(TEST_ROOT_DIR, "hl");
+  private static String DIR = "dir_";
+  //define source and target directories
+  private static File src = new File(TEST_DIR, DIR + "src");
+  private static File tgt_mult = new File(TEST_DIR, DIR + "tgt_mult");
+  private static File tgt_one = new File(TEST_DIR, DIR + "tgt_one");
+  //define source files
+  private static File x1 = new File(src, "x1");
+  private static File x2 = new File(src, "x2");
+  private static File x3 = new File(src, "x3");
+  //define File objects for the target hardlinks
+  private static File x1_one = new File(tgt_one, "x1");
+  private static File y_one = new File(tgt_one, "y");
+  private static File x3_one = new File(tgt_one, "x3");
+  private static File x11_one = new File(tgt_one, "x11");
+  private static File x1_mult = new File(tgt_mult, "x1");
+  private static File x2_mult = new File(tgt_mult, "x2");
+  private static File x3_mult = new File(tgt_mult, "x3");
+  //content strings for file content testing
+  private static String str1 = "11111";
+  private static String str2 = "22222";
+  private static String str3 = "33333";
+
+  /**
+   * Assure clean environment for start of testing
+   * @throws IOException
+   */
+  @BeforeClass
+  public static void setupClean() throws IOException {
+    //delete source and target directories if they exist
+    FileUtil.fullyDelete(src);
+    FileUtil.fullyDelete(tgt_one);
+    FileUtil.fullyDelete(tgt_mult);
+    //check that they are gone
+    assertFalse(src.exists());
+    assertFalse(tgt_one.exists());
+    assertFalse(tgt_mult.exists());
+  }
+  
+  /**
+   * Initialize clean environment for start of each test
+   */
+  @Before
+  public void setupDirs() throws IOException {
+    //check that we start out with empty top-level test data directory
+    assertFalse(src.exists());
+    assertFalse(tgt_one.exists());
+    assertFalse(tgt_mult.exists());
+    //make the source and target directories
+    src.mkdirs();
+    tgt_one.mkdirs();
+    tgt_mult.mkdirs();
+    
+    //create the source files in src, with unique contents per file
+    makeNonEmptyFile(x1, str1);
+    makeNonEmptyFile(x2, str2);
+    makeNonEmptyFile(x3, str3);
+    //validate
+    validateSetup();
+  }
+  
+  /**
+   * validate that {@link setupDirs()} produced the expected result
+   */
+  private void validateSetup() throws IOException {
+    //check existence of source directory and files
+    assertTrue(src.exists());
+    assertEquals(3, src.list().length);
+    assertTrue(x1.exists());
+    assertTrue(x2.exists());
+    assertTrue(x3.exists());
+    //check contents of source files
+    assertTrue(fetchFileContents(x1).equals(str1));
+    assertTrue(fetchFileContents(x2).equals(str2));
+    assertTrue(fetchFileContents(x3).equals(str3));
+    //check target directories exist and are empty
+    assertTrue(tgt_one.exists());
+    assertTrue(tgt_mult.exists());
+    assertEquals(0, tgt_one.list().length);
+    assertEquals(0, tgt_mult.list().length);    
+  }
+  
+  /**
+   * validate that single-file link operations produced the expected results
+   */
+  private void validateTgtOne() throws IOException {
+    //check that target directory tgt_one ended up with expected four files
+    assertTrue(tgt_one.exists());
+    assertEquals(4, tgt_one.list().length);
+    assertTrue(x1_one.exists());
+    assertTrue(x11_one.exists());
+    assertTrue(y_one.exists());
+    assertTrue(x3_one.exists());
+    //confirm the contents of those four files reflects the known contents
+    //of the files they were hardlinked from.
+    assertTrue(fetchFileContents(x1_one).equals(str1));
+    assertTrue(fetchFileContents(x11_one).equals(str1));
+    assertTrue(fetchFileContents(y_one).equals(str2));
+    assertTrue(fetchFileContents(x3_one).equals(str3));
+  }
+  
+  /**
+   * validate that multi-file link operations produced the expected results
+   */
+  private void validateTgtMult() throws IOException {
+    //check that target directory tgt_mult ended up with expected three files
+    assertTrue(tgt_mult.exists());
+    assertEquals(3, tgt_mult.list().length);
+    assertTrue(x1_mult.exists());
+    assertTrue(x2_mult.exists());
+    assertTrue(x3_mult.exists());
+    //confirm the contents of those three files reflects the known contents
+    //of the files they were hardlinked from.
+    assertTrue(fetchFileContents(x1_mult).equals(str1));
+    assertTrue(fetchFileContents(x2_mult).equals(str2));
+    assertTrue(fetchFileContents(x3_mult).equals(str3));
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    setupClean();
+  }
+
+  private void makeNonEmptyFile(File file, String contents) 
+  throws IOException {
+    FileWriter fw = new FileWriter(file);
+    fw.write(contents);
+    fw.close();
+  }
+  
+  private void appendToFile(File file, String contents) 
+  throws IOException {
+    FileWriter fw = new FileWriter(file, true);
+    fw.write(contents);
+    fw.close();
+  }
+  
+  private String fetchFileContents(File file) 
+  throws IOException {
+    char[] buf = new char[20];
+    FileReader fr = new FileReader(file);
+    int cnt = fr.read(buf); 
+    fr.close();
+    char[] result = Arrays.copyOf(buf, cnt);
+    return new String(result);
+  }
+  
+  /**
+   * Sanity check the simplest case of HardLink.getLinkCount()
+   * to make sure we get back "1" for ordinary single-linked files.
+   * Tests with multiply-linked files are in later test cases.
+   * 
+   * If this fails on Windows but passes on Unix, the most likely cause is 
+   * incorrect configuration of the Cygwin installation; see above.
+   */
+  @Test
+  public void testGetLinkCount() throws IOException {
+    //at beginning of world, check that source files have link count "1"
+    //since they haven't been hardlinked yet
+    assertEquals(1, getLinkCount(x1));
+    assertEquals(1, getLinkCount(x2));
+    assertEquals(1, getLinkCount(x3));
+  }
+
+  /**
+   * Test the single-file method HardLink.createHardLink().
+   * Also tests getLinkCount() with values greater than one.
+   */
+  @Test
+  public void testCreateHardLink() throws IOException {
+    //hardlink a single file and confirm expected result
+    createHardLink(x1, x1_one);
+    assertTrue(x1_one.exists());
+    assertEquals(2, getLinkCount(x1));     //x1 and x1_one are linked now 
+    assertEquals(2, getLinkCount(x1_one)); //so they both have count "2"
+    //confirm that x2, which we didn't change, still shows count "1"
+    assertEquals(1, getLinkCount(x2));
+    
+    //now do a few more
+    createHardLink(x2, y_one);
+    createHardLink(x3, x3_one);
+    assertEquals(2, getLinkCount(x2)); 
+    assertEquals(2, getLinkCount(x3));
+
+    //create another link to a file that already has count 2
+    createHardLink(x1, x11_one);
+    assertEquals(3, getLinkCount(x1));      //x1, x1_one, and x11_one
+    assertEquals(3, getLinkCount(x1_one));  //are all linked, so they
+    assertEquals(3, getLinkCount(x11_one)); //should all have count "3"
+    
+    //validate by contents
+    validateTgtOne();
+    
+    //validate that change of content is reflected in the other linked files
+    appendToFile(x1_one, str3);
+    assertTrue(fetchFileContents(x1_one).equals(str1 + str3));
+    assertTrue(fetchFileContents(x11_one).equals(str1 + str3));
+    assertTrue(fetchFileContents(x1).equals(str1 + str3));
+  }
+  
+  /*
+   * Test the multi-file method HardLink.createHardLinkMult(),
+   * multiple files within a directory into one target directory
+   */
+  @Test
+  public void testCreateHardLinkMult() throws IOException {
+    //hardlink a whole list of three files at once
+    String[] fileNames = src.list();
+    createHardLinkMult(src, fileNames, tgt_mult);
+    
+    //validate by link count - each file has been linked once,
+    //so each count is "2"
+    assertEquals(2, getLinkCount(x1));
+    assertEquals(2, getLinkCount(x2));
+    assertEquals(2, getLinkCount(x3));
+    assertEquals(2, getLinkCount(x1_mult));
+    assertEquals(2, getLinkCount(x2_mult));
+    assertEquals(2, getLinkCount(x3_mult));
+
+    //validate by contents
+    validateTgtMult();
+    
+    //validate that change of content is reflected in the other linked files
+    appendToFile(x1_mult, str3);
+    assertTrue(fetchFileContents(x1_mult).equals(str1 + str3));
+    assertTrue(fetchFileContents(x1).equals(str1 + str3));
+  }
+
+  /**
+   * Test createHardLinkMult() with empty list of files.
+   * We use an extended version of the method call, that
+   * returns the number of System exec calls made, which should
+   * be zero in this case.
+   */
+  @Test
+  public void testCreateHardLinkMultEmptyList() throws IOException {
+    String[] emptyList = {};
+    
+    //test the case of empty file list
+    int callCount = createHardLinkMult(src, emptyList, tgt_mult, 
+        getMaxAllowedCmdArgLength());
+    //check no exec calls were made
+    assertEquals(0, callCount);
+    //check nothing changed in the directory tree
+    validateSetup();
+  }
+  
+  /**
+   * Test createHardLinkMult(), again, this time with the "too long list" 
+   * case where the total size of the command line arguments exceed the 
+   * allowed maximum.  In this case, the list should be automatically 
+   * broken up into chunks, each chunk no larger than the max allowed.
+   * 
+   * We use an extended version of the method call, specifying the
+   * size limit explicitly, to simulate the "too long" list with a 
+   * relatively short list.
+   */
+  @Test
+  public void testCreateHardLinkMultOversizeAndEmpty() throws IOException {
+    
+    // prep long filenames - each name takes 10 chars in the arg list
+    // (9 actual chars plus terminal null or delimeter blank)
+    String name1 = "x11111111";
+    String name2 = "x22222222";
+    String name3 = "x33333333";
+    File x1_long = new File(src, name1);
+    File x2_long = new File(src, name2);
+    File x3_long = new File(src, name3);
+    //set up source files with long file names
+    x1.renameTo(x1_long);
+    x2.renameTo(x2_long);
+    x3.renameTo(x3_long);
+    //validate setup
+    assertTrue(x1_long.exists());
+    assertTrue(x2_long.exists());
+    assertTrue(x3_long.exists());
+    assertFalse(x1.exists());
+    assertFalse(x2.exists());
+    assertFalse(x3.exists());
+    
+    //prep appropriate length information to construct test case for
+    //oversize filename list
+    int callCount;
+    String[] emptyList = {};
+    String[] fileNames = src.list();
+    //get fixed size of arg list without any filenames
+    int overhead = getLinkMultArgLength(src, emptyList, tgt_mult);
+    //select a maxLength that is slightly too short to hold 3 filenames
+    int maxLength = overhead + (int)(2.5 * (float)(1 + name1.length())); 
+    
+    //now test list of three filenames when there is room for only 2.5
+    callCount = createHardLinkMult(src, fileNames, tgt_mult, maxLength);
+    //check the request was completed in exactly two "chunks"
+    assertEquals(2, callCount);
+    //and check the results were as expected in the dir tree
+    assertTrue(Arrays.deepEquals(fileNames, tgt_mult.list()));
+    
+    //Test the case where maxlength is too small even for one filename.
+    //It should go ahead and try the single files.
+    
+    //Clear the test dir tree
+    FileUtil.fullyDelete(tgt_mult);
+    assertFalse(tgt_mult.exists());
+    tgt_mult.mkdirs();
+    assertTrue(tgt_mult.exists() && tgt_mult.list().length == 0);
+    //set a limit size much smaller than a single filename
+    maxLength = overhead + (int)(0.5 * (float)(1 + name1.length()));
+    //attempt the method call
+    callCount = createHardLinkMult(src, fileNames, tgt_mult, 
+        maxLength);
+    //should go ahead with each of the three single file names
+    assertEquals(3, callCount);
+    //check the results were as expected in the dir tree
+    assertTrue(Arrays.deepEquals(fileNames, tgt_mult.list()));
+  }
+  
+  /*
+   * Assume that this test won't usually be run on a Windows box.
+   * This test case allows testing of the correct syntax of the Windows
+   * commands, even though they don't actually get executed on a non-Win box.
+   * The basic idea is to have enough here that substantive changes will
+   * fail and the author will fix and add to this test as appropriate.
+   * 
+   * Depends on the HardLinkCGWin class and member fields being accessible
+   * from this test method.
+   */
+  @Test
+  public void testWindowsSyntax() {
+    class win extends HardLinkCGWin {};
+
+    //basic checks on array lengths
+    assertEquals(5, win.hardLinkCommand.length); 
+    assertEquals(7, win.hardLinkMultPrefix.length);
+    assertEquals(8, win.hardLinkMultSuffix.length);
+    assertEquals(3, win.getLinkCountCommand.length);
+
+    assertTrue(win.hardLinkMultPrefix[4].equals("%f"));
+    //make sure "%f" was not munged
+    assertEquals(2, ("%f").length()); 
+    assertTrue(win.hardLinkMultDir.equals("\\%f"));
+    //make sure "\\%f" was munged correctly
+    assertEquals(3, ("\\%f").length()); 
+    assertTrue(win.hardLinkMultSuffix[7].equals("1>NUL"));
+    //make sure "1>NUL" was not munged
+    assertEquals(5, ("1>NUL").length()); 
+    assertTrue(win.getLinkCountCommand[1].equals("-c%h"));
+    //make sure "-c%h" was not munged
+    assertEquals(4, ("-c%h").length()); 
+  }
+
+}