Browse Source

MAPREDUCE-3829. [Gridmix] Gridmix should give better error message when input data directory already exists and -generate option is given.(ravigummadi)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1299625 13f79535-47bb-0310-9956-ffa450edef68
Ravi Gummadi 13 years ago
parent
commit
7d60932060

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

@@ -52,6 +52,10 @@ Trunk (unreleased changes)
 
 
   BUG FIXES
   BUG FIXES
 
 
+    MAPREDUCE-3829. [Gridmix] Gridmix should give better error message when
+                    input data directory already exists and -generate opton is
+                    given.(ravigummadi)
+
     MAPREDUCE-2722. [Gridmix] Gridmix simulated job's map's hdfsBytesRead
     MAPREDUCE-2722. [Gridmix] Gridmix simulated job's map's hdfsBytesRead
                     counter is wrong when compressed input is used.(ravigummadi)
                     counter is wrong when compressed input is used.(ravigummadi)
 
 

+ 2 - 7
hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/DistributedCacheEmulator.java

@@ -86,11 +86,6 @@ class DistributedCacheEmulator {
 
 
   static final long AVG_BYTES_PER_MAP = 128 * 1024 * 1024L;// 128MB
   static final long AVG_BYTES_PER_MAP = 128 * 1024 * 1024L;// 128MB
 
 
-  // If at least 1 distributed cache file is missing in the expected
-  // distributed cache dir, Gridmix cannot proceed with emulation of
-  // distributed cache load.
-  int MISSING_DIST_CACHE_FILES_ERROR = 1;
-
   private Path distCachePath;
   private Path distCachePath;
 
 
   /**
   /**
@@ -154,7 +149,7 @@ class DistributedCacheEmulator {
    * <li> execute permission is not there for any of the ascendant directories
    * <li> execute permission is not there for any of the ascendant directories
    * of &lt;ioPath&gt; till root. This is because for emulation of distributed
    * of &lt;ioPath&gt; till root. This is because for emulation of distributed
    * cache load, distributed cache files created under
    * cache load, distributed cache files created under
-   * &lt;ioPath/distributedCache/public/&gt; should be considered by hadoop
+   * &lt;ioPath/distributedCache/&gt; should be considered by hadoop
    * as public distributed cache files.
    * as public distributed cache files.
    * <li> creation of pseudo local file system fails.</ol>
    * <li> creation of pseudo local file system fails.</ol>
    * <br> For (2), (3), (4) and (5), generation of distributed cache data
    * <br> For (2), (3), (4) and (5), generation of distributed cache data
@@ -470,7 +465,7 @@ class DistributedCacheEmulator {
           + "disable\ndistributed cache emulation by configuring '"
           + "disable\ndistributed cache emulation by configuring '"
           + DistributedCacheEmulator.GRIDMIX_EMULATE_DISTRIBUTEDCACHE
           + DistributedCacheEmulator.GRIDMIX_EMULATE_DISTRIBUTEDCACHE
           + "' to false.");
           + "' to false.");
-      return MISSING_DIST_CACHE_FILES_ERROR;
+      return Gridmix.MISSING_DIST_CACHE_FILES_ERROR;
     }
     }
     return 0;
     return 0;
   }
   }

+ 97 - 46
hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Gridmix.java

@@ -145,6 +145,18 @@ public class Gridmix extends Configured implements Tool {
   // Shutdown hook
   // Shutdown hook
   private final Shutdown sdh = new Shutdown();
   private final Shutdown sdh = new Shutdown();
 
 
+  /** Error while parsing/analyzing the arguments to Gridmix */
+  static final int ARGS_ERROR = 1;
+  /** Error while trying to start/setup the Gridmix run */
+  static final int STARTUP_FAILED_ERROR = 2;
+  /**
+   * If at least 1 distributed cache file is missing in the expected
+   * distributed cache dir, Gridmix cannot proceed with emulation of
+   * distributed cache load.
+   */
+  static final int MISSING_DIST_CACHE_FILES_ERROR = 3;
+
+
   Gridmix(String[] args) {
   Gridmix(String[] args) {
     summarizer = new Summarizer(args);
     summarizer = new Summarizer(args);
   }
   }
@@ -160,31 +172,42 @@ public class Gridmix extends Configured implements Tool {
   }
   }
   
   
   /**
   /**
-   * Write random bytes at the path &lt;inputDir&gt;.
+   * Write random bytes at the path &lt;inputDir&gt; if needed.
    * @see org.apache.hadoop.mapred.gridmix.GenerateData
    * @see org.apache.hadoop.mapred.gridmix.GenerateData
+   * @return exit status
    */
    */
-  protected void writeInputData(long genbytes, Path inputDir)
+  protected int writeInputData(long genbytes, Path inputDir)
       throws IOException, InterruptedException {
       throws IOException, InterruptedException {
-    final Configuration conf = getConf();
-    
-    // configure the compression ratio if needed
-    CompressionEmulationUtil.setupDataGeneratorConfig(conf);
+    if (genbytes > 0) {
+      final Configuration conf = getConf();
+
+      if (inputDir.getFileSystem(conf).exists(inputDir)) {
+        LOG.error("Gridmix input data directory " + inputDir
+                  + " already exists when -generate option is used.\n");
+        return STARTUP_FAILED_ERROR;
+      }
+
+      // configure the compression ratio if needed
+      CompressionEmulationUtil.setupDataGeneratorConfig(conf);
     
     
-    final GenerateData genData = new GenerateData(conf, inputDir, genbytes);
-    LOG.info("Generating " + StringUtils.humanReadableInt(genbytes) +
-        " of test data...");
-    launchGridmixJob(genData);
+      final GenerateData genData = new GenerateData(conf, inputDir, genbytes);
+      LOG.info("Generating " + StringUtils.humanReadableInt(genbytes) +
+               " of test data...");
+      launchGridmixJob(genData);
     
     
-    FsShell shell = new FsShell(conf);
-    try {
-      LOG.info("Changing the permissions for inputPath " + inputDir.toString());
-      shell.run(new String[] {"-chmod","-R","777", inputDir.toString()});
-    } catch (Exception e) {
-      LOG.error("Couldnt change the file permissions " , e);
-      throw new IOException(e);
+      FsShell shell = new FsShell(conf);
+      try {
+        LOG.info("Changing the permissions for inputPath " + inputDir.toString());
+        shell.run(new String[] {"-chmod","-R","777", inputDir.toString()});
+      } catch (Exception e) {
+        LOG.error("Couldnt change the file permissions " , e);
+        throw new IOException(e);
+      }
+
+      LOG.info("Input data generation successful.");
     }
     }
-    
-    LOG.info("Input data generation successful.");
+
+    return 0;
   }
   }
 
 
   /**
   /**
@@ -363,31 +386,33 @@ public class Gridmix extends Configured implements Tool {
   private int runJob(Configuration conf, String[] argv)
   private int runJob(Configuration conf, String[] argv)
     throws IOException, InterruptedException {
     throws IOException, InterruptedException {
     if (argv.length < 2) {
     if (argv.length < 2) {
+      LOG.error("Too few arguments to Gridmix.\n");
       printUsage(System.err);
       printUsage(System.err);
-      return 1;
+      return ARGS_ERROR;
     }
     }
-    
-    // Should gridmix generate distributed cache data ?
-    boolean generate = false;
+
     long genbytes = -1L;
     long genbytes = -1L;
     String traceIn = null;
     String traceIn = null;
     Path ioPath = null;
     Path ioPath = null;
     URI userRsrc = null;
     URI userRsrc = null;
-    userResolver = ReflectionUtils.newInstance(
-                     conf.getClass(GRIDMIX_USR_RSV, 
-                       SubmitterUserResolver.class,
-                       UserResolver.class), 
-                     conf);
     try {
     try {
+      userResolver = ReflectionUtils.newInstance(conf.getClass(GRIDMIX_USR_RSV, 
+                       SubmitterUserResolver.class, UserResolver.class), conf);
+
       for (int i = 0; i < argv.length - 2; ++i) {
       for (int i = 0; i < argv.length - 2; ++i) {
         if ("-generate".equals(argv[i])) {
         if ("-generate".equals(argv[i])) {
           genbytes = StringUtils.TraditionalBinaryPrefix.string2long(argv[++i]);
           genbytes = StringUtils.TraditionalBinaryPrefix.string2long(argv[++i]);
-          generate = true;
+          if (genbytes <= 0) {
+            LOG.error("size of input data to be generated specified using "
+                      + "-generate option should be nonnegative.\n");
+            return ARGS_ERROR;
+          }
         } else if ("-users".equals(argv[i])) {
         } else if ("-users".equals(argv[i])) {
           userRsrc = new URI(argv[++i]);
           userRsrc = new URI(argv[++i]);
         } else {
         } else {
+          LOG.error("Unknown option " + argv[i] + " specified.\n");
           printUsage(System.err);
           printUsage(System.err);
-          return 1;
+          return ARGS_ERROR;
         }
         }
       }
       }
 
 
@@ -397,10 +422,10 @@ public class Gridmix extends Configured implements Tool {
             LOG.warn("Ignoring the user resource '" + userRsrc + "'.");
             LOG.warn("Ignoring the user resource '" + userRsrc + "'.");
           }
           }
         } else {
         } else {
-          System.err.println("\n\n" + userResolver.getClass()
-              + " needs target user list. Use -users option." + "\n\n");
+          LOG.error(userResolver.getClass()
+              + " needs target user list. Use -users option.\n");
           printUsage(System.err);
           printUsage(System.err);
-          return 1;
+          return ARGS_ERROR;
         }
         }
       } else if (userRsrc != null) {
       } else if (userRsrc != null) {
         LOG.warn("Ignoring the user resource '" + userRsrc + "'.");
         LOG.warn("Ignoring the user resource '" + userRsrc + "'.");
@@ -409,11 +434,32 @@ public class Gridmix extends Configured implements Tool {
       ioPath = new Path(argv[argv.length - 2]);
       ioPath = new Path(argv[argv.length - 2]);
       traceIn = argv[argv.length - 1];
       traceIn = argv[argv.length - 1];
     } catch (Exception e) {
     } catch (Exception e) {
-      e.printStackTrace();
+      LOG.error(e.toString() + "\n");
+      if (LOG.isDebugEnabled()) {
+        e.printStackTrace();
+      }
+
       printUsage(System.err);
       printUsage(System.err);
-      return 1;
+      return ARGS_ERROR;
+    }
+
+    // Create <ioPath> with 777 permissions
+    final FileSystem inputFs = ioPath.getFileSystem(conf);
+    ioPath = ioPath.makeQualified(inputFs);
+    boolean succeeded = false;
+    try {
+      succeeded = FileSystem.mkdirs(inputFs, ioPath,
+                                    new FsPermission((short)0777));
+    } catch(IOException e) {
+      // No need to emit this exception message
+    } finally {
+      if (!succeeded) {
+        LOG.error("Failed creation of <ioPath> directory " + ioPath + "\n");
+        return STARTUP_FAILED_ERROR;
+      }
     }
     }
-    return start(conf, traceIn, ioPath, genbytes, userResolver, generate);
+
+    return start(conf, traceIn, ioPath, genbytes, userResolver);
   }
   }
 
 
   /**
   /**
@@ -429,17 +475,16 @@ public class Gridmix extends Configured implements Tool {
    * @param genbytes size of input data to be generated under the directory
    * @param genbytes size of input data to be generated under the directory
    *                 &lt;ioPath&gt;/input/
    *                 &lt;ioPath&gt;/input/
    * @param userResolver gridmix user resolver
    * @param userResolver gridmix user resolver
-   * @param generate true if -generate option was specified
    * @return exit code
    * @return exit code
    * @throws IOException
    * @throws IOException
    * @throws InterruptedException
    * @throws InterruptedException
    */
    */
   int start(Configuration conf, String traceIn, Path ioPath, long genbytes,
   int start(Configuration conf, String traceIn, Path ioPath, long genbytes,
-      UserResolver userResolver, boolean generate)
+      UserResolver userResolver)
       throws IOException, InterruptedException {
       throws IOException, InterruptedException {
     DataStatistics stats = null;
     DataStatistics stats = null;
     InputStream trace = null;
     InputStream trace = null;
-    ioPath = ioPath.makeQualified(ioPath.getFileSystem(conf));
+    int exitCode = 0;
 
 
     try {
     try {
       Path scratchDir = new Path(ioPath, conf.get(GRIDMIX_OUT_DIR, "gridmix"));
       Path scratchDir = new Path(ioPath, conf.get(GRIDMIX_OUT_DIR, "gridmix"));
@@ -455,19 +500,21 @@ public class Gridmix extends Configured implements Tool {
         Path inputDir = getGridmixInputDataPath(ioPath);
         Path inputDir = getGridmixInputDataPath(ioPath);
         
         
         // Write input data if specified
         // Write input data if specified
-        if (genbytes > 0) {
-          writeInputData(genbytes, inputDir);
+        exitCode = writeInputData(genbytes, inputDir);
+        if (exitCode != 0) {
+          return exitCode;
         }
         }
-        
+
         // publish the data statistics
         // publish the data statistics
         stats = GenerateData.publishDataStatistics(inputDir, genbytes, conf);
         stats = GenerateData.publishDataStatistics(inputDir, genbytes, conf);
         
         
         // scan input dir contents
         // scan input dir contents
         submitter.refreshFilePool();
         submitter.refreshFilePool();
 
 
+        boolean shouldGenerate = (genbytes > 0);
         // set up the needed things for emulation of various loads
         // set up the needed things for emulation of various loads
-        int exitCode = setupEmulation(conf, traceIn, scratchDir, ioPath,
-                                      generate);
+        exitCode = setupEmulation(conf, traceIn, scratchDir, ioPath,
+                                  shouldGenerate);
         if (exitCode != 0) {
         if (exitCode != 0) {
           return exitCode;
           return exitCode;
         }
         }
@@ -478,8 +525,12 @@ public class Gridmix extends Configured implements Tool {
         factory.start();
         factory.start();
         statistics.start();
         statistics.start();
       } catch (Throwable e) {
       } catch (Throwable e) {
-        LOG.error("Startup failed", e);
+        LOG.error("Startup failed. " + e.toString() + "\n");
+        if (LOG.isDebugEnabled()) {
+          e.printStackTrace();
+        }
         if (factory != null) factory.abort(); // abort pipeline
         if (factory != null) factory.abort(); // abort pipeline
+        exitCode = STARTUP_FAILED_ERROR;
       } finally {
       } finally {
         // signal for factory to start; sets start time
         // signal for factory to start; sets start time
         startFlag.countDown();
         startFlag.countDown();
@@ -510,7 +561,7 @@ public class Gridmix extends Configured implements Tool {
       }
       }
       IOUtils.cleanup(LOG, trace);
       IOUtils.cleanup(LOG, trace);
     }
     }
-    return 0;
+    return exitCode;
   }
   }
 
 
   /**
   /**

+ 2 - 1
hadoop-mapreduce-project/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestDistCacheEmulation.java

@@ -204,7 +204,8 @@ public class TestDistCacheEmulation {
 
 
     dce = createDistributedCacheEmulator(jobConf, ioPath, generate);
     dce = createDistributedCacheEmulator(jobConf, ioPath, generate);
     int exitCode = dce.setupGenerateDistCacheData(jobProducer);
     int exitCode = dce.setupGenerateDistCacheData(jobProducer);
-    int expectedExitCode = generate ? 0 : dce.MISSING_DIST_CACHE_FILES_ERROR;
+    int expectedExitCode =
+        generate ? 0 : Gridmix.MISSING_DIST_CACHE_FILES_ERROR;
     assertEquals("setupGenerateDistCacheData failed.",
     assertEquals("setupGenerateDistCacheData failed.",
                  expectedExitCode, exitCode);
                  expectedExitCode, exitCode);
 
 

+ 176 - 0
hadoop-mapreduce-project/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixExitCodes.java

@@ -0,0 +1,176 @@
+/**
+ * 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.mapred.gridmix;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+
+/** Test Gridmix exit codes for different error types */
+public class TestGridmixExitCodes {
+
+  @BeforeClass
+  public static void init() throws IOException {
+    TestGridmixSubmission.init();
+  }
+
+  @AfterClass
+  public static void shutDown() throws IOException {
+    TestGridmixSubmission.shutDown();
+  }
+
+  /**
+   * Test Gridmix exit codes for different error types like
+   * <li> when less than 2 arguments are provided to Gridmix
+   * <li> when input data dir already exists and -generate option is specified
+   * <li> Specifying negative input-data-size using -generate option
+   * <li> specifying a non-existing option to Gridmix command-line
+   * <li> Wrong combination of arguments to Gridmix run
+   * <li> Unable to create ioPath dir
+   * <li> Bad class specified as a user resolver
+   */
+  @Test
+  public void testGridmixExitCodes() throws Exception {
+    testTooFewArgs();
+    testNegativeInputDataSize();
+    testNonexistingOption();
+    testWrongArgs();
+    testBadUserResolvers();
+
+    testBadIOPath();
+    testExistingInputDataDir();
+  }
+
+  /**
+   * Specify less than 2 arguments to Gridmix and verify the exit code
+   */
+  private void testTooFewArgs() throws Exception {
+    int expectedExitCode = Gridmix.ARGS_ERROR;
+    // Provide only 1 argument to Gridmix
+    String[] argv = new String[1];
+    argv[0] = "ioPath";
+    TestGridmixSubmission.testGridmixExitCode(true, argv, expectedExitCode);
+  }
+
+  /**
+   * Specify -ve input data size to be generated and verify the exit code
+   */
+  private void testNegativeInputDataSize() throws Exception {
+    int expectedExitCode = Gridmix.ARGS_ERROR;
+
+    String[] argv = new String[4];
+    argv[0] = "-generate";
+    argv[1] = "-5m"; // -ve size
+    argv[2] = "ioPath";
+    argv[3] = "-";
+    TestGridmixSubmission.testGridmixExitCode(true, argv, expectedExitCode);
+  }
+
+  /**
+   * Specify a non-existing option to Gridmix command-line and verify
+   * the exit code
+   */
+  private void testNonexistingOption() throws Exception {
+    int expectedExitCode = Gridmix.ARGS_ERROR;
+    String[] argv = new String[3];
+    argv[0] = "-unknownOption";
+    argv[1] = "dummyArg1";
+    argv[2] = "dummyArg2";
+    // No need to call prepareArgs() as -unknownOption should make Gridmix fail
+    TestGridmixSubmission.testGridmixExitCode(true, argv, expectedExitCode);
+  }
+
+  /**
+   * Specify wrong combination of arguments to Gridmix run and verify
+   * the exit code. This is done by specifying RoundRobinUserResolver and not
+   * specifying -users option
+   */
+  private void testWrongArgs() throws Exception {
+    int expectedExitCode = Gridmix.ARGS_ERROR;
+    String[] argv = TestGridmixSubmission.prepareArgs(true,
+        RoundRobinUserResolver.class.getName());
+    TestGridmixSubmission.testGridmixExitCode(true, argv, expectedExitCode);
+  }
+
+  /**
+   * <li> Specify a non-existing class as a userResolver class and validate the
+   * exit code
+   * <li> Specify an existing class which doesn't implement {@link UserResolver}
+   * as a userResolver class and validate the exit code
+   */
+  private void testBadUserResolvers() throws Exception {
+    int expectedExitCode = Gridmix.ARGS_ERROR;
+
+    // Verify the case of an existing class that doesn't implement the
+    // interface UserResolver
+    String[] argv = TestGridmixSubmission.prepareArgs(true,
+        WrongUserResolver.class.getName());
+    TestGridmixSubmission.testGridmixExitCode(true, argv, expectedExitCode);
+
+    // Verify the case of a nonexisting class name as user resolver class
+    argv = TestGridmixSubmission.prepareArgs(true, "NonExistingUserResolver");
+    TestGridmixSubmission.testGridmixExitCode(true, argv, expectedExitCode);
+  }
+
+  /** A class which doesn't implement the interface {@link UserResolver} */
+  static class WrongUserResolver {}
+
+
+  /**
+   * Setup such that creation of ioPath dir fails and verify the exit code
+   */
+  private void testBadIOPath() throws Exception {
+    // Create foo as a file (not as a directory).
+    GridmixTestUtils.dfs.create(TestGridmixSubmission.ioPath);
+    // This ioPath cannot be created as a directory now.
+    int expectedExitCode = Gridmix.STARTUP_FAILED_ERROR;
+
+    String[] argv = TestGridmixSubmission.prepareArgs(true,
+        EchoUserResolver.class.getName());
+    TestGridmixSubmission.testGridmixExitCode(true, argv, expectedExitCode);
+  }
+
+  /**
+   * Create input data dir and specify -generate option verify the exit code
+   */
+  private void testExistingInputDataDir() throws Exception {
+    createInputDataDirectory(TestGridmixSubmission.ioPath);
+    int expectedExitCode = Gridmix.STARTUP_FAILED_ERROR;
+
+    String[] argv = TestGridmixSubmission.prepareArgs(true,
+        EchoUserResolver.class.getName());
+    TestGridmixSubmission.testGridmixExitCode(true, argv, expectedExitCode);
+  }
+
+  /**
+   * Create input data directory of Gridmix run
+   * @param ioPath ioPath argument of Gridmix run
+   */
+  private static void createInputDataDirectory(Path ioPath)
+      throws IOException {
+    Path inputDir = Gridmix.getGridmixInputDataPath(ioPath);
+    FileSystem.mkdirs(GridmixTestUtils.dfs, inputDir,
+                      new FsPermission((short)0777));
+  }
+}

+ 95 - 46
hadoop-mapreduce-project/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixSubmission.java

@@ -80,9 +80,15 @@ public class TestGridmixSubmission {
   private static final long GENDATA = 30; // in megabytes
   private static final long GENDATA = 30; // in megabytes
   private static final int GENSLOP = 100 * 1024; // +/- 100k for logs
   private static final int GENSLOP = 100 * 1024; // +/- 100k for logs
 
 
+  static Path ioPath;
+  private static Path out;
+  private static final Path root = new Path("/user");
+
   @BeforeClass
   @BeforeClass
   public static void init() throws IOException {
   public static void init() throws IOException {
     GridmixTestUtils.initCluster();
     GridmixTestUtils.initCluster();
+    ioPath = new Path("foo").makeQualified(GridmixTestUtils.dfs);
+    out = GridmixTestUtils.DEST.makeQualified(GridmixTestUtils.dfs);
   }
   }
 
 
   @AfterClass
   @AfterClass
@@ -453,7 +459,7 @@ public class TestGridmixSubmission {
     } finally {
     } finally {
       System.setIn(origStdIn);
       System.setIn(origStdIn);
       if (tmpIs != null) {
       if (tmpIs != null) {
-	tmpIs.close();
+	       tmpIs.close();
       }
       }
       lfs.delete(rootTempDir, true);
       lfs.delete(rootTempDir, true);
     }
     }
@@ -499,59 +505,102 @@ public class TestGridmixSubmission {
     System.out.println("Serial ended at " + System.currentTimeMillis());
     System.out.println("Serial ended at " + System.currentTimeMillis());
   }
   }
 
 
+  /** Submit Gridmix run and verify that it succeeds */
   private void doSubmission(boolean useDefaultQueue,
   private void doSubmission(boolean useDefaultQueue,
       boolean defaultOutputPath) throws Exception {
       boolean defaultOutputPath) throws Exception {
-    final Path in = new Path("foo").makeQualified(GridmixTestUtils.dfs);
-    final Path out = GridmixTestUtils.DEST.makeQualified(GridmixTestUtils.dfs);
-    final Path root = new Path("/user");
-    Configuration conf = null;
-
-    try{
-      ArrayList<String> argsList = new ArrayList<String>();
-
-      argsList.add("-D" + FilePool.GRIDMIX_MIN_FILE + "=0");
-      argsList.add("-D" + Gridmix.GRIDMIX_USR_RSV + "="
-          + EchoUserResolver.class.getName());
-
-      // Set the config property gridmix.output.directory only if
-      // defaultOutputPath is false. If defaultOutputPath is true, then
-      // let us allow gridmix to use the path foo/gridmix/ as output dir.
-      if (!defaultOutputPath) {
-        argsList.add("-D" + Gridmix.GRIDMIX_OUT_DIR + "=" + out);
-      }
-      argsList.add("-generate");
-      argsList.add(String.valueOf(GENDATA) + "m");
-      argsList.add(in.toString());
-      argsList.add("-"); // ignored by DebugGridmix
-
-      String[] argv = argsList.toArray(new String[argsList.size()]);
-
-      DebugGridmix client = new DebugGridmix();
-      conf = new Configuration();
-      conf.setEnum(GridmixJobSubmissionPolicy.JOB_SUBMISSION_POLICY,policy);
-      conf.set("mapreduce.job.hdfs-servers", "");
-      if (useDefaultQueue) {
-        conf.setBoolean(GridmixJob.GRIDMIX_USE_QUEUE_IN_TRACE, false);
-        conf.set(GridmixJob.GRIDMIX_DEFAULT_QUEUE, "q1");
-      } else {
-        conf.setBoolean(GridmixJob.GRIDMIX_USE_QUEUE_IN_TRACE, true);
-      }
-      conf = GridmixTestUtils.mrCluster.createJobConf(new JobConf(conf));
-      // allow synthetic users to create home directories
-      GridmixTestUtils.dfs.mkdirs(root, new FsPermission((short)0777));
-      GridmixTestUtils.dfs.setPermission(root, new FsPermission((short)0777));
-      int res = ToolRunner.run(conf, client, argv);
-      assertEquals("Client exited with nonzero status", 0, res);
-      client.checkMonitor(conf);
+    String[] argv = prepareArgs(defaultOutputPath,
+        EchoUserResolver.class.getName());
+    testGridmixExitCode(useDefaultQueue, argv, 0);
+  }
+
+  /**
+   * Setup args for Gridmix run and run gridmix and verify the exit code.
+   * @param useDefaultQueue whether to use default queue or not
+   * @param argv array of arguments to gridmix
+   * @param expectedExitCode the expected exit code of Gridmix run
+   */
+  static void testGridmixExitCode(boolean useDefaultQueue, String[] argv,
+      int expectedExitCode) throws Exception {
+
+    try {
+      // Allow synthetic users to create home directories
+      FileSystem.mkdirs(GridmixTestUtils.dfs, root,
+                        new FsPermission((short)0777));
+
+      runGridmix(useDefaultQueue, argv, expectedExitCode);
     } catch (Exception e) {
     } catch (Exception e) {
       e.printStackTrace();
       e.printStackTrace();
       // fail the test if there is an exception
       // fail the test if there is an exception
       throw new RuntimeException(e);
       throw new RuntimeException(e);
     } finally {
     } finally {
-      in.getFileSystem(conf).delete(in, true);
-      out.getFileSystem(conf).delete(out, true);
-      root.getFileSystem(conf).delete(root,true);
+      deletePath(ioPath);
+      deletePath(out);
+      deletePath(root);
     }
     }
   }
   }
 
 
+  /** Run gridmix with specified arguments and verify the exit code. */
+  private static void runGridmix(boolean useDefaultQueue, String[] argv,
+      int expectedExitCode) throws Exception {
+
+    DebugGridmix client = new DebugGridmix();
+    Configuration conf = new Configuration();
+    conf.setEnum(GridmixJobSubmissionPolicy.JOB_SUBMISSION_POLICY,policy);
+    conf.set("mapreduce.job.hdfs-servers", "");
+    if (useDefaultQueue) {
+      conf.setBoolean(GridmixJob.GRIDMIX_USE_QUEUE_IN_TRACE, false);
+      conf.set(GridmixJob.GRIDMIX_DEFAULT_QUEUE, "q1");
+    } else {
+      conf.setBoolean(GridmixJob.GRIDMIX_USE_QUEUE_IN_TRACE, true);
+    }
+    conf = GridmixTestUtils.mrCluster.createJobConf(new JobConf(conf));
+
+    int res = ToolRunner.run(conf, client, argv);
+    assertEquals("Gridmix exited with wrong exit status",
+                 expectedExitCode, res);
+    if (expectedExitCode == 0) {
+      client.checkMonitor(conf);
+
+      // Verify the permissions of ioPath
+      FsPermission perm =
+          GridmixTestUtils.dfs.getFileStatus(ioPath).getPermission();
+      assertEquals("Wrong permissions of ioPath",
+                   new FsPermission((short)0777), perm);
+    }
+  }
+
+  /**
+   * Create the list of arguments for the Gridmix run.
+   * @param defaultOutputPath Should the default output path be used for the
+   *                          Gridmix run ?
+   * @param userResolver the user resolver for the Gridmix run
+   * @return the array of arguments to Gridmix
+   */
+  static String[] prepareArgs(boolean defaultOutputPath, String userResolver) {
+    ArrayList<String> argsList = new ArrayList<String>();
+
+    argsList.add("-D" + FilePool.GRIDMIX_MIN_FILE + "=0");
+    argsList.add("-D" + Gridmix.GRIDMIX_USR_RSV + "=" + userResolver);
+
+    // Set the config property gridmix.output.directory only if
+    // defaultOutputPath is false. If defaultOutputPath is true, then
+    // let us allow gridmix to use the path foo/gridmix/ as output dir.
+    if (!defaultOutputPath) {
+      argsList.add("-D" + Gridmix.GRIDMIX_OUT_DIR + "=" + out);
+    }
+
+    argsList.add("-generate");
+    argsList.add(String.valueOf(GENDATA) + "m");
+    argsList.add(ioPath.toString());
+    argsList.add("-"); // ignored by DebugGridmix
+    return argsList.toArray(new String[argsList.size()]);
+  }
+
+  /** If the given path exists, deletes it and its contents recursively */
+  private static void deletePath(Path dir) throws IOException {
+    if (GridmixTestUtils.dfs.exists(dir)) {
+      GridmixTestUtils.dfs.setPermission(dir, new FsPermission((short)0777));
+      GridmixTestUtils.dfs.delete(dir, true);
+    }
+  }
 }
 }