Pārlūkot izejas kodu

HDFS-4219. Backport slive to branch-1. Backported by Arpit Gupta.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-1@1413767 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 12 gadi atpakaļ
vecāks
revīzija
108b0e8c30
39 mainītis faili ar 6464 papildinājumiem un 0 dzēšanām
  1. 3 0
      CHANGES.txt
  2. 119 0
      src/test/org/apache/hadoop/fs/slive/AppendOp.java
  3. 176 0
      src/test/org/apache/hadoop/fs/slive/ArgumentParser.java
  4. 39 0
      src/test/org/apache/hadoop/fs/slive/BadFileException.java
  5. 733 0
      src/test/org/apache/hadoop/fs/slive/ConfigExtractor.java
  6. 572 0
      src/test/org/apache/hadoop/fs/slive/ConfigMerger.java
  7. 150 0
      src/test/org/apache/hadoop/fs/slive/ConfigOption.java
  8. 92 0
      src/test/org/apache/hadoop/fs/slive/Constants.java
  9. 182 0
      src/test/org/apache/hadoop/fs/slive/CreateOp.java
  10. 46 0
      src/test/org/apache/hadoop/fs/slive/DataHasher.java
  11. 401 0
      src/test/org/apache/hadoop/fs/slive/DataVerifier.java
  12. 375 0
      src/test/org/apache/hadoop/fs/slive/DataWriter.java
  13. 92 0
      src/test/org/apache/hadoop/fs/slive/DeleteOp.java
  14. 95 0
      src/test/org/apache/hadoop/fs/slive/DummyInputFormat.java
  15. 67 0
      src/test/org/apache/hadoop/fs/slive/Formatter.java
  16. 89 0
      src/test/org/apache/hadoop/fs/slive/Helper.java
  17. 93 0
      src/test/org/apache/hadoop/fs/slive/ListOp.java
  18. 95 0
      src/test/org/apache/hadoop/fs/slive/MkdirOp.java
  19. 72 0
      src/test/org/apache/hadoop/fs/slive/ObserveableOp.java
  20. 111 0
      src/test/org/apache/hadoop/fs/slive/Operation.java
  21. 92 0
      src/test/org/apache/hadoop/fs/slive/OperationData.java
  22. 82 0
      src/test/org/apache/hadoop/fs/slive/OperationFactory.java
  23. 249 0
      src/test/org/apache/hadoop/fs/slive/OperationOutput.java
  24. 51 0
      src/test/org/apache/hadoop/fs/slive/OperationWeight.java
  25. 107 0
      src/test/org/apache/hadoop/fs/slive/PathFinder.java
  26. 76 0
      src/test/org/apache/hadoop/fs/slive/Range.java
  27. 141 0
      src/test/org/apache/hadoop/fs/slive/ReadOp.java
  28. 119 0
      src/test/org/apache/hadoop/fs/slive/RenameOp.java
  29. 186 0
      src/test/org/apache/hadoop/fs/slive/ReportWriter.java
  30. 65 0
      src/test/org/apache/hadoop/fs/slive/RouletteSelector.java
  31. 95 0
      src/test/org/apache/hadoop/fs/slive/SleepOp.java
  32. 197 0
      src/test/org/apache/hadoop/fs/slive/SliveMapper.java
  33. 39 0
      src/test/org/apache/hadoop/fs/slive/SlivePartitioner.java
  34. 124 0
      src/test/org/apache/hadoop/fs/slive/SliveReducer.java
  35. 328 0
      src/test/org/apache/hadoop/fs/slive/SliveTest.java
  36. 535 0
      src/test/org/apache/hadoop/fs/slive/TestSlive.java
  37. 55 0
      src/test/org/apache/hadoop/fs/slive/Timer.java
  38. 195 0
      src/test/org/apache/hadoop/fs/slive/WeightSelector.java
  39. 126 0
      src/test/org/apache/hadoop/fs/slive/Weights.java

+ 3 - 0
CHANGES.txt

@@ -39,6 +39,9 @@ Release 1.2.0 - unreleased
     support block placement with 4-layer network topology.  (Junping Du and
     support block placement with 4-layer network topology.  (Junping Du and
     Jing Zhao via szetszwo)
     Jing Zhao via szetszwo)
 
 
+    HDFS-4219. Backport slive to branch-1. (Backported by Arpit Gupta via
+    suresh)
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     HDFS-3515. Port HDFS-1457 to branch-1. (eli)
     HDFS-3515. Port HDFS-1457 to branch-1. (eli)

+ 119 - 0
src/test/org/apache/hadoop/fs/slive/AppendOp.java

@@ -0,0 +1,119 @@
+/**
+ * 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.slive;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.slive.DataWriter.GenerateOutput;
+import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
+
+/**
+ * Operation which selects a random file and appends a random amount of bytes
+ * (selected from the configuration for append size) to that file if it exists.
+ * 
+ * This operation will capture statistics on success for bytes written, time
+ * taken (milliseconds), and success count and on failure it will capture the
+ * number of failures and the time taken (milliseconds) to fail.
+ */
+class AppendOp extends Operation {
+
+  private static final Log LOG = LogFactory.getLog(AppendOp.class);
+
+  AppendOp(ConfigExtractor cfg, Random rnd) {
+    super(AppendOp.class.getSimpleName(), cfg, rnd);
+  }
+
+  /**
+   * Gets the file to append to
+   * 
+   * @return Path
+   */
+  protected Path getAppendFile() {
+    Path fn = getFinder().getFile();
+    return fn;
+  }
+
+  @Override // Operation
+  List<OperationOutput> run(FileSystem fs) {
+    List<OperationOutput> out = super.run(fs);
+    OutputStream os = null;
+    try {
+      Path fn = getAppendFile();
+      // determine file status for file length requirement
+      // to know if should fill in partial bytes
+      Range<Long> appendSizeRange = getConfig().getAppendSize();
+      if (getConfig().shouldAppendUseBlockSize()) {
+        appendSizeRange = getConfig().getBlockSize();
+      }
+      long appendSize = Range.betweenPositive(getRandom(), appendSizeRange);
+      long timeTaken = 0, bytesAppended = 0;
+      DataWriter writer = new DataWriter(getRandom());
+      LOG.info("Attempting to append to file at " + fn + " of size "
+          + Helper.toByteInfo(appendSize));
+      {
+        // open
+        long startTime = Timer.now();
+        os = fs.append(fn);
+        timeTaken += Timer.elapsed(startTime);
+        // append given length
+        GenerateOutput stats = writer.writeSegment(appendSize, os);
+        timeTaken += stats.getTimeTaken();
+        bytesAppended += stats.getBytesWritten();
+        // capture close time
+        startTime = Timer.now();
+        os.close();
+        os = null;
+        timeTaken += Timer.elapsed(startTime);
+      }
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.BYTES_WRITTEN, bytesAppended));
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.OK_TIME_TAKEN, timeTaken));
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.SUCCESSES, 1L));
+      LOG.info("Appended " + Helper.toByteInfo(bytesAppended) + " to file "
+          + fn + " in " + timeTaken + " milliseconds");
+    } catch (FileNotFoundException e) {
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.NOT_FOUND, 1L));
+      LOG.warn("Error with appending", e);
+    } catch (IOException e) {
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.FAILURES, 1L));
+      LOG.warn("Error with appending", e);
+    } finally {
+      if (os != null) {
+        try {
+          os.close();
+        } catch (IOException e) {
+          LOG.warn("Error with closing append stream", e);
+        }
+      }
+    }
+    return out;
+  }
+}

+ 176 - 0
src/test/org/apache/hadoop/fs/slive/ArgumentParser.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.fs.slive;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.PosixParser;
+import org.apache.hadoop.fs.slive.Constants.Distribution;
+import org.apache.hadoop.fs.slive.Constants.OperationType;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * Class which abstracts the parsing of command line arguments for slive test
+ */
+class ArgumentParser {
+
+  private Options optList;
+  private String[] argumentList;
+  private ParsedOutput parsed;
+
+  /**
+   * Result of a parse is the following object
+   */
+  static class ParsedOutput {
+    private CommandLine parsedData;
+    private ArgumentParser source;
+    private boolean needHelp;
+
+    ParsedOutput(CommandLine parsedData, ArgumentParser source,
+        boolean needHelp) {
+      this.parsedData = parsedData;
+      this.source = source;
+      this.needHelp = needHelp;
+    }
+
+    /**
+     * @return whether the calling object should call output help and exit
+     */
+    boolean shouldOutputHelp() {
+      return needHelp;
+    }
+
+    /**
+     * Outputs the formatted help to standard out
+     */
+    void outputHelp() {
+      if (!shouldOutputHelp()) {
+        return;
+      }
+      if (source != null) {
+        HelpFormatter hlp = new HelpFormatter();
+        hlp.printHelp(Constants.PROG_NAME + " " + Constants.PROG_VERSION,
+            source.getOptionList());
+      }
+    }
+
+    /**
+     * @param optName
+     *          the option name to get the value for
+     * 
+     * @return the option value or null if it does not exist
+     */
+    String getValue(String optName) {
+      if (parsedData == null) {
+        return null;
+      }
+      return parsedData.getOptionValue(optName);
+    }
+
+    public String toString() {
+      StringBuilder s = new StringBuilder();
+      if (parsedData != null) {
+        Option[] ops = parsedData.getOptions();
+        for (int i = 0; i < ops.length; ++i) {
+          s.append(ops[i].getOpt() + " = " + s.append(ops[i].getValue()) + ",");
+        }
+      }
+      return s.toString();
+    }
+
+  }
+
+  ArgumentParser(String[] args) {
+    optList = getOptions();
+    if (args == null) {
+      args = new String[] {};
+    }
+    argumentList = args;
+    parsed = null;
+  }
+
+  private Options getOptionList() {
+    return optList;
+  }
+
+  /**
+   * Parses the command line options
+   * 
+   * @return false if need to print help output
+   * 
+   * @throws Exception
+   *           when parsing fails
+   */
+  ParsedOutput parse() throws Exception {
+    if (parsed == null) {
+      PosixParser parser = new PosixParser();
+      CommandLine popts = parser.parse(getOptionList(), argumentList, true);
+      if (popts.hasOption(ConfigOption.HELP.getOpt())) {
+        parsed = new ParsedOutput(null, this, true);
+      } else {
+        parsed = new ParsedOutput(popts, this, false);
+      }
+    }
+    return parsed;
+  }
+
+  /**
+   * @return the option set to be used in command line parsing
+   */
+  private Options getOptions() {
+    Options cliopt = new Options();
+    cliopt.addOption(ConfigOption.MAPS);
+    cliopt.addOption(ConfigOption.REDUCES);
+    cliopt.addOption(ConfigOption.PACKET_SIZE);
+    cliopt.addOption(ConfigOption.OPS);
+    cliopt.addOption(ConfigOption.DURATION);
+    cliopt.addOption(ConfigOption.EXIT_ON_ERROR);
+    cliopt.addOption(ConfigOption.SLEEP_TIME);
+    cliopt.addOption(ConfigOption.FILES);
+    cliopt.addOption(ConfigOption.DIR_SIZE);
+    cliopt.addOption(ConfigOption.BASE_DIR);
+    cliopt.addOption(ConfigOption.RESULT_FILE);
+    cliopt.addOption(ConfigOption.CLEANUP);
+    {
+      String distStrs[] = new String[Distribution.values().length];
+      Distribution distValues[] = Distribution.values();
+      for (int i = 0; i < distValues.length; ++i) {
+        distStrs[i] = distValues[i].lowerName();
+      }
+      String opdesc = String.format(Constants.OP_DESCR, StringUtils
+          .arrayToString(distStrs));
+      for (OperationType type : OperationType.values()) {
+        String opname = type.lowerName();
+        cliopt.addOption(new Option(opname, true, opdesc));
+      }
+    }
+    cliopt.addOption(ConfigOption.REPLICATION_AM);
+    cliopt.addOption(ConfigOption.BLOCK_SIZE);
+    cliopt.addOption(ConfigOption.READ_SIZE);
+    cliopt.addOption(ConfigOption.WRITE_SIZE);
+    cliopt.addOption(ConfigOption.APPEND_SIZE);
+    cliopt.addOption(ConfigOption.RANDOM_SEED);
+    cliopt.addOption(ConfigOption.QUEUE_NAME);
+    cliopt.addOption(ConfigOption.HELP);
+    return cliopt;
+  }
+
+}

+ 39 - 0
src/test/org/apache/hadoop/fs/slive/BadFileException.java

@@ -0,0 +1,39 @@
+/**
+ * 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.slive;
+
+import java.io.IOException;
+
+/**
+ * Exception used to signify file reading failures where headers are bad or an
+ * unexpected EOF occurs when it should not.
+ */
+class BadFileException extends IOException {
+
+  private static final long serialVersionUID = 463201983951298129L;
+
+  BadFileException(String msg) {
+    super(msg);
+  }
+
+  BadFileException(String msg, Throwable e) {
+    super(msg, e);
+  }
+
+}

+ 733 - 0
src/test/org/apache/hadoop/fs/slive/ConfigExtractor.java

@@ -0,0 +1,733 @@
+/**
+ * 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.slive;
+
+import java.text.NumberFormat;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.slive.Constants.OperationType;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * Simple access layer onto of a configuration object that extracts the slive
+ * specific configuration values needed for slive running
+ */
+class ConfigExtractor {
+
+  private static final Log LOG = LogFactory.getLog(ConfigExtractor.class);
+
+  private Configuration config;
+
+  ConfigExtractor(Configuration cfg) {
+    this.config = cfg;
+  }
+
+  /**
+   * @return the wrapped configuration that this extractor will use
+   */
+  Configuration getConfig() {
+    return this.config;
+  }
+
+  /**
+   * @return the location of where data should be written to
+   */
+  Path getDataPath() {
+    Path base = getBaseDirectory();
+    if (base == null) {
+      return null;
+    }
+    return new Path(base, Constants.DATA_DIR);
+  }
+
+  /**
+   * @return the location of where the reducer should write its data to
+   */
+  Path getOutputPath() {
+    Path base = getBaseDirectory();
+    if (base == null) {
+      return null;
+    }
+    return new Path(base, Constants.OUTPUT_DIR);
+  }
+
+  /**
+   * @param primary
+   *          primary the initial string to be used for the value of this
+   *          configuration option (if not provided then config and then the
+   *          default are used)
+   * 
+   * @return the base directory where output & data should be stored using
+   *         primary,config,default (in that order)
+   */
+  Path getBaseDirectory(String primary) {
+    String path = primary;
+    if (path == null) {
+      path = config.get(ConfigOption.BASE_DIR.getCfgOption());
+    }
+    if (path == null) {
+      path = ConfigOption.BASE_DIR.getDefault();
+    }
+    if (path == null) {
+      return null;
+    }
+    return new Path(path);
+  }
+
+  /**
+   * @return the base directory using only config and default values
+   */
+  Path getBaseDirectory() {
+    return getBaseDirectory(null);
+  }
+
+  /**
+   * @return whether the mapper or reducer should exit when they get there first
+   *         error using only config and default values
+   */
+  boolean shouldExitOnFirstError() {
+    return shouldExitOnFirstError(null);
+  }
+
+  /**
+   * @param primary
+   *          primary the initial string to be used for the value of this
+   *          configuration option (if not provided then config and then the
+   *          default are used)
+   * 
+   * @return the boolean of whether the mapper/reducer should exit when they
+   *         first error from primary,config,default (in that order)
+   */
+  boolean shouldExitOnFirstError(String primary) {
+    String val = primary;
+    if (val == null) {
+      val = config.get(ConfigOption.EXIT_ON_ERROR.getCfgOption());
+    }
+    if (val == null) {
+      return ConfigOption.EXIT_ON_ERROR.getDefault();
+    }
+    return Boolean.parseBoolean(val);
+  }
+
+  /**
+   * @return the number of reducers to use
+   */
+  Integer getReducerAmount() {
+    // should be slive.reduces
+    return getInteger(null, ConfigOption.REDUCES);
+  }
+
+  /**
+   * @return the number of mappers to use using config and default values for
+   *         lookup
+   */
+  Integer getMapAmount() {
+    return getMapAmount(null);
+  }
+
+  /**
+   * @param primary
+   *          primary the initial string to be used for the value of this
+   *          configuration option (if not provided then config and then the
+   *          default are used)
+   * @return the reducer amount to use
+   */
+  Integer getMapAmount(String primary) {
+    return getInteger(primary, ConfigOption.MAPS);
+  }
+
+  /**
+   * @return the duration in seconds (or null or Integer.MAX for no limit) using
+   *         the configuration and default as lookup
+   */
+  Integer getDuration() {
+    return getDuration(null);
+  }
+
+  /**
+   * @return the duration in milliseconds or null if no limit using config and
+   *         default as lookup
+   */
+  Integer getDurationMilliseconds() {
+    Integer seconds = getDuration();
+    if (seconds == null || seconds == Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    }
+    int milliseconds = (seconds * 1000);
+    if (milliseconds < 0) {
+      milliseconds = 0;
+    }
+    return milliseconds;
+  }
+
+  /**
+   * @param primary
+   *          primary the initial string to be used for the value of this
+   *          configuration option (if not provided then config and then the
+   *          default are used)
+   * @return the duration in seconds (or null or Integer.MAX for no limit)
+   */
+  Integer getDuration(String primary) {
+    return getInteger(primary, ConfigOption.DURATION);
+  }
+
+  /**
+   * @return the total number of operations to run using config and default as
+   *         lookup
+   */
+  Integer getOpCount() {
+    return getOpCount(null);
+  }
+
+  /**
+   * @param primary
+   *          primary the initial string to be used for the value of this
+   *          configuration option (if not provided then config and then the
+   *          default are used)
+   * @return the total number of operations to run
+   */
+  Integer getOpCount(String primary) {
+    return getInteger(primary, ConfigOption.OPS);
+  }
+
+  /**
+   * @return the total number of files per directory using config and default as
+   *         lookup
+   */
+  Integer getDirSize() {
+    return getDirSize(null);
+  }
+
+  /**
+   * @param primary
+   *          primary the initial string to be used for the value of this
+   *          configuration option (if not provided then config and then the
+   *          default are used)
+   * @return the total number of files per directory
+   */
+  Integer getDirSize(String primary) {
+    return getInteger(primary, ConfigOption.DIR_SIZE);
+  }
+
+  /**
+   * @param primary
+   *          the primary string to attempt to convert into a integer
+   * @param opt
+   *          the option to use as secondary + default if no primary given
+   * @return a parsed integer
+   */
+  private Integer getInteger(String primary, ConfigOption<Integer> opt) {
+    String value = primary;
+    if (value == null) {
+      value = config.get(opt.getCfgOption());
+    }
+    if (value == null) {
+      return opt.getDefault();
+    }
+    return Integer.parseInt(value);
+  }
+
+  /**
+   * @return the total number of files allowed using configuration and default
+   *         for lookup
+   */
+  Integer getTotalFiles() {
+    return getTotalFiles(null);
+  }
+
+  /**
+   * @param primary
+   *          primary the initial string to be used for the value of this
+   *          configuration option (if not provided then config and then the
+   *          default are used)
+   * @return the total number of files allowed
+   */
+  Integer getTotalFiles(String primary) {
+    return getInteger(primary, ConfigOption.FILES);
+  }
+
+  /**
+   * @param primary
+   *          primary the initial string to be used for the value of this
+   *          configuration option (if not provided then config and then the
+   *          default are used)
+   * @return the random seed start point or null if none
+   */
+  Long getRandomSeed(String primary) {
+    String seed = primary;
+    if (seed == null) {
+      seed = config.get(ConfigOption.RANDOM_SEED.getCfgOption());
+    }
+    if (seed == null) {
+      return null;
+    }
+    return Long.parseLong(seed);
+  }
+
+  /**
+   * @return the random seed start point or null if none using config and then
+   *         default as lookup
+   */
+  Long getRandomSeed() {
+    return getRandomSeed(null);
+  }
+
+  /**
+   * @return the result file location or null if none using config and then
+   *         default as lookup
+   */
+  String getResultFile() {
+    return getResultFile(null);
+  }
+
+  /**
+   * Gets the grid queue name to run on using config and default only
+   * 
+   * @return String
+   */
+  String getQueueName() {
+    return getQueueName(null);
+  }
+
+  /**
+   * Gets the grid queue name to run on using the primary string or config or
+   * default
+   * 
+   * @param primary
+   * 
+   * @return String
+   */
+  String getQueueName(String primary) {
+    String q = primary;
+    if (q == null) {
+      q = config.get(ConfigOption.QUEUE_NAME.getCfgOption());
+    }
+    if (q == null) {
+      q = ConfigOption.QUEUE_NAME.getDefault();
+    }
+    return q;
+  }
+
+  /**
+   * @param primary
+   *          primary the initial string to be used for the value of this
+   *          configuration option (if not provided then config and then the
+   *          default are used)
+   * @return the result file location
+   */
+  String getResultFile(String primary) {
+    String fn = primary;
+    if (fn == null) {
+      fn = config.get(ConfigOption.RESULT_FILE.getCfgOption());
+    }
+    if (fn == null) {
+      fn = ConfigOption.RESULT_FILE.getDefault();
+    }
+    return fn;
+  }
+
+  /**
+   * @param primary
+   *          primary the initial string to be used for the value of this
+   *          configuration option (if not provided then config and then the
+   *          default are used)
+   * @return the integer range allowed for the block size
+   */
+  Range<Long> getBlockSize(String primary) {
+    return getMinMaxBytes(ConfigOption.BLOCK_SIZE, primary);
+  }
+
+  /**
+   * @return the integer range allowed for the block size using config and
+   *         default for lookup
+   */
+  Range<Long> getBlockSize() {
+    return getBlockSize(null);
+  }
+
+  /**
+   * @param cfgopt
+   *          the configuration option to use for config and default lookup
+   * @param primary
+   *          the initial string to be used for the value of this configuration
+   *          option (if not provided then config and then the default are used)
+   * @return the parsed short range from primary, config, default
+   */
+  private Range<Short> getMinMaxShort(ConfigOption<Short> cfgopt, String primary) {
+    String sval = primary;
+    if (sval == null) {
+      sval = config.get(cfgopt.getCfgOption());
+    }
+    Range<Short> range = null;
+    if (sval != null) {
+      String pieces[] = Helper.getTrimmedStrings(sval);
+      if (pieces.length == 2) {
+        String min = pieces[0];
+        String max = pieces[1];
+        short minVal = Short.parseShort(min);
+        short maxVal = Short.parseShort(max);
+        if (minVal > maxVal) {
+          short tmp = minVal;
+          minVal = maxVal;
+          maxVal = tmp;
+        }
+        range = new Range<Short>(minVal, maxVal);
+      }
+    }
+    if (range == null) {
+      Short def = cfgopt.getDefault();
+      if (def != null) {
+        range = new Range<Short>(def, def);
+      }
+    }
+    return range;
+  }
+
+  /**
+   * @param cfgopt
+   *          the configuration option to use for config and default lookup
+   * @param primary
+   *          the initial string to be used for the value of this configuration
+   *          option (if not provided then config and then the default are used)
+   * @return the parsed long range from primary, config, default
+   */
+  private Range<Long> getMinMaxLong(ConfigOption<Long> cfgopt, String primary) {
+    String sval = primary;
+    if (sval == null) {
+      sval = config.get(cfgopt.getCfgOption());
+    }
+    Range<Long> range = null;
+    if (sval != null) {
+      String pieces[] = Helper.getTrimmedStrings(sval);
+      if (pieces.length == 2) {
+        String min = pieces[0];
+        String max = pieces[1];
+        long minVal = Long.parseLong(min);
+        long maxVal = Long.parseLong(max);
+        if (minVal > maxVal) {
+          long tmp = minVal;
+          minVal = maxVal;
+          maxVal = tmp;
+        }
+        range = new Range<Long>(minVal, maxVal);
+      }
+    }
+    if (range == null) {
+      Long def = cfgopt.getDefault();
+      if (def != null) {
+        range = new Range<Long>(def, def);
+      }
+    }
+    return range;
+  }
+
+  /**
+   * @param cfgopt
+   *          the configuration option to use for config and default lookup
+   * @param primary
+   *          the initial string to be used for the value of this configuration
+   *          option (if not provided then config and then the default are used)
+   * @return the parsed integer byte range from primary, config, default
+   */
+  private Range<Long> getMinMaxBytes(ConfigOption<Long> cfgopt, String primary) {
+    String sval = primary;
+    if (sval == null) {
+      sval = config.get(cfgopt.getCfgOption());
+    }
+    Range<Long> range = null;
+    if (sval != null) {
+      String pieces[] = Helper.getTrimmedStrings(sval);
+      if (pieces.length == 2) {
+        String min = pieces[0];
+        String max = pieces[1];
+        long tMin = StringUtils.TraditionalBinaryPrefix.string2long(min);
+        long tMax = StringUtils.TraditionalBinaryPrefix.string2long(max);
+        if (tMin > tMax) {
+          long tmp = tMin;
+          tMin = tMax;
+          tMax = tmp;
+        }
+        range = new Range<Long>(tMin, tMax);
+      }
+    }
+    if (range == null) {
+      Long def = cfgopt.getDefault();
+      if (def != null) {
+        range = new Range<Long>(def, def);
+      }
+    }
+    return range;
+  }
+
+  /**
+   * @param primary
+   *          the initial string to be used for the value of this configuration
+   *          option (if not provided then config and then the default are used)
+   * @return the replication range
+   */
+  Range<Short> getReplication(String primary) {
+    return getMinMaxShort(ConfigOption.REPLICATION_AM, primary);
+  }
+
+  /**
+   * @return the replication range using config and default for lookup
+   */
+  Range<Short> getReplication() {
+    return getReplication(null);
+  }
+
+  /**
+   * @return the map of operations to perform using config (percent may be null
+   *         if unspecified)
+   */
+  Map<OperationType, OperationData> getOperations() {
+    Map<OperationType, OperationData> operations = new HashMap<OperationType, OperationData>();
+    for (OperationType type : OperationType.values()) {
+      String opname = type.lowerName();
+      String keyname = String.format(Constants.OP, opname);
+      String kval = config.get(keyname);
+      if (kval == null) {
+        continue;
+      }
+      operations.put(type, new OperationData(kval));
+    }
+    return operations;
+  }
+
+  /**
+   * @param primary
+   *          the initial string to be used for the value of this configuration
+   *          option (if not provided then config and then the default are used)
+   * @return the append byte size range (or null if none)
+   */
+  Range<Long> getAppendSize(String primary) {
+    return getMinMaxBytes(ConfigOption.APPEND_SIZE, primary);
+  }
+
+  /**
+   * @return the append byte size range (or null if none) using config and
+   *         default for lookup
+   */
+  Range<Long> getAppendSize() {
+    return getAppendSize(null);
+  }
+
+  /**
+   * @param primary
+   *          the initial string to be used for the value of this configuration
+   *          option (if not provided then config and then the default are used)
+   * @return the sleep range (or null if none)
+   */
+  Range<Long> getSleepRange(String primary) {
+    return getMinMaxLong(ConfigOption.SLEEP_TIME, primary);
+  }
+
+  /**
+   * @return the sleep range (or null if none) using config and default for
+   *         lookup
+   */
+  Range<Long> getSleepRange() {
+    return getSleepRange(null);
+  }
+
+  /**
+   * @param primary
+   *          the initial string to be used for the value of this configuration
+   *          option (if not provided then config and then the default are used)
+   * @return the write byte size range (or null if none)
+   */
+  Range<Long> getWriteSize(String primary) {
+    return getMinMaxBytes(ConfigOption.WRITE_SIZE, primary);
+  }
+
+  /**
+   * @return the write byte size range (or null if none) using config and
+   *         default for lookup
+   */
+  Range<Long> getWriteSize() {
+    return getWriteSize(null);
+  }
+
+  /**
+   * Returns whether the write range should use the block size range
+   * 
+   * @return true|false
+   */
+  boolean shouldWriteUseBlockSize() {
+    Range<Long> writeRange = getWriteSize();
+    if (writeRange == null
+        || (writeRange.getLower() == writeRange.getUpper() && (writeRange
+            .getUpper() == Long.MAX_VALUE))) {
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Returns whether the append range should use the block size range
+   * 
+   * @return true|false
+   */
+  boolean shouldAppendUseBlockSize() {
+    Range<Long> appendRange = getAppendSize();
+    if (appendRange == null
+        || (appendRange.getLower() == appendRange.getUpper() && (appendRange
+            .getUpper() == Long.MAX_VALUE))) {
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Returns whether the read range should use the entire file
+   * 
+   * @return true|false
+   */
+  boolean shouldReadFullFile() {
+    Range<Long> readRange = getReadSize();
+    if (readRange == null
+        || (readRange.getLower() == readRange.getUpper() && (readRange
+            .getUpper() == Long.MAX_VALUE))) {
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * @param primary
+   *          the initial string to be used for the value of this configuration
+   *          option (if not provided then config and then the default are used)
+   * @return the read byte size range (or null if none)
+   */
+  Range<Long> getReadSize(String primary) {
+    return getMinMaxBytes(ConfigOption.READ_SIZE, primary);
+  }
+  
+  /**
+   * Gets the bytes per checksum (if it exists or null if not)
+   * 
+   * @return Long 
+   */
+  Long getByteCheckSum() {
+    String val = config.get(Constants.BYTES_PER_CHECKSUM);
+    if(val == null) {
+      return null;
+    }
+    return Long.parseLong(val);
+  }
+
+  /**
+   * @return the read byte size range (or null if none) using config and default
+   *         for lookup
+   */
+  Range<Long> getReadSize() {
+    return getReadSize(null);
+  }
+
+  /**
+   * Dumps out the given options for the given config extractor
+   * 
+   * @param cfg
+   *          the config to write to the log
+   */
+  static void dumpOptions(ConfigExtractor cfg) {
+    if (cfg == null) {
+      return;
+    }
+    LOG.info("Base directory = " + cfg.getBaseDirectory());
+    LOG.info("Data directory = " + cfg.getDataPath());
+    LOG.info("Output directory = " + cfg.getOutputPath());
+    LOG.info("Result file = " + cfg.getResultFile());
+    LOG.info("Grid queue = " + cfg.getQueueName());
+    LOG.info("Should exit on first error = " + cfg.shouldExitOnFirstError());
+    {
+      String duration = "Duration = ";
+      if (cfg.getDurationMilliseconds() == Integer.MAX_VALUE) {
+        duration += "unlimited";
+      } else {
+        duration += cfg.getDurationMilliseconds() + " milliseconds";
+      }
+      LOG.info(duration);
+    }
+    LOG.info("Map amount = " + cfg.getMapAmount());
+    LOG.info("Reducer amount = " + cfg.getReducerAmount());
+    LOG.info("Operation amount = " + cfg.getOpCount());
+    LOG.info("Total file limit = " + cfg.getTotalFiles());
+    LOG.info("Total dir file limit = " + cfg.getDirSize());
+    {
+      String read = "Read size = ";
+      if (cfg.shouldReadFullFile()) {
+        read += "entire file";
+      } else {
+        read += cfg.getReadSize() + " bytes";
+      }
+      LOG.info(read);
+    }
+    {
+      String write = "Write size = ";
+      if (cfg.shouldWriteUseBlockSize()) {
+        write += "blocksize";
+      } else {
+        write += cfg.getWriteSize() + " bytes";
+      }
+      LOG.info(write);
+    }
+    {
+      String append = "Append size = ";
+      if (cfg.shouldAppendUseBlockSize()) {
+        append += "blocksize";
+      } else {
+        append += cfg.getAppendSize() + " bytes";
+      }
+      LOG.info(append);
+    }
+    {
+      String bsize = "Block size = ";
+      bsize += cfg.getBlockSize() + " bytes";
+      LOG.info(bsize);
+    }
+    if (cfg.getRandomSeed() != null) {
+      LOG.info("Random seed = " + cfg.getRandomSeed());
+    }
+    if (cfg.getSleepRange() != null) {
+      LOG.info("Sleep range = " + cfg.getSleepRange() + " milliseconds");
+    }
+    LOG.info("Replication amount = " + cfg.getReplication());
+    LOG.info("Operations are:");
+    NumberFormat percFormatter = Formatter.getPercentFormatter();
+    Map<OperationType, OperationData> operations = cfg.getOperations();
+    for (OperationType type : operations.keySet()) {
+      String name = type.name();
+      LOG.info(name);
+      OperationData opInfo = operations.get(type);
+      LOG.info(" " + opInfo.getDistribution().name());
+      if (opInfo.getPercent() != null) {
+        LOG.info(" " + percFormatter.format(opInfo.getPercent()));
+      } else {
+        LOG.info(" ???");
+      }
+    }
+  }
+
+}

+ 572 - 0
src/test/org/apache/hadoop/fs/slive/ConfigMerger.java

@@ -0,0 +1,572 @@
+/**
+ * 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.slive;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.slive.ArgumentParser.ParsedOutput;
+import org.apache.hadoop.fs.slive.Constants.Distribution;
+import org.apache.hadoop.fs.slive.Constants.OperationType;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * Class which merges options given from a config file and the command line and
+ * performs some basic verification of the data retrieved and sets the verified
+ * values back into the configuration object for return
+ */
+class ConfigMerger {
+  /**
+   * Exception that represents config problems...
+   */
+  static class ConfigException extends IOException {
+
+    private static final long serialVersionUID = 2047129184917444550L;
+
+    ConfigException(String msg) {
+      super(msg);
+    }
+
+    ConfigException(String msg, Throwable e) {
+      super(msg, e);
+    }
+  }
+
+  /**
+   * Merges the given command line parsed output with the given configuration
+   * object and returns the new configuration object with the correct options
+   * overwritten
+   * 
+   * @param opts
+   *          the parsed command line option output
+   * @param base
+   *          the base configuration to merge with
+   * @return merged configuration object
+   * @throws ConfigException
+   *           when configuration errors or verification occur
+   */
+  Configuration getMerged(ParsedOutput opts, Configuration base)
+      throws ConfigException {
+    return handleOptions(opts, base);
+  }
+
+  /**
+   * Gets the base set of operations to use
+   * 
+   * @return Map
+   */
+  private Map<OperationType, OperationData> getBaseOperations() {
+    Map<OperationType, OperationData> base = new HashMap<OperationType, OperationData>();
+    // add in all the operations
+    // since they will all be applied unless changed
+    OperationType[] types = OperationType.values();
+    for (OperationType type : types) {
+      base.put(type, new OperationData(Distribution.UNIFORM, null));
+    }
+    return base;
+  }
+
+  /**
+   * Handles the specific task of merging operations from the command line or
+   * extractor object into the base configuration provided
+   * 
+   * @param opts
+   *          the parsed command line option output
+   * @param base
+   *          the base configuration to merge with
+   * @param extractor
+   *          the access object to fetch operations from if none from the
+   *          command line
+   * @return merged configuration object
+   * @throws ConfigException
+   *           when verification fails
+   */
+  private Configuration handleOperations(ParsedOutput opts, Configuration base,
+      ConfigExtractor extractor) throws ConfigException {
+    // get the base set to start off with
+    Map<OperationType, OperationData> operations = getBaseOperations();
+    // merge with what is coming from config
+    Map<OperationType, OperationData> cfgOperations = extractor.getOperations();
+    for (OperationType opType : cfgOperations.keySet()) {
+      operations.put(opType, cfgOperations.get(opType));
+    }
+    // see if any coming in from the command line
+    for (OperationType opType : OperationType.values()) {
+      String opName = opType.lowerName();
+      String opVal = opts.getValue(opName);
+      if (opVal != null) {
+        operations.put(opType, new OperationData(opVal));
+      }
+    }
+    // remove those with <= zero percent
+    {
+      Map<OperationType, OperationData> cleanedOps = new HashMap<OperationType, OperationData>();
+      for (OperationType opType : operations.keySet()) {
+        OperationData data = operations.get(opType);
+        if (data.getPercent() == null || data.getPercent() > 0.0d) {
+          cleanedOps.put(opType, data);
+        }
+      }
+      operations = cleanedOps;
+    }
+    if (operations.isEmpty()) {
+      throw new ConfigException("No operations provided!");
+    }
+    // verify and adjust
+    double currPct = 0;
+    int needFill = 0;
+    for (OperationType type : operations.keySet()) {
+      OperationData op = operations.get(type);
+      if (op.getPercent() != null) {
+        currPct += op.getPercent();
+      } else {
+        needFill++;
+      }
+    }
+    if (currPct > 1) {
+      throw new ConfigException(
+          "Unable to have accumlative percent greater than 100%");
+    }
+    if (needFill > 0 && currPct < 1) {
+      double leftOver = 1.0 - currPct;
+      Map<OperationType, OperationData> mpcp = new HashMap<OperationType, OperationData>();
+      for (OperationType type : operations.keySet()) {
+        OperationData op = operations.get(type);
+        if (op.getPercent() == null) {
+          op = new OperationData(op.getDistribution(), (leftOver / needFill));
+        }
+        mpcp.put(type, op);
+      }
+      operations = mpcp;
+    } else if (needFill == 0 && currPct < 1) {
+      // redistribute
+      double leftOver = 1.0 - currPct;
+      Map<OperationType, OperationData> mpcp = new HashMap<OperationType, OperationData>();
+      double each = leftOver / operations.keySet().size();
+      for (OperationType t : operations.keySet()) {
+        OperationData op = operations.get(t);
+        op = new OperationData(op.getDistribution(), (op.getPercent() + each));
+        mpcp.put(t, op);
+      }
+      operations = mpcp;
+    } else if (needFill > 0 && currPct >= 1) {
+      throw new ConfigException(needFill
+          + " unfilled operations but no percentage left to fill with");
+    }
+    // save into base
+    for (OperationType opType : operations.keySet()) {
+      String opName = opType.lowerName();
+      OperationData opData = operations.get(opType);
+      String distr = opData.getDistribution().lowerName();
+      String ratio = new Double(opData.getPercent() * 100.0d).toString();
+      base.set(String.format(Constants.OP, opName), opData.toString());
+      base.set(String.format(Constants.OP_DISTR, opName), distr);
+      base.set(String.format(Constants.OP_PERCENT, opName), ratio);
+    }
+    return base;
+  }
+
+  /**
+   * Handles merging all options and verifying from the given command line
+   * output and the given base configuration and returns the merged
+   * configuration
+   * 
+   * @param opts
+   *          the parsed command line option output
+   * @param base
+   *          the base configuration to merge with
+   * @return the merged configuration
+   * @throws ConfigException
+   */
+  private Configuration handleOptions(ParsedOutput opts, Configuration base)
+      throws ConfigException {
+    // ensure variables are overwritten and verified
+    ConfigExtractor extractor = new ConfigExtractor(base);
+    // overwrite the map amount and check to ensure > 0
+    {
+      Integer mapAmount = null;
+      try {
+        mapAmount = extractor.getMapAmount(opts.getValue(ConfigOption.MAPS
+            .getOpt()));
+      } catch (Exception e) {
+        throw new ConfigException("Error extracting & merging map amount", e);
+      }
+      if (mapAmount != null) {
+        if (mapAmount <= 0) {
+          throw new ConfigException(
+              "Map amount can not be less than or equal to zero");
+        }
+        base.set(ConfigOption.MAPS.getCfgOption(), mapAmount.toString());
+      }
+    }
+    // overwrite the reducer amount and check to ensure > 0
+    {
+      Integer reduceAmount = null;
+      try {
+        reduceAmount = extractor.getMapAmount(opts.getValue(ConfigOption.REDUCES
+            .getOpt()));
+      } catch (Exception e) {
+        throw new ConfigException(
+            "Error extracting & merging reducer amount", e);
+      }
+      if (reduceAmount != null) {
+        if (reduceAmount <= 0) {
+          throw new ConfigException(
+              "Reducer amount can not be less than or equal to zero");
+        }
+        base.set(ConfigOption.REDUCES.getCfgOption(), reduceAmount.toString());
+      }
+    }
+    // overwrite the duration amount and ensure > 0
+    {
+      Integer duration = null;
+      try {
+        duration = extractor.getDuration(opts.getValue(ConfigOption.DURATION
+            .getOpt()));
+      } catch (Exception e) {
+        throw new ConfigException("Error extracting & merging duration", e);
+      }
+      if (duration != null) {
+        if (duration <= 0) {
+          throw new ConfigException(
+              "Duration can not be less than or equal to zero");
+        }
+        base.set(ConfigOption.DURATION.getCfgOption(), duration.toString());
+      }
+    }
+    // overwrite the operation amount and ensure > 0
+    {
+      Integer operationAmount = null;
+      try {
+        operationAmount = extractor.getOpCount(opts.getValue(ConfigOption.OPS
+            .getOpt()));
+      } catch (Exception e) {
+        throw new ConfigException(
+            "Error extracting & merging operation amount", e);
+      }
+      if (operationAmount != null) {
+        if (operationAmount <= 0) {
+          throw new ConfigException(
+              "Operation amount can not be less than or equal to zero");
+        }
+        base.set(ConfigOption.OPS.getCfgOption(), operationAmount.toString());
+      }
+    }
+    // overwrite the exit on error setting
+    {
+      try {
+        boolean exitOnError = extractor.shouldExitOnFirstError(opts
+            .getValue(ConfigOption.EXIT_ON_ERROR.getOpt()));
+        base.setBoolean(ConfigOption.EXIT_ON_ERROR.getCfgOption(), exitOnError);
+      } catch (Exception e) {
+        throw new ConfigException(
+            "Error extracting & merging exit on error value", e);
+      }
+    }
+    // verify and set file limit and ensure > 0
+    {
+      Integer fileAm = null;
+      try {
+        fileAm = extractor.getTotalFiles(opts.getValue(ConfigOption.FILES
+            .getOpt()));
+      } catch (Exception e) {
+        throw new ConfigException(
+            "Error extracting & merging total file limit amount", e);
+      }
+      if (fileAm != null) {
+        if (fileAm <= 0) {
+          throw new ConfigException(
+              "File amount can not be less than or equal to zero");
+        }
+        base.set(ConfigOption.FILES.getCfgOption(), fileAm.toString());
+      }
+    }
+    // set the grid queue to run on
+    {
+      try {
+        String qname = extractor.getQueueName(opts
+            .getValue(ConfigOption.QUEUE_NAME.getOpt()));
+        if (qname != null) {
+          base.set(ConfigOption.QUEUE_NAME.getCfgOption(), qname);
+        }
+      } catch (Exception e) {
+        throw new ConfigException("Error extracting & merging queue name", e);
+      }
+    }
+    // verify and set the directory limit and ensure > 0
+    {
+      Integer directoryLimit = null;
+      try {
+        directoryLimit = extractor.getDirSize(opts
+            .getValue(ConfigOption.DIR_SIZE.getOpt()));
+      } catch (Exception e) {
+        throw new ConfigException(
+            "Error extracting & merging directory file limit", e);
+      }
+      if (directoryLimit != null) {
+        if (directoryLimit <= 0) {
+          throw new ConfigException(
+              "Directory file limit can not be less than or equal to zero");
+        }
+        base.set(ConfigOption.DIR_SIZE.getCfgOption(), directoryLimit
+            .toString());
+      }
+    }
+    // set the base directory
+    {
+      Path basedir = null;
+      try {
+        basedir = extractor.getBaseDirectory(opts
+            .getValue(ConfigOption.BASE_DIR.getOpt()));
+      } catch (Exception e) {
+        throw new ConfigException("Error extracting & merging base directory",
+            e);
+      }
+      if (basedir != null) {
+        // always ensure in slive dir
+        basedir = new Path(basedir, Constants.BASE_DIR);
+        base.set(ConfigOption.BASE_DIR.getCfgOption(), basedir.toString());
+      }
+    }
+    // set the result file
+    {
+      String fn = null;
+      try {
+        fn = extractor.getResultFile(opts.getValue(ConfigOption.RESULT_FILE
+            .getOpt()));
+      } catch (Exception e) {
+        throw new ConfigException("Error extracting & merging result file", e);
+      }
+      if (fn != null) {
+        base.set(ConfigOption.RESULT_FILE.getCfgOption(), fn);
+      }
+    }
+    {
+      String fn = null;
+      try {
+        fn = extractor.getResultFile(opts.getValue(ConfigOption.RESULT_FILE
+            .getOpt()));
+      } catch (Exception e) {
+        throw new ConfigException("Error extracting & merging result file", e);
+      }
+      if (fn != null) {
+        base.set(ConfigOption.RESULT_FILE.getCfgOption(), fn);
+      }
+    }
+    // set the operations
+    {
+      try {
+        base = handleOperations(opts, base, extractor);
+      } catch (Exception e) {
+        throw new ConfigException("Error extracting & merging operations", e);
+      }
+    }
+    // set the replication amount range
+    {
+      Range<Short> replicationAm = null;
+      try {
+        replicationAm = extractor.getReplication(opts
+            .getValue(ConfigOption.REPLICATION_AM.getOpt()));
+      } catch (Exception e) {
+        throw new ConfigException(
+            "Error extracting & merging replication amount range", e);
+      }
+      if (replicationAm != null) {
+        int minRepl = base.getInt(Constants.MIN_REPLICATION, 1);
+        if (replicationAm.getLower() < minRepl) {
+          throw new ConfigException(
+              "Replication amount minimum is less than property configured minimum "
+                  + minRepl);
+        }
+        if (replicationAm.getLower() > replicationAm.getUpper()) {
+          throw new ConfigException(
+              "Replication amount minimum is greater than its maximum");
+        }
+        if (replicationAm.getLower() <= 0) {
+          throw new ConfigException(
+              "Replication amount minimum must be greater than zero");
+        }
+        base.set(ConfigOption.REPLICATION_AM.getCfgOption(), replicationAm
+            .toString());
+      }
+    }
+    // set the sleep range
+    {
+      Range<Long> sleepRange = null;
+      try {
+        sleepRange = extractor.getSleepRange(opts
+            .getValue(ConfigOption.SLEEP_TIME.getOpt()));
+      } catch (Exception e) {
+        throw new ConfigException(
+            "Error extracting & merging sleep size range", e);
+      }
+      if (sleepRange != null) {
+        if (sleepRange.getLower() > sleepRange.getUpper()) {
+          throw new ConfigException(
+              "Sleep range minimum is greater than its maximum");
+        }
+        if (sleepRange.getLower() <= 0) {
+          throw new ConfigException(
+              "Sleep range minimum must be greater than zero");
+        }
+        base.set(ConfigOption.SLEEP_TIME.getCfgOption(), sleepRange.toString());
+      }
+    }
+    // set the packet size if given
+    {
+      String pSize = opts.getValue(ConfigOption.PACKET_SIZE.getOpt());
+      if (pSize == null) {
+        pSize = ConfigOption.PACKET_SIZE.getDefault();
+      }
+      if (pSize != null) {
+        try {
+          Long packetSize = StringUtils.TraditionalBinaryPrefix
+              .string2long(pSize);
+          base.set(ConfigOption.PACKET_SIZE.getCfgOption(), packetSize
+              .toString());
+        } catch (Exception e) {
+          throw new ConfigException(
+              "Error extracting & merging write packet size", e);
+        }
+      }
+    }
+    // set the block size range
+    {
+      Range<Long> blockSize = null;
+      try {
+        blockSize = extractor.getBlockSize(opts
+            .getValue(ConfigOption.BLOCK_SIZE.getOpt()));
+      } catch (Exception e) {
+        throw new ConfigException(
+            "Error extracting & merging block size range", e);
+      }
+      if (blockSize != null) {
+        if (blockSize.getLower() > blockSize.getUpper()) {
+          throw new ConfigException(
+              "Block size minimum is greater than its maximum");
+        }
+        if (blockSize.getLower() <= 0) {
+          throw new ConfigException(
+              "Block size minimum must be greater than zero");
+        }
+        // ensure block size is a multiple of BYTES_PER_CHECKSUM
+        // if a value is set in the configuration
+        Long bytesPerChecksum = extractor.getByteCheckSum();
+        if (bytesPerChecksum != null) {
+          if ((blockSize.getLower() % bytesPerChecksum) != 0) {
+            throw new ConfigException(
+                "Blocksize lower bound must be a multiple of "
+                    + bytesPerChecksum);
+          }
+          if ((blockSize.getUpper() % bytesPerChecksum) != 0) {
+            throw new ConfigException(
+                "Blocksize upper bound must be a multiple of "
+                    + bytesPerChecksum);
+          }
+        }
+        base.set(ConfigOption.BLOCK_SIZE.getCfgOption(), blockSize.toString());
+      }
+    }
+    // set the read size range
+    {
+      Range<Long> readSize = null;
+      try {
+        readSize = extractor.getReadSize(opts.getValue(ConfigOption.READ_SIZE
+            .getOpt()));
+      } catch (Exception e) {
+        throw new ConfigException("Error extracting & merging read size range",
+            e);
+      }
+      if (readSize != null) {
+        if (readSize.getLower() > readSize.getUpper()) {
+          throw new ConfigException(
+              "Read size minimum is greater than its maximum");
+        }
+        if (readSize.getLower() < 0) {
+          throw new ConfigException(
+              "Read size minimum must be greater than or equal to zero");
+        }
+        base.set(ConfigOption.READ_SIZE.getCfgOption(), readSize.toString());
+      }
+    }
+    // set the write size range
+    {
+      Range<Long> writeSize = null;
+      try {
+        writeSize = extractor.getWriteSize(opts
+            .getValue(ConfigOption.WRITE_SIZE.getOpt()));
+      } catch (Exception e) {
+        throw new ConfigException(
+            "Error extracting & merging write size range", e);
+      }
+      if (writeSize != null) {
+        if (writeSize.getLower() > writeSize.getUpper()) {
+          throw new ConfigException(
+              "Write size minimum is greater than its maximum");
+        }
+        if (writeSize.getLower() < 0) {
+          throw new ConfigException(
+              "Write size minimum must be greater than or equal to zero");
+        }
+        base.set(ConfigOption.WRITE_SIZE.getCfgOption(), writeSize.toString());
+      }
+    }
+    // set the append size range
+    {
+      Range<Long> appendSize = null;
+      try {
+        appendSize = extractor.getAppendSize(opts
+            .getValue(ConfigOption.APPEND_SIZE.getOpt()));
+      } catch (Exception e) {
+        throw new ConfigException(
+            "Error extracting & merging append size range", e);
+      }
+      if (appendSize != null) {
+        if (appendSize.getLower() > appendSize.getUpper()) {
+          throw new ConfigException(
+              "Append size minimum is greater than its maximum");
+        }
+        if (appendSize.getLower() < 0) {
+          throw new ConfigException(
+              "Append size minimum must be greater than or equal to zero");
+        }
+        base
+            .set(ConfigOption.APPEND_SIZE.getCfgOption(), appendSize.toString());
+      }
+    }
+    // set the seed
+    {
+      Long seed = null;
+      try {
+        seed = extractor.getRandomSeed(opts.getValue(ConfigOption.RANDOM_SEED
+            .getOpt()));
+      } catch (Exception e) {
+        throw new ConfigException(
+            "Error extracting & merging random number seed", e);
+      }
+      if (seed != null) {
+        base.set(ConfigOption.RANDOM_SEED.getCfgOption(), seed.toString());
+      }
+    }
+    return base;
+  }
+}

+ 150 - 0
src/test/org/apache/hadoop/fs/slive/ConfigOption.java

@@ -0,0 +1,150 @@
+/**
+ * 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.slive;
+
+import org.apache.commons.cli.Option;
+
+/**
+ * Class which extends the basic option object and adds in the configuration id
+ * and a default value so a central place can be used for retrieval of these as
+ * needed
+ */
+class ConfigOption<T> extends Option {
+
+  private static final long serialVersionUID = 7218954906367671150L;
+
+  // config starts with this prefix
+  private static final String SLIVE_PREFIX = "slive";
+
+  // command line options and descriptions and config option name
+  static final ConfigOption<Integer> MAPS = new ConfigOption<Integer>(
+      "maps", true, "Number of maps", SLIVE_PREFIX + ".maps", 10);
+
+  static final ConfigOption<Integer> REDUCES = new ConfigOption<Integer>(
+      "reduces", true, "Number of reduces", SLIVE_PREFIX + ".reduces", 1);
+
+  static final ConfigOption<Integer> OPS = new ConfigOption<Integer>(
+      "ops", true, "Max number of operations per map", SLIVE_PREFIX
+          + ".map.ops", 1000);
+
+  static final ConfigOption<Integer> DURATION = new ConfigOption<Integer>(
+      "duration", true,
+      "Duration of a map task in seconds (MAX_INT for no limit)", SLIVE_PREFIX
+          + ".duration", Integer.MAX_VALUE);
+
+  static final ConfigOption<Boolean> EXIT_ON_ERROR = new ConfigOption<Boolean>(
+      "exitOnError", false, "Exit on first error", SLIVE_PREFIX
+          + ".exit.on.error", false);
+
+  static final ConfigOption<Integer> FILES = new ConfigOption<Integer>(
+      "files", true, "Max total number of files",
+      SLIVE_PREFIX + ".total.files", 10);
+
+  static final ConfigOption<Integer> DIR_SIZE = new ConfigOption<Integer>(
+      "dirSize", true, "Max files per directory", SLIVE_PREFIX + ".dir.size",
+      32);
+
+  static final ConfigOption<String> BASE_DIR = new ConfigOption<String>(
+      "baseDir", true, "Base directory path", SLIVE_PREFIX + ".base.dir",
+      "/test/slive");
+
+  static final ConfigOption<String> RESULT_FILE = new ConfigOption<String>(
+      "resFile", true, "Result file name", SLIVE_PREFIX + ".result.file",
+      "part-0000");
+
+  static final ConfigOption<Short> REPLICATION_AM = new ConfigOption<Short>(
+      "replication", true, "Min,max value for replication amount", SLIVE_PREFIX
+          + ".file.replication", (short) 3);
+
+  static final ConfigOption<Long> BLOCK_SIZE = new ConfigOption<Long>(
+      "blockSize", true, "Min,max for dfs file block size", SLIVE_PREFIX
+          + ".block.size", 64L * Constants.MEGABYTES);
+
+  static final ConfigOption<Long> READ_SIZE = new ConfigOption<Long>(
+      "readSize", true,
+      "Min,max for size to read (min=max=MAX_LONG=read entire file)",
+      SLIVE_PREFIX + ".op.read.size", null);
+
+  static final ConfigOption<Long> WRITE_SIZE = new ConfigOption<Long>(
+      "writeSize", true,
+      "Min,max for size to write (min=max=MAX_LONG=blocksize)", SLIVE_PREFIX
+          + ".op.write.size", null);
+
+  static final ConfigOption<Long> SLEEP_TIME = new ConfigOption<Long>(
+      "sleep",
+      true,
+      "Min,max for millisecond of random sleep to perform (between operations)",
+      SLIVE_PREFIX + ".op.sleep.range", null);
+
+  static final ConfigOption<Long> APPEND_SIZE = new ConfigOption<Long>(
+      "appendSize", true,
+      "Min,max for size to append (min=max=MAX_LONG=blocksize)", SLIVE_PREFIX
+          + ".op.append.size", null);
+
+  static final ConfigOption<Long> RANDOM_SEED = new ConfigOption<Long>(
+      "seed", true, "Random number seed", SLIVE_PREFIX + ".seed", null);
+
+  // command line only options
+  static final Option HELP = new Option("help", false,
+      "Usage information");
+
+  static final Option CLEANUP = new Option("cleanup", true,
+      "Cleanup & remove directory after reporting");
+
+  // non slive specific settings
+  static final ConfigOption<String> QUEUE_NAME = new ConfigOption<String>(
+      "queue", true, "Queue name", "mapred.job.queue.name", "default");
+
+  static final ConfigOption<String> PACKET_SIZE = new ConfigOption<String>(
+      "packetSize", true, "Dfs write packet size", "dfs.write.packet.size",
+      null);
+
+  /**
+   * Hadoop configuration property name
+   */
+  private String cfgOption;
+
+  /**
+   * Default value if no value is located by other means
+   */
+  private T defaultValue;
+
+  ConfigOption(String cliOption, boolean hasArg, String description,
+      String cfgOption, T def) {
+    super(cliOption, hasArg, description);
+    this.cfgOption = cfgOption;
+    this.defaultValue = def;
+  }
+
+  /**
+   * @return the configuration option name to lookup in Configuration objects
+   *         for this option
+   */
+  String getCfgOption() {
+    return cfgOption;
+  }
+
+  /**
+   * @return the default object for this option
+   */
+  T getDefault() {
+    return defaultValue;
+  }
+
+}

+ 92 - 0
src/test/org/apache/hadoop/fs/slive/Constants.java

@@ -0,0 +1,92 @@
+/**
+ * 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.slive;
+
+/**
+ * Constants used in various places in slive
+ */
+class Constants {
+
+  /**
+   * This class should be static members only - no construction allowed
+   */
+  private Constants() {
+  }
+
+  /**
+   * The distributions supported (or that maybe supported)
+   */
+  enum Distribution {
+    BEG, END, UNIFORM, MID;
+    String lowerName() {
+      return this.name().toLowerCase();
+    }
+  }
+
+  /**
+   * Allowed operation types
+   */
+  enum OperationType {
+    READ, APPEND, RENAME, LS, MKDIR, DELETE, CREATE;
+    String lowerName() {
+      return this.name().toLowerCase();
+    }
+  }
+
+  // program info
+  static final String PROG_NAME = SliveTest.class.getSimpleName();
+  static final String PROG_VERSION = "0.0.2";
+
+  // useful constants
+  static final int MEGABYTES = 1048576;
+
+  // must be a multiple of
+  // BYTES_PER_LONG - used for reading and writing buffer sizes
+  static final int BUFFERSIZE = 64 * 1024;
+
+  // 8 bytes per long
+  static final int BYTES_PER_LONG = 8;
+
+  // used for finding the reducer file for a given number
+  static final String REDUCER_FILE = "part-%s";
+
+  // this is used to ensure the blocksize is a multiple of this config setting
+  static final String BYTES_PER_CHECKSUM = "io.bytes.per.checksum";
+
+  // min replication setting for verification
+  static final String MIN_REPLICATION = "dfs.namenode.replication.min";
+
+  // used for getting an option description given a set of distributions
+  // to substitute
+  static final String OP_DESCR = "pct,distribution where distribution is one of %s";
+
+  // keys for looking up a specific operation in the hadoop config
+  static final String OP_PERCENT = "slive.op.%s.pct";
+  static final String OP = "slive.op.%s";
+  static final String OP_DISTR = "slive.op.%s.dist";
+
+  // path constants
+  static final String BASE_DIR = "slive";
+  static final String DATA_DIR = "data";
+  static final String OUTPUT_DIR = "output";
+
+  // whether whenever data is written a flush should occur
+  static final boolean FLUSH_WRITES = false;
+
+}

+ 182 - 0
src/test/org/apache/hadoop/fs/slive/CreateOp.java

@@ -0,0 +1,182 @@
+/**
+ * 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.slive;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.slive.DataWriter.GenerateOutput;
+import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
+
+/**
+ * Operation which selects a random file and a random number of bytes to create
+ * that file with (from the write size option) and selects a random block size
+ * (from the block size option) and a random replication amount (from the
+ * replication option) and attempts to create a file with those options.
+ * 
+ * This operation will capture statistics on success for bytes written, time
+ * taken (milliseconds), and success count and on failure it will capture the
+ * number of failures and the time taken (milliseconds) to fail.
+ */
+class CreateOp extends Operation {
+
+  private static final Log LOG = LogFactory.getLog(CreateOp.class);
+
+  private static int DEF_IO_BUFFER_SIZE = 4096;
+
+  private static final String IO_BUF_CONFIG = ("io.file.buffer.size");
+
+  CreateOp(ConfigExtractor cfg, Random rnd) {
+    super(CreateOp.class.getSimpleName(), cfg, rnd);
+  }
+
+  /**
+   * Returns the block size to use (aligned to nearest BYTES_PER_CHECKSUM if
+   * configuration says a value exists) - this will avoid the warnings caused by
+   * this not occurring and the file will not be created if it is not correct...
+   * 
+   * @return long
+   */
+  private long determineBlockSize() {
+    Range<Long> blockSizeRange = getConfig().getBlockSize();
+    long blockSize = Range.betweenPositive(getRandom(), blockSizeRange);
+    Long byteChecksum = getConfig().getByteCheckSum();
+    if (byteChecksum == null) {
+      return blockSize;
+    }
+    // adjust to nearest multiple
+    long full = (blockSize / byteChecksum) * byteChecksum;
+    long toFull = blockSize - full;
+    if (toFull >= (byteChecksum / 2)) {
+      full += byteChecksum;
+    }
+    // adjust if over extended
+    if (full > blockSizeRange.getUpper()) {
+      full = blockSizeRange.getUpper();
+    }
+    if (full < blockSizeRange.getLower()) {
+      full = blockSizeRange.getLower();
+    }
+    return full;
+  }
+
+  /**
+   * Gets the replication amount
+   * 
+   * @return short
+   */
+  private short determineReplication() {
+    Range<Short> replicationAmountRange = getConfig().getReplication();
+    Range<Long> repRange = new Range<Long>(replicationAmountRange.getLower()
+        .longValue(), replicationAmountRange.getUpper().longValue());
+    short replicationAmount = (short) Range.betweenPositive(getRandom(),
+        repRange);
+    return replicationAmount;
+  }
+
+  /**
+   * Gets the output buffering size to use
+   * 
+   * @return int
+   */
+  private int getBufferSize() {
+    return getConfig().getConfig().getInt(IO_BUF_CONFIG, DEF_IO_BUFFER_SIZE);
+  }
+
+  /**
+   * Gets the file to create
+   * 
+   * @return Path
+   */
+  protected Path getCreateFile() {
+    Path fn = getFinder().getFile();
+    return fn;
+  }
+
+  @Override // Operation
+  List<OperationOutput> run(FileSystem fs) {
+    List<OperationOutput> out = super.run(fs);
+    FSDataOutputStream os = null;
+    try {
+      Path fn = getCreateFile();
+      Range<Long> writeSizeRange = getConfig().getWriteSize();
+      long writeSize = 0;
+      long blockSize = determineBlockSize();
+      short replicationAmount = determineReplication();
+      if (getConfig().shouldWriteUseBlockSize()) {
+        writeSizeRange = getConfig().getBlockSize();
+      }
+      writeSize = Range.betweenPositive(getRandom(), writeSizeRange);
+      long bytesWritten = 0;
+      long timeTaken = 0;
+      int bufSize = getBufferSize();
+      boolean overWrite = false;
+      DataWriter writer = new DataWriter(getRandom());
+      LOG.info("Attempting to create file at " + fn + " of size "
+          + Helper.toByteInfo(writeSize) + " using blocksize "
+          + Helper.toByteInfo(blockSize) + " and replication amount "
+          + replicationAmount);
+      {
+        // open & create
+        long startTime = Timer.now();
+        os = fs.create(fn, overWrite, bufSize, replicationAmount, blockSize);
+        timeTaken += Timer.elapsed(startTime);
+        // write the given length
+        GenerateOutput stats = writer.writeSegment(writeSize, os);
+        bytesWritten += stats.getBytesWritten();
+        timeTaken += stats.getTimeTaken();
+        // capture close time
+        startTime = Timer.now();
+        os.close();
+        os = null;
+        timeTaken += Timer.elapsed(startTime);
+      }
+      LOG.info("Created file at " + fn + " of size "
+          + Helper.toByteInfo(bytesWritten) + " bytes using blocksize "
+          + Helper.toByteInfo(blockSize) + " and replication amount "
+          + replicationAmount + " in " + timeTaken + " milliseconds");
+      // collect all the stats
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.OK_TIME_TAKEN, timeTaken));
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.BYTES_WRITTEN, bytesWritten));
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.SUCCESSES, 1L));
+    } catch (IOException e) {
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.FAILURES, 1L));
+      LOG.warn("Error with creating", e);
+    } finally {
+      if (os != null) {
+        try {
+          os.close();
+        } catch (IOException e) {
+          LOG.warn("Error closing create stream", e);
+        }
+      }
+    }
+    return out;
+  }
+}

+ 46 - 0
src/test/org/apache/hadoop/fs/slive/DataHasher.java

@@ -0,0 +1,46 @@
+/**
+ * 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.slive;
+
+import java.util.Random;
+
+/**
+ * Class which is used to create the data to write for a given path and offset
+ * into that file for writing and later verification that the expected value is
+ * read at that file bytes offset
+ */
+class DataHasher {
+
+  private Random rnd;
+
+  DataHasher(long mixIn) {
+    this.rnd = new Random(mixIn);
+  }
+
+  /**
+   * @param offSet
+   *          the byte offset into the file
+   * 
+   * @return the data to be expected at that offset
+   */
+  long generate(long offSet) {
+    return ((offSet * 47) ^ (rnd.nextLong() * 97)) * 37;
+  }
+
+}

+ 401 - 0
src/test/org/apache/hadoop/fs/slive/DataVerifier.java

@@ -0,0 +1,401 @@
+/**
+ * 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.slive;
+
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * Class which reads in and verifies bytes that have been read in
+ */
+class DataVerifier {
+  private static final int BYTES_PER_LONG = Constants.BYTES_PER_LONG;
+
+  private int bufferSize;
+
+  /**
+   * The output from verification includes the number of chunks that were the
+   * same as expected and the number of segments that were different than what
+   * was expected and the number of total bytes read
+   */
+  static class VerifyOutput {
+    private long same;
+    private long different;
+    private long read;
+    private long readTime;
+
+    VerifyOutput(long sameChunks, long differentChunks, long readBytes,
+        long readTime) {
+      this.same = sameChunks;
+      this.different = differentChunks;
+      this.read = readBytes;
+      this.readTime = readTime;
+    }
+
+    long getReadTime() {
+      return this.readTime;
+    }
+
+    long getBytesRead() {
+      return this.read;
+    }
+
+    long getChunksSame() {
+      return same;
+    }
+
+    long getChunksDifferent() {
+      return different;
+    }
+
+    public String toString() {
+      return "Bytes read = " + getBytesRead() + " same = " + getChunksSame()
+          + " different = " + getChunksDifferent() + " in " + getReadTime()
+          + " milliseconds";
+    }
+
+  }
+
+  /**
+   * Class used to hold the result of a read on a header
+   */
+  private static class ReadInfo {
+    private long byteAm;
+    private long hash;
+    private long timeTaken;
+    private long bytesRead;
+
+    ReadInfo(long byteAm, long hash, long timeTaken, long bytesRead) {
+      this.byteAm = byteAm;
+      this.hash = hash;
+      this.timeTaken = timeTaken;
+      this.bytesRead = bytesRead;
+    }
+
+    long getByteAm() {
+      return byteAm;
+    }
+
+    long getHashValue() {
+      return hash;
+    }
+
+    long getTimeTaken() {
+      return timeTaken;
+    }
+
+    long getBytesRead() {
+      return bytesRead;
+    }
+
+  }
+
+  /**
+   * Storage class used to hold the chunks same and different for buffered reads
+   * and the resultant verification
+   */
+  private static class VerifyInfo {
+
+    VerifyInfo(long same, long different) {
+      this.same = same;
+      this.different = different;
+    }
+
+    long getSame() {
+      return same;
+    }
+
+    long getDifferent() {
+      return different;
+    }
+
+    private long same;
+    private long different;
+  }
+
+  /**
+   * Inits with given buffer size (must be greater than bytes per long and a
+   * multiple of bytes per long)
+   * 
+   * @param bufferSize
+   *          size which must be greater than BYTES_PER_LONG and which also must
+   *          be a multiple of BYTES_PER_LONG
+   */
+  DataVerifier(int bufferSize) {
+    if (bufferSize < BYTES_PER_LONG) {
+      throw new IllegalArgumentException(
+          "Buffer size must be greater than or equal to " + BYTES_PER_LONG);
+    }
+    if ((bufferSize % BYTES_PER_LONG) != 0) {
+      throw new IllegalArgumentException("Buffer size must be a multiple of "
+          + BYTES_PER_LONG);
+    }
+    this.bufferSize = bufferSize;
+  }
+
+  /**
+   * Inits with the default buffer size
+   */
+  DataVerifier() {
+    this(Constants.BUFFERSIZE);
+  }
+
+  /**
+   * Verifies a buffer of a given size using the given start hash offset
+   * 
+   * @param buf
+   *          the buffer to verify
+   * @param size
+   *          the number of bytes to be used in that buffer
+   * @param startOffset
+   *          the start hash offset
+   * @param hasher
+   *          the hasher to use for calculating expected values
+   * 
+   * @return ResumeBytes a set of data about the next offset and chunks analyzed
+   */
+  private VerifyInfo verifyBuffer(ByteBuffer buf, int size, long startOffset,
+      DataHasher hasher) {
+    ByteBuffer cmpBuf = ByteBuffer.wrap(new byte[BYTES_PER_LONG]);
+    long hashOffset = startOffset;
+    long chunksSame = 0;
+    long chunksDifferent = 0;
+    for (long i = 0; i < size; ++i) {
+      cmpBuf.put(buf.get());
+      if (!cmpBuf.hasRemaining()) {
+        cmpBuf.rewind();
+        long receivedData = cmpBuf.getLong();
+        cmpBuf.rewind();
+        long expected = hasher.generate(hashOffset);
+        hashOffset += BYTES_PER_LONG;
+        if (receivedData == expected) {
+          ++chunksSame;
+        } else {
+          ++chunksDifferent;
+        }
+      }
+    }
+    // any left over??
+    if (cmpBuf.hasRemaining() && cmpBuf.position() != 0) {
+      // partial capture
+      // zero fill and compare with zero filled
+      int curSize = cmpBuf.position();
+      while (cmpBuf.hasRemaining()) {
+        cmpBuf.put((byte) 0);
+      }
+      long expected = hasher.generate(hashOffset);
+      ByteBuffer tempBuf = ByteBuffer.wrap(new byte[BYTES_PER_LONG]);
+      tempBuf.putLong(expected);
+      tempBuf.position(curSize);
+      while (tempBuf.hasRemaining()) {
+        tempBuf.put((byte) 0);
+      }
+      cmpBuf.rewind();
+      tempBuf.rewind();
+      if (cmpBuf.equals(tempBuf)) {
+        ++chunksSame;
+      } else {
+        ++chunksDifferent;
+      }
+    }
+    return new VerifyInfo(chunksSame, chunksDifferent);
+  }
+
+  /**
+   * Determines the offset to use given a byte counter
+   * 
+   * @param byteRead
+   * 
+   * @return offset position
+   */
+  private long determineOffset(long byteRead) {
+    if (byteRead < 0) {
+      byteRead = 0;
+    }
+    return (byteRead / BYTES_PER_LONG) * BYTES_PER_LONG;
+  }
+
+  /**
+   * Verifies a given number of bytes from a file - less number of bytes may be
+   * read if a header can not be read in due to the byte limit
+   * 
+   * @param byteAm
+   *          the byte amount to limit to (should be less than or equal to file
+   *          size)
+   * 
+   * @param in
+   *          the input stream to read from
+   * 
+   * @return VerifyOutput with data about reads
+   * 
+   * @throws IOException
+   *           if a read failure occurs
+   * 
+   * @throws BadFileException
+   *           if a header can not be read or end of file is reached
+   *           unexpectedly
+   */
+  VerifyOutput verifyFile(long byteAm, DataInputStream in)
+      throws IOException, BadFileException {
+    return verifyBytes(byteAm, 0, in);
+  }
+
+  /**
+   * Verifies a given number of bytes from a file - less number of bytes may be
+   * read if a header can not be read in due to the byte limit
+   * 
+   * @param byteAm
+   *          the byte amount to limit to (should be less than or equal to file
+   *          size)
+   * 
+   * @param bytesRead
+   *          the starting byte location
+   * 
+   * @param in
+   *          the input stream to read from
+   * 
+   * @return VerifyOutput with data about reads
+   * 
+   * @throws IOException
+   *           if a read failure occurs
+   * 
+   * @throws BadFileException
+   *           if a header can not be read or end of file is reached
+   *           unexpectedly
+   */
+  private VerifyOutput verifyBytes(long byteAm, long bytesRead,
+      DataInputStream in) throws IOException, BadFileException {
+    if (byteAm <= 0) {
+      return new VerifyOutput(0, 0, 0, 0);
+    }
+    long chunksSame = 0;
+    long chunksDifferent = 0;
+    long readTime = 0;
+    long bytesLeft = byteAm;
+    long bufLeft = 0;
+    long bufRead = 0;
+    long seqNum = 0;
+    DataHasher hasher = null;
+    ByteBuffer readBuf = ByteBuffer.wrap(new byte[bufferSize]);
+    while (bytesLeft > 0) {
+      if (bufLeft <= 0) {
+        if (bytesLeft < DataWriter.getHeaderLength()) {
+          // no bytes left to read a header
+          break;
+        }
+        // time to read a new header
+        ReadInfo header = null;
+        try {
+          header = readHeader(in);
+        } catch (EOFException e) {
+          // eof ok on header reads
+          // but not on data readers
+          break;
+        }
+        ++seqNum;
+        hasher = new DataHasher(header.getHashValue());
+        bufLeft = header.getByteAm();
+        readTime += header.getTimeTaken();
+        bytesRead += header.getBytesRead();
+        bytesLeft -= header.getBytesRead();
+        bufRead = 0;
+        // number of bytes to read greater than how many we want to read
+        if (bufLeft > bytesLeft) {
+          bufLeft = bytesLeft;
+        }
+        // does the buffer amount have anything??
+        if (bufLeft <= 0) {
+          continue;
+        }
+      }
+      // figure out the buffer size to read
+      int bufSize = bufferSize;
+      if (bytesLeft < bufSize) {
+        bufSize = (int) bytesLeft;
+      }
+      if (bufLeft < bufSize) {
+        bufSize = (int) bufLeft;
+      }
+      // read it in
+      try {
+        readBuf.rewind();
+        long startTime = Timer.now();
+        in.readFully(readBuf.array(), 0, bufSize);
+        readTime += Timer.elapsed(startTime);
+      } catch (EOFException e) {
+        throw new BadFileException(
+            "Could not read the number of expected data bytes " + bufSize
+                + " due to unexpected end of file during sequence " + seqNum, e);
+      }
+      // update the counters
+      bytesRead += bufSize;
+      bytesLeft -= bufSize;
+      bufLeft -= bufSize;
+      // verify what we read
+      readBuf.rewind();
+      // figure out the expected hash offset start point
+      long vOffset = determineOffset(bufRead);
+      // now update for new position
+      bufRead += bufSize;
+      // verify
+      VerifyInfo verifyRes = verifyBuffer(readBuf, bufSize, vOffset, hasher);
+      // update the verification counters
+      chunksSame += verifyRes.getSame();
+      chunksDifferent += verifyRes.getDifferent();
+    }
+    return new VerifyOutput(chunksSame, chunksDifferent, bytesRead, readTime);
+  }
+
+
+  /**
+   * Reads a header from the given input stream
+   * 
+   * @param in
+   *          input stream to read from
+   * 
+   * @return ReadInfo
+   * 
+   * @throws IOException
+   *           if a read error occurs or EOF occurs
+   * 
+   * @throws BadFileException
+   *           if end of file occurs or the byte amount read is invalid
+   */
+  ReadInfo readHeader(DataInputStream in) throws IOException,
+      BadFileException {
+    int headerLen = DataWriter.getHeaderLength();
+    ByteBuffer headerBuf = ByteBuffer.wrap(new byte[headerLen]);
+    long elapsed = 0;
+    {
+      long startTime = Timer.now();
+      in.readFully(headerBuf.array());
+      elapsed += Timer.elapsed(startTime);
+    }
+    headerBuf.rewind();
+    long hashValue = headerBuf.getLong();
+    long byteAvailable = headerBuf.getLong();
+    if (byteAvailable < 0) {
+      throw new BadFileException("Invalid negative amount " + byteAvailable
+          + " determined for header data amount");
+    }
+    return new ReadInfo(byteAvailable, hashValue, elapsed, headerLen);
+  }
+}

+ 375 - 0
src/test/org/apache/hadoop/fs/slive/DataWriter.java

@@ -0,0 +1,375 @@
+/**
+ * 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.slive;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.Random;
+
+import static org.apache.hadoop.fs.slive.Constants.BYTES_PER_LONG;
+
+/**
+ * Class which handles generating data (creating and appending) along with
+ * ensuring the correct data is written out for the given path name so that it
+ * can be later verified
+ */
+class DataWriter {
+  /**
+   * Header size in bytes
+   */
+  private static final int HEADER_LENGTH = (BYTES_PER_LONG * 2);
+
+  private int bufferSize;
+  private Random rnd;
+
+  /**
+   * Class used to hold the number of bytes written and time taken for write
+   * operations for callers to use
+   */
+  static class GenerateOutput {
+
+    private long bytes;
+    private long time;
+
+    GenerateOutput(long bytesWritten, long timeTaken) {
+      this.bytes = bytesWritten;
+      this.time = timeTaken;
+    }
+
+    long getBytesWritten() {
+      return bytes;
+    }
+
+    long getTimeTaken() {
+      return time;
+    }
+
+    public String toString() {
+      return "Wrote " + getBytesWritten() + " bytes " + " which took "
+          + getTimeTaken() + " milliseconds";
+    }
+  }
+
+  /**
+   * Class used to hold a byte buffer and offset position for generating data
+   */
+  private static class GenerateResult {
+    private long offset;
+    private ByteBuffer buffer;
+
+    GenerateResult(long offset, ByteBuffer buffer) {
+      this.offset = offset;
+      this.buffer = buffer;
+    }
+
+    long getOffset() {
+      return offset;
+    }
+
+    ByteBuffer getBuffer() {
+      return buffer;
+    }
+  }
+
+  /**
+   * What a header write output returns need the hash value to use and the time
+   * taken to perform the write + bytes written
+   */
+  private static class WriteInfo {
+    private long hashValue;
+    private long bytesWritten;
+    private long timeTaken;
+
+    WriteInfo(long hashValue, long bytesWritten, long timeTaken) {
+      this.hashValue = hashValue;
+      this.bytesWritten = bytesWritten;
+      this.timeTaken = timeTaken;
+    }
+
+    long getHashValue() {
+      return hashValue;
+    }
+
+    long getTimeTaken() {
+      return timeTaken;
+    }
+
+    long getBytesWritten() {
+      return bytesWritten;
+    }
+  }
+
+  /**
+   * Inits with given buffer size (must be greater than bytes per long and a
+   * multiple of bytes per long)
+   * 
+   * @param rnd
+   *          random number generator to use for hash value creation
+   * 
+   * @param bufferSize
+   *          size which must be greater than BYTES_PER_LONG and which also must
+   *          be a multiple of BYTES_PER_LONG
+   */
+  DataWriter(Random rnd, int bufferSize) {
+    if (bufferSize < BYTES_PER_LONG) {
+      throw new IllegalArgumentException(
+          "Buffer size must be greater than or equal to " + BYTES_PER_LONG);
+    }
+    if ((bufferSize % BYTES_PER_LONG) != 0) {
+      throw new IllegalArgumentException("Buffer size must be a multiple of "
+          + BYTES_PER_LONG);
+    }
+    this.bufferSize = bufferSize;
+    this.rnd = rnd;
+  }
+
+  /**
+   * Inits with default buffer size
+   */
+  DataWriter(Random rnd) {
+    this(rnd, Constants.BUFFERSIZE);
+  }
+
+  /**
+   * Generates a partial segment which is less than bytes per long size
+   * 
+   * @param byteAm
+   *          the number of bytes to generate (less than bytes per long)
+   * @param offset
+   *          the staring offset
+   * @param hasher
+   *          hasher to use for generating data given an offset
+   * 
+   * @return GenerateResult containing new offset and byte buffer
+   */
+  private GenerateResult generatePartialSegment(int byteAm, long offset,
+      DataHasher hasher) {
+    if (byteAm > BYTES_PER_LONG) {
+      throw new IllegalArgumentException(
+          "Partial bytes must be less or equal to " + BYTES_PER_LONG);
+    }
+    if (byteAm <= 0) {
+      throw new IllegalArgumentException(
+          "Partial bytes must be greater than zero and not " + byteAm);
+    }
+    ByteBuffer buf = ByteBuffer.wrap(new byte[BYTES_PER_LONG]);
+    buf.putLong(hasher.generate(offset));
+    ByteBuffer allBytes = ByteBuffer.wrap(new byte[byteAm]);
+    buf.rewind();
+    for (int i = 0; i < byteAm; ++i) {
+      allBytes.put(buf.get());
+    }
+    allBytes.rewind();
+    return new GenerateResult(offset, allBytes);
+  }
+
+  /**
+   * Generates a full segment (aligned to bytes per long) of the given byte
+   * amount size
+   * 
+   * @param byteAm
+   *          long aligned size
+   * @param startOffset
+   *          starting hash offset
+   * @param hasher
+   *          hasher to use for generating data given an offset
+   * @return GenerateResult containing new offset and byte buffer
+   */
+  private GenerateResult generateFullSegment(int byteAm, long startOffset,
+      DataHasher hasher) {
+    if (byteAm <= 0) {
+      throw new IllegalArgumentException(
+          "Byte amount must be greater than zero and not " + byteAm);
+    }
+    if ((byteAm % BYTES_PER_LONG) != 0) {
+      throw new IllegalArgumentException("Byte amount " + byteAm
+          + " must be a multiple of " + BYTES_PER_LONG);
+    }
+    // generate all the segments
+    ByteBuffer allBytes = ByteBuffer.wrap(new byte[byteAm]);
+    long offset = startOffset;
+    ByteBuffer buf = ByteBuffer.wrap(new byte[BYTES_PER_LONG]);
+    for (long i = 0; i < byteAm; i += BYTES_PER_LONG) {
+      buf.rewind();
+      buf.putLong(hasher.generate(offset));
+      buf.rewind();
+      allBytes.put(buf);
+      offset += BYTES_PER_LONG;
+    }
+    allBytes.rewind();
+    return new GenerateResult(offset, allBytes);
+  }
+
+  /**
+   * Writes a set of bytes to the output stream, for full segments it will write
+   * out the complete segment but for partial segments, ie when the last
+   * position does not fill up a full long then a partial set will be written
+   * out containing the needed bytes from the expected full segment
+   * 
+   * @param byteAm
+   *          the amount of bytes to write
+   * @param startPos
+   *          a BYTES_PER_LONG aligned start position
+   * @param hasher
+   *          hasher to use for generating data given an offset
+   * @param out
+   *          the output stream to write to
+   * @return how many bytes were written
+   * @throws IOException
+   */
+  private GenerateOutput writePieces(long byteAm, long startPos,
+      DataHasher hasher, OutputStream out) throws IOException {
+    if (byteAm <= 0) {
+      return new GenerateOutput(0, 0);
+    }
+    if (startPos < 0) {
+      startPos = 0;
+    }
+    int leftOver = (int) (byteAm % bufferSize);
+    long fullPieces = byteAm / bufferSize;
+    long offset = startPos;
+    long bytesWritten = 0;
+    long timeTaken = 0;
+    // write the full pieces that fit in the buffer size
+    for (long i = 0; i < fullPieces; ++i) {
+      GenerateResult genData = generateFullSegment(bufferSize, offset, hasher);
+      offset = genData.getOffset();
+      ByteBuffer gBuf = genData.getBuffer();
+      {
+        byte[] buf = gBuf.array();
+        long startTime = Timer.now();
+        out.write(buf);
+        if (Constants.FLUSH_WRITES) {
+          out.flush();
+        }
+        timeTaken += Timer.elapsed(startTime);
+        bytesWritten += buf.length;
+      }
+    }
+    if (leftOver > 0) {
+      ByteBuffer leftOverBuf = ByteBuffer.wrap(new byte[leftOver]);
+      int bytesLeft = leftOver % BYTES_PER_LONG;
+      leftOver = leftOver - bytesLeft;
+      // collect the piece which do not fit in the buffer size but is
+      // also greater or eq than BYTES_PER_LONG and a multiple of it
+      if (leftOver > 0) {
+        GenerateResult genData = generateFullSegment(leftOver, offset, hasher);
+        offset = genData.getOffset();
+        leftOverBuf.put(genData.getBuffer());
+      }
+      // collect any single partial byte segment
+      if (bytesLeft > 0) {
+        GenerateResult genData = generatePartialSegment(bytesLeft, offset,
+            hasher);
+        offset = genData.getOffset();
+        leftOverBuf.put(genData.getBuffer());
+      }
+      // do the write of both
+      leftOverBuf.rewind();
+      {
+        byte[] buf = leftOverBuf.array();
+        long startTime = Timer.now();
+        out.write(buf);
+        if (Constants.FLUSH_WRITES) {
+          out.flush();
+        }
+        timeTaken += Timer.elapsed(startTime);
+        bytesWritten += buf.length;
+      }
+    }
+    return new GenerateOutput(bytesWritten, timeTaken);
+  }
+
+  /**
+   * Writes to a stream the given number of bytes specified
+   * 
+   * @param byteAm
+   *          the file size in number of bytes to write
+   * 
+   * @param out
+   *          the outputstream to write to
+   * 
+   * @return the number of bytes written + time taken
+   * 
+   * @throws IOException
+   */
+  GenerateOutput writeSegment(long byteAm, OutputStream out)
+      throws IOException {
+    long headerLen = getHeaderLength();
+    if (byteAm < headerLen) {
+      // not enough bytes to write even the header
+      return new GenerateOutput(0, 0);
+    }
+    // adjust for header length
+    byteAm -= headerLen;
+    if (byteAm < 0) {
+      byteAm = 0;
+    }
+    WriteInfo header = writeHeader(out, byteAm);
+    DataHasher hasher = new DataHasher(header.getHashValue());
+    GenerateOutput pRes = writePieces(byteAm, 0, hasher, out);
+    long bytesWritten = pRes.getBytesWritten() + header.getBytesWritten();
+    long timeTaken = header.getTimeTaken() + pRes.getTimeTaken();
+    return new GenerateOutput(bytesWritten, timeTaken);
+  }
+
+  /**
+   * Gets the header length
+   * 
+   * @return int
+   */
+  static int getHeaderLength() {
+    return HEADER_LENGTH;
+  }
+
+  /**
+   * Writes a header to the given output stream
+   * 
+   * @param os
+   *          output stream to write to
+   * 
+   * @param fileSize
+   *          the file size to write
+   * 
+   * @return WriteInfo
+   * 
+   * @throws IOException
+   *           if a write failure occurs
+   */
+  WriteInfo writeHeader(OutputStream os, long fileSize) throws IOException {
+    int headerLen = getHeaderLength();
+    ByteBuffer buf = ByteBuffer.wrap(new byte[headerLen]);
+    long hash = rnd.nextLong();
+    buf.putLong(hash);
+    buf.putLong(fileSize);
+    buf.rewind();
+    byte[] headerData = buf.array();
+    long elapsed = 0;
+    {
+      long startTime = Timer.now();
+      os.write(headerData);
+      if (Constants.FLUSH_WRITES) {
+        os.flush();
+      }
+      elapsed += Timer.elapsed(startTime);
+    }
+    return new WriteInfo(hash, headerLen, elapsed);
+  }
+}

+ 92 - 0
src/test/org/apache/hadoop/fs/slive/DeleteOp.java

@@ -0,0 +1,92 @@
+/**
+ * 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.slive;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
+
+/**
+ * Operation which selects a random file and attempts to delete that file (if it
+ * exists)
+ * 
+ * This operation will capture statistics on success the time taken to delete
+ * and the number of successful deletions that occurred and on failure or error
+ * it will capture the number of failures and the amount of time taken to fail
+ */
+class DeleteOp extends Operation {
+
+  private static final Log LOG = LogFactory.getLog(DeleteOp.class);
+
+  DeleteOp(ConfigExtractor cfg, Random rnd) {
+    super(DeleteOp.class.getSimpleName(), cfg, rnd);
+  }
+
+  /**
+   * Gets the file to delete
+   */
+  protected Path getDeleteFile() {
+    Path fn = getFinder().getFile();
+    return fn;
+  }
+
+  @Override // Operation
+  List<OperationOutput> run(FileSystem fs) {
+    List<OperationOutput> out = super.run(fs);
+    try {
+      Path fn = getDeleteFile();
+      long timeTaken = 0;
+      boolean deleteStatus = false;
+      {
+        long startTime = Timer.now();
+        deleteStatus = fs.delete(fn, false);
+        timeTaken = Timer.elapsed(startTime);
+      }
+      // collect the stats
+      if (!deleteStatus) {
+        out.add(new OperationOutput(OutputType.LONG, getType(),
+            ReportWriter.FAILURES, 1L));
+        LOG.info("Could not delete " + fn);
+      } else {
+        out.add(new OperationOutput(OutputType.LONG, getType(),
+            ReportWriter.OK_TIME_TAKEN, timeTaken));
+        out.add(new OperationOutput(OutputType.LONG, getType(),
+            ReportWriter.SUCCESSES, 1L));
+        LOG.info("Could delete " + fn);
+      }
+    } catch (FileNotFoundException e) {
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.NOT_FOUND, 1L));
+      LOG.warn("Error with deleting", e);
+    } catch (IOException e) {
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.FAILURES, 1L));
+      LOG.warn("Error with deleting", e);
+    }
+    return out;
+  }
+
+}

+ 95 - 0
src/test/org/apache/hadoop/fs/slive/DummyInputFormat.java

@@ -0,0 +1,95 @@
+/**
+ * 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.slive;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+
+/**
+ * A input format which returns one dummy key and value
+ */
+@SuppressWarnings("deprecation")
+class DummyInputFormat implements InputFormat<Object, Object> {
+
+  static class EmptySplit implements InputSplit {
+    public void write(DataOutput out) throws IOException {
+    }
+
+    public void readFields(DataInput in) throws IOException {
+    }
+
+    public long getLength() {
+      return 0L;
+    }
+
+    public String[] getLocations() {
+      return new String[0];
+    }
+  }
+
+  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
+    InputSplit[] splits = new InputSplit[numSplits];
+    for (int i = 0; i < splits.length; ++i) {
+      splits[i] = new EmptySplit();
+    }
+    return splits;
+  }
+
+  public RecordReader<Object, Object> getRecordReader(InputSplit split,
+      JobConf job, Reporter reporter) throws IOException {
+    return new RecordReader<Object, Object>() {
+
+      boolean once = false;
+
+      public boolean next(Object key, Object value) throws IOException {
+        if (!once) {
+          once = true;
+          return true;
+        }
+        return false;
+      }
+
+      public Object createKey() {
+        return new Object();
+      }
+
+      public Object createValue() {
+        return new Object();
+      }
+
+      public long getPos() throws IOException {
+        return 0L;
+      }
+
+      public void close() throws IOException {
+      }
+
+      public float getProgress() throws IOException {
+        return 0.0f;
+      }
+    };
+  }
+}

+ 67 - 0
src/test/org/apache/hadoop/fs/slive/Formatter.java

@@ -0,0 +1,67 @@
+/**
+ * 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.slive;
+
+import java.text.DecimalFormat;
+import java.text.NumberFormat;
+
+/**
+ * Simple class that holds the number formatters used in the slive application
+ */
+class Formatter {
+
+  private static final String NUMBER_FORMAT = "###.###";
+
+  private static NumberFormat decFormatter = null;
+
+  private static NumberFormat percFormatter = null;
+
+  /**
+   * No construction allowed - only simple static accessor functions
+   */
+  private Formatter() {
+
+  }
+
+  /**
+   * Gets a decimal formatter that has 3 decimal point precision
+   * 
+   * @return NumberFormat formatter
+   */
+  static synchronized NumberFormat getDecimalFormatter() {
+    if (decFormatter == null) {
+      decFormatter = new DecimalFormat(NUMBER_FORMAT);
+    }
+    return decFormatter;
+  }
+
+  /**
+   * Gets a percent formatter that has 3 decimal point precision
+   * 
+   * @return NumberFormat formatter
+   */
+  static synchronized NumberFormat getPercentFormatter() {
+    if (percFormatter == null) {
+      percFormatter = NumberFormat.getPercentInstance();
+      percFormatter.setMaximumFractionDigits(3);
+    }
+    return percFormatter;
+  }
+
+}

+ 89 - 0
src/test/org/apache/hadoop/fs/slive/Helper.java

@@ -0,0 +1,89 @@
+/**
+ * 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.slive;
+
+/**
+ * Simple slive helper methods (may not exist in 0.20)
+ */
+class Helper {
+
+  private Helper() {
+
+  }
+
+  private static final String[] emptyStringArray = {};
+
+  /**
+   * Splits strings on comma and trims accordingly
+   * 
+   * @param str
+   * @return array of split
+   */
+  static String[] getTrimmedStrings(String str) {
+    if (null == str || "".equals(str.trim())) {
+      return emptyStringArray;
+    }
+    return str.trim().split("\\s*,\\s*");
+  }
+
+  /**
+   * Converts a byte value into a useful string for output
+   * 
+   * @param bytes
+   * 
+   * @return String
+   */
+  static String toByteInfo(long bytes) {
+    StringBuilder str = new StringBuilder();
+    if (bytes < 0) {
+      bytes = 0;
+    }
+    str.append(bytes);
+    str.append(" bytes or ");
+    str.append(bytes / 1024);
+    str.append(" kilobytes or ");
+    str.append(bytes / (1024 * 1024));
+    str.append(" megabytes or ");
+    str.append(bytes / (1024 * 1024 * 1024));
+    str.append(" gigabytes");
+    return str.toString();
+  }
+
+  /**
+   * Stringifys an array using the given separator.
+   * 
+   * @param args
+   *          the array to format
+   * @param sep
+   *          the separator string to use (ie comma or space)
+   * 
+   * @return String representing that array
+   */
+  static String stringifyArray(Object[] args, String sep) {
+    StringBuilder optStr = new StringBuilder();
+    for (int i = 0; i < args.length; ++i) {
+      optStr.append(args[i]);
+      if ((i + 1) != args.length) {
+        optStr.append(sep);
+      }
+    }
+    return optStr.toString();
+  }
+
+}

+ 93 - 0
src/test/org/apache/hadoop/fs/slive/ListOp.java

@@ -0,0 +1,93 @@
+/**
+ * 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.slive;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
+
+/**
+ * Operation which selects a random directory and attempts to list that
+ * directory (if it exists)
+ * 
+ * This operation will capture statistics on success the time taken to list that
+ * directory and the number of successful listings that occurred as well as the
+ * number of entries in the selected directory and on failure or error it will
+ * capture the number of failures and the amount of time taken to fail
+ */
+class ListOp extends Operation {
+
+  private static final Log LOG = LogFactory.getLog(ListOp.class);
+
+  ListOp(ConfigExtractor cfg, Random rnd) {
+    super(ListOp.class.getSimpleName(), cfg, rnd);
+  }
+
+  /**
+   * Gets the directory to list
+   * 
+   * @return Path
+   */
+  protected Path getDirectory() {
+    Path dir = getFinder().getDirectory();
+    return dir;
+  }
+
+  @Override // Operation
+  List<OperationOutput> run(FileSystem fs) {
+    List<OperationOutput> out = super.run(fs);
+    try {
+      Path dir = getDirectory();
+      long dirEntries = 0;
+      long timeTaken = 0;
+      {
+        long startTime = Timer.now();
+        FileStatus[] files = fs.listStatus(dir);
+        timeTaken = Timer.elapsed(startTime);
+        dirEntries = files.length;
+      }
+      // log stats
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.OK_TIME_TAKEN, timeTaken));
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.SUCCESSES, 1L));
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.DIR_ENTRIES, dirEntries));
+      LOG.info("Directory " + dir + " has " + dirEntries + " entries");
+    } catch (FileNotFoundException e) {
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.NOT_FOUND, 1L));
+      LOG.warn("Error with listing", e);
+    } catch (IOException e) {
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.FAILURES, 1L));
+      LOG.warn("Error with listing", e);
+    }
+    return out;
+  }
+
+}

+ 95 - 0
src/test/org/apache/hadoop/fs/slive/MkdirOp.java

@@ -0,0 +1,95 @@
+/**
+ * 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.slive;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
+
+/**
+ * Operation which selects a random directory and attempts to create that
+ * directory.
+ * 
+ * This operation will capture statistics on success the time taken to create
+ * that directory and the number of successful creations that occurred and on
+ * failure or error it will capture the number of failures and the amount of
+ * time taken to fail
+ */
+class MkdirOp extends Operation {
+
+  private static final Log LOG = LogFactory.getLog(MkdirOp.class);
+
+  MkdirOp(ConfigExtractor cfg, Random rnd) {
+    super(MkdirOp.class.getSimpleName(), cfg, rnd);
+  }
+  
+  /**
+   * Gets the directory name to try to make
+   * 
+   * @return Path
+   */
+  protected Path getDirectory() {
+    Path dir = getFinder().getDirectory();
+    return dir;
+  }
+
+  @Override // Operation
+  List<OperationOutput> run(FileSystem fs) {
+    List<OperationOutput> out = super.run(fs);
+    try {
+      Path dir = getDirectory();
+      boolean mkRes = false;
+      long timeTaken = 0;
+      {
+        long startTime = Timer.now();
+        mkRes = fs.mkdirs(dir);
+        timeTaken = Timer.elapsed(startTime);
+      }
+      // log stats
+      if (mkRes) {
+        out.add(new OperationOutput(OutputType.LONG, getType(),
+            ReportWriter.OK_TIME_TAKEN, timeTaken));
+        out.add(new OperationOutput(OutputType.LONG, getType(),
+            ReportWriter.SUCCESSES, 1L));
+        LOG.info("Made directory " + dir);
+      } else {
+        out.add(new OperationOutput(OutputType.LONG, getType(),
+            ReportWriter.FAILURES, 1L));
+        LOG.warn("Could not make " + dir);
+      }
+    } catch (FileNotFoundException e) {
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.NOT_FOUND, 1L));
+      LOG.warn("Error with mkdir", e);
+    } catch (IOException e) {
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.FAILURES, 1L));
+      LOG.warn("Error with mkdir", e);
+    }
+    return out;
+  }
+
+}

+ 72 - 0
src/test/org/apache/hadoop/fs/slive/ObserveableOp.java

@@ -0,0 +1,72 @@
+/**
+ * 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.slive;
+
+import java.util.List;
+
+import org.apache.hadoop.fs.FileSystem;
+
+/**
+ * Operation which wraps a given operation and allows an observer to be notified
+ * when the operation is about to start and when the operation has finished
+ */
+class ObserveableOp extends Operation {
+
+  /**
+   * The observation interface which class that wish to monitor starting and
+   * ending events must implement.
+   */
+  interface Observer {
+    void notifyStarting(Operation op);
+    void notifyFinished(Operation op);
+  }
+
+  private Operation op;
+  private Observer observer;
+
+  ObserveableOp(Operation op, Observer observer) {
+    super(op.getType(), op.getConfig(), op.getRandom());
+    this.op = op;
+    this.observer = observer;
+  }
+
+  /**
+   * Proxy to underlying operation toString()
+   */
+  public String toString() {
+    return op.toString();
+  }
+
+  @Override // Operation
+  List<OperationOutput> run(FileSystem fs) {
+    List<OperationOutput> result = null;
+    try {
+      if (observer != null) {
+        observer.notifyStarting(op);
+      }
+      result = op.run(fs);
+    } finally {
+      if (observer != null) {
+        observer.notifyFinished(op);
+      }
+    }
+    return result;
+  }
+
+}

+ 111 - 0
src/test/org/apache/hadoop/fs/slive/Operation.java

@@ -0,0 +1,111 @@
+/**
+ * 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.slive;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
+
+/**
+ * An operation provides these abstractions and if it desires to perform any
+ * operations it must implement a override of the run() function to provide
+ * varying output to be captured.
+ */
+abstract class Operation {
+
+  private ConfigExtractor config;
+  private PathFinder finder;
+  private String type;
+  private Random rnd;
+
+  protected Operation(String type, ConfigExtractor cfg, Random rnd) {
+    this.config = cfg;
+    this.type = type;
+    this.rnd = rnd;
+    // Use a new Random instance so that the sequence of file names produced is
+    // the same even in case of unsuccessful operations
+    this.finder = new PathFinder(cfg, new Random(rnd.nextInt()));
+  }
+
+  /**
+   * Gets the configuration object this class is using
+   * 
+   * @return ConfigExtractor
+   */
+  protected ConfigExtractor getConfig() {
+    return this.config;
+  }
+
+  /**
+   * Gets the random number generator to use for this operation
+   * 
+   * @return Random
+   */
+  protected Random getRandom() {
+    return this.rnd;
+  }
+
+  /**
+   * Gets the type of operation that this class belongs to
+   * 
+   * @return String
+   */
+  String getType() {
+    return type;
+  }
+
+  /**
+   * Gets the path finding/generating instance that this class is using
+   * 
+   * @return PathFinder
+   */
+  protected PathFinder getFinder() {
+    return this.finder;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.lang.Object#toString()
+   */
+  public String toString() {
+    return getType();
+  }
+
+  /**
+   * This run() method simply sets up the default output container and adds in a
+   * data member to keep track of the number of operations that occurred
+   * 
+   * @param fs
+   *          FileSystem object to perform operations with
+   * 
+   * @return List of operation outputs to be collected and output in the overall
+   *         map reduce operation (or empty or null if none)
+   */
+  List<OperationOutput> run(FileSystem fs) {
+    List<OperationOutput> out = new LinkedList<OperationOutput>();
+    out.add(new OperationOutput(OutputType.LONG, getType(),
+        ReportWriter.OP_COUNT, 1L));
+    return out;
+  }
+
+}

+ 92 - 0
src/test/org/apache/hadoop/fs/slive/OperationData.java

@@ -0,0 +1,92 @@
+/**
+ * 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.slive;
+
+import org.apache.hadoop.fs.slive.Constants.Distribution;
+
+/**
+ * This class holds the data representing what an operations distribution and
+ * its percentage is (between 0 and 1) and provides operations to access those
+ * types and parse and unparse from and into strings
+ */
+class OperationData {
+
+  private static final String SEP = ",";
+
+  private Distribution distribution;
+  private Double percent;
+
+  OperationData(Distribution d, Double p) {
+    this.distribution = d;
+    this.percent = p;
+  }
+
+  /**
+   * Expects a comma separated list (where the first element is the ratio
+   * (between 0 and 100)) and the second element is the distribution (if
+   * non-existent then uniform will be selected). If an empty list is passed in
+   * then this element will just set the distribution (to uniform) and leave the
+   * percent as null.
+   */
+  OperationData(String data) {
+    String pieces[] = Helper.getTrimmedStrings(data);
+    distribution = Distribution.UNIFORM;
+    percent = null;
+    if (pieces.length == 1) {
+      percent = (Double.parseDouble(pieces[0]) / 100.0d);
+    } else if (pieces.length >= 2) {
+      percent = (Double.parseDouble(pieces[0]) / 100.0d);
+      distribution = Distribution.valueOf(pieces[1].toUpperCase());
+    }
+  }
+
+  /**
+   * Gets the distribution this operation represents
+   * 
+   * @return Distribution
+   */
+  Distribution getDistribution() {
+    return distribution;
+  }
+
+  /**
+   * Gets the 0 - 1 percent that this operations run ratio should be
+   * 
+   * @return Double (or null if not given)
+   */
+  Double getPercent() {
+    return percent;
+  }
+
+  /**
+   * Returns a string list representation of this object (if the percent is
+   * null) then NaN will be output instead. Format is percent,distribution.
+   */
+  public String toString() {
+    StringBuilder str = new StringBuilder();
+    if (getPercent() != null) {
+      str.append(getPercent() * 100.0d);
+    } else {
+      str.append(Double.NaN);
+    }
+    str.append(SEP);
+    str.append(getDistribution().lowerName());
+    return str.toString();
+  }
+}

+ 82 - 0
src/test/org/apache/hadoop/fs/slive/OperationFactory.java

@@ -0,0 +1,82 @@
+/**
+ * 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.slive;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.hadoop.fs.slive.Constants.OperationType;
+
+/**
+ * Factory class which returns instances of operations given there operation
+ * type enumeration (in string or enumeration format).
+ */
+class OperationFactory {
+
+  private Map<OperationType, Operation> typedOperations;
+  private ConfigExtractor config;
+  private Random rnd;
+
+  OperationFactory(ConfigExtractor cfg, Random rnd) {
+    this.typedOperations = new HashMap<OperationType, Operation>();
+    this.config = cfg;
+    this.rnd = rnd;
+  }
+
+  /**
+   * Gets an operation instance (cached) for a given operation type
+   * 
+   * @param type
+   *          the operation type to fetch for
+   * 
+   * @return Operation operation instance or null if it can not be fetched.
+   */
+  Operation getOperation(OperationType type) {
+    Operation op = typedOperations.get(type);
+    if (op != null) {
+      return op;
+    }
+    switch (type) {
+    case READ:
+      op = new ReadOp(this.config, rnd);
+      break;
+    case LS:
+      op = new ListOp(this.config, rnd);
+      break;
+    case MKDIR:
+      op = new MkdirOp(this.config, rnd);
+      break;
+    case APPEND:
+      op = new AppendOp(this.config, rnd);
+      break;
+    case RENAME:
+      op = new RenameOp(this.config, rnd);
+      break;
+    case DELETE:
+      op = new DeleteOp(this.config, rnd);
+      break;
+    case CREATE:
+      op = new CreateOp(this.config, rnd);
+      break;
+    }
+    typedOperations.put(type, op);
+    return op;
+  }
+}

+ 249 - 0
src/test/org/apache/hadoop/fs/slive/OperationOutput.java

@@ -0,0 +1,249 @@
+/**
+ * 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.slive;
+
+import org.apache.hadoop.io.Text;
+
+/**
+ * An operation output has the following object format whereby simple types are
+ * represented as a key of dataType:operationType*measurementType and these
+ * simple types can be combined (mainly in the reducer) using there given types
+ * into a single operation output.
+ * 
+ * Combination is done based on the data types and the following convention is
+ * followed (in the following order). If one is a string then the other will be
+ * concated as a string with a ";" separator. If one is a double then the other
+ * will be added as a double and the output will be a double. If one is a float
+ * then the other will be added as a float and the the output will be a float.
+ * Following this if one is a long the other will be added as a long and the
+ * output type will be a long and if one is a integer the other will be added as
+ * a integer and the output type will be an integer.
+ */
+class OperationOutput {
+
+  private OutputType dataType;
+  private String opType, measurementType;
+  private Object value;
+
+  private static final String TYPE_SEP = ":";
+  private static final String MEASUREMENT_SEP = "*";
+  private static final String STRING_SEP = ";";
+
+  static enum OutputType {
+    STRING, FLOAT, LONG, DOUBLE, INTEGER
+  }
+
+  /**
+   * Parses a given key according to the expected key format and forms the given
+   * segments.
+   * 
+   * @param key
+   *          the key in expected dataType:operationType*measurementType format
+   * @param value
+   *          a generic value expected to match the output type
+   * @throws IllegalArgumentException
+   *           if invalid format
+   */
+  OperationOutput(String key, Object value) {
+    int place = key.indexOf(TYPE_SEP);
+    if (place == -1) {
+      throw new IllegalArgumentException(
+          "Invalid key format - no type seperator - " + TYPE_SEP);
+    }
+    try {
+      dataType = OutputType.valueOf(key.substring(0, place).toUpperCase());
+    } catch (Exception e) {
+      throw new IllegalArgumentException(
+          "Invalid key format - invalid output type", e);
+    }
+    key = key.substring(place + 1);
+    place = key.indexOf(MEASUREMENT_SEP);
+    if (place == -1) {
+      throw new IllegalArgumentException(
+          "Invalid key format - no measurement seperator - " + MEASUREMENT_SEP);
+    }
+    opType = key.substring(0, place);
+    measurementType = key.substring(place + 1);
+    this.value = value;
+  }
+
+  OperationOutput(Text key, Object value) {
+    this(key.toString(), value);
+  }
+
+  public String toString() {
+    return getKeyString() + " (" + this.value + ")";
+  }
+
+  OperationOutput(OutputType dataType, String opType, String measurementType,
+      Object value) {
+    this.dataType = dataType;
+    this.opType = opType;
+    this.measurementType = measurementType;
+    this.value = value;
+  }
+
+  /**
+   * Merges according to the documented rules for merging. Only will merge if
+   * measurement type and operation type is the same.
+   * 
+   * @param o1
+   *          the first object to merge with the second
+   * @param o2
+   *          the second object.
+   * 
+   * @return OperationOutput merged output.
+   * 
+   * @throws IllegalArgumentException
+   *           if unable to merge due to incompatible formats/types
+   */
+  static OperationOutput merge(OperationOutput o1, OperationOutput o2) {
+    if (o1.getMeasurementType().equals(o2.getMeasurementType())
+        && o1.getOperationType().equals(o2.getOperationType())) {
+      Object newvalue = null;
+      OutputType newtype = null;
+      String opType = o1.getOperationType();
+      String mType = o1.getMeasurementType();
+      if (o1.getOutputType() == OutputType.STRING
+          || o2.getOutputType() == OutputType.STRING) {
+        newtype = OutputType.STRING;
+        StringBuilder str = new StringBuilder();
+        str.append(o1.getValue());
+        str.append(STRING_SEP);
+        str.append(o2.getValue());
+        newvalue = str.toString();
+      } else if (o1.getOutputType() == OutputType.DOUBLE
+          || o2.getOutputType() == OutputType.DOUBLE) {
+        newtype = OutputType.DOUBLE;
+        try {
+          newvalue = Double.parseDouble(o1.getValue().toString())
+              + Double.parseDouble(o2.getValue().toString());
+        } catch (NumberFormatException e) {
+          throw new IllegalArgumentException(
+              "Unable to combine a type with a double " + o1 + " & " + o2, e);
+        }
+      } else if (o1.getOutputType() == OutputType.FLOAT
+          || o2.getOutputType() == OutputType.FLOAT) {
+        newtype = OutputType.FLOAT;
+        try {
+          newvalue = Float.parseFloat(o1.getValue().toString())
+              + Float.parseFloat(o2.getValue().toString());
+        } catch (NumberFormatException e) {
+          throw new IllegalArgumentException(
+              "Unable to combine a type with a float " + o1 + " & " + o2, e);
+        }
+      } else if (o1.getOutputType() == OutputType.LONG
+          || o2.getOutputType() == OutputType.LONG) {
+        newtype = OutputType.LONG;
+        try {
+          newvalue = Long.parseLong(o1.getValue().toString())
+              + Long.parseLong(o2.getValue().toString());
+        } catch (NumberFormatException e) {
+          throw new IllegalArgumentException(
+              "Unable to combine a type with a long " + o1 + " & " + o2, e);
+        }
+      } else if (o1.getOutputType() == OutputType.INTEGER
+          || o2.getOutputType() == OutputType.INTEGER) {
+        newtype = OutputType.INTEGER;
+        try {
+          newvalue = Integer.parseInt(o1.getValue().toString())
+              + Integer.parseInt(o2.getValue().toString());
+        } catch (NumberFormatException e) {
+          throw new IllegalArgumentException(
+              "Unable to combine a type with an int " + o1 + " & " + o2, e);
+        }
+      }
+      return new OperationOutput(newtype, opType, mType, newvalue);
+    } else {
+      throw new IllegalArgumentException("Unable to combine dissimilar types "
+          + o1 + " & " + o2);
+    }
+  }
+
+  /**
+   * Formats the key for output
+   * 
+   * @return String
+   */
+  private String getKeyString() {
+    StringBuilder str = new StringBuilder();
+    str.append(getOutputType().name());
+    str.append(TYPE_SEP);
+    str.append(getOperationType());
+    str.append(MEASUREMENT_SEP);
+    str.append(getMeasurementType());
+    return str.toString();
+  }
+
+  /**
+   * Retrieves the key in a hadoop text object
+   * 
+   * @return Text text output
+   */
+  Text getKey() {
+    return new Text(getKeyString());
+  }
+
+  /**
+   * Gets the output value in text format
+   * 
+   * @return Text
+   */
+  Text getOutputValue() {
+    StringBuilder valueStr = new StringBuilder();
+    valueStr.append(getValue());
+    return new Text(valueStr.toString());
+  }
+
+  /**
+   * Gets the object that represents this value (expected to match the output
+   * data type)
+   * 
+   * @return Object
+   */
+  Object getValue() {
+    return value;
+  }
+
+  /**
+   * Gets the output data type of this class.
+   */
+  OutputType getOutputType() {
+    return dataType;
+  }
+
+  /**
+   * Gets the operation type this object represents.
+   * 
+   * @return String
+   */
+  String getOperationType() {
+    return opType;
+  }
+
+  /**
+   * Gets the measurement type this object represents.
+   * 
+   * @return String
+   */
+  String getMeasurementType() {
+    return measurementType;
+  }
+
+}

+ 51 - 0
src/test/org/apache/hadoop/fs/slive/OperationWeight.java

@@ -0,0 +1,51 @@
+/**
+ * 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.slive;
+
+/**
+ * Class which holds an operation and its weight (used in operation selection)
+ */
+class OperationWeight {
+  private double weight;
+  private Operation operation;
+
+  OperationWeight(Operation op, double weight) {
+    this.operation = op;
+    this.weight = weight;
+  }
+
+  /**
+   * Fetches the given operation weight
+   * 
+   * @return Double
+   */
+  double getWeight() {
+    return weight;
+  }
+
+  /**
+   * Gets the operation
+   * 
+   * @return Operation
+   */
+  Operation getOperation() {
+    return operation;
+  }
+
+}

+ 107 - 0
src/test/org/apache/hadoop/fs/slive/PathFinder.java

@@ -0,0 +1,107 @@
+/**
+ * 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.slive;
+
+import java.util.Random;
+
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Class which generates a file or directory path using a simple random
+ * generation algorithm stated in http://issues.apache.org/jira/browse/HDFS-708
+ */
+class PathFinder {
+
+  private static enum Type {
+    FILE, DIRECTORY
+  }
+
+  private static final String DIR_PREFIX = "sl_dir_";
+  private static final String FILE_PREFIX = "sl_file_";
+
+  private Path basePath;
+  private ConfigExtractor config;
+  private Random rnd;
+
+  PathFinder(ConfigExtractor cfg, Random rnd) {
+    this.basePath = cfg.getDataPath();
+    this.config = cfg;
+    this.rnd = rnd;
+  }
+
+  /**
+   * This function uses a simple recursive algorithm to generate a path name
+   * using the current id % limitPerDir and using current id / limitPerDir to
+   * form the rest of the tree segments
+   * 
+   * @param curId
+   *          the current id to use for determining the current directory id %
+   *          per directory limit and then used for determining the next segment
+   *          of the path to use, if <= zero this will return the base path
+   * @param limitPerDir
+   *          the per directory file limit used in modulo and division
+   *          operations to calculate the file name and path tree
+   * @param type
+   *          directory or file enumeration
+   * @return Path
+   */
+  private Path getPath(int curId, int limitPerDir, Type type) {
+    if (curId <= 0) {
+      return basePath;
+    }
+    String name = "";
+    switch (type) {
+    case FILE:
+      name = FILE_PREFIX + new Integer(curId % limitPerDir).toString();
+      break;
+    case DIRECTORY:
+      name = DIR_PREFIX + new Integer(curId % limitPerDir).toString();
+      break;
+    }
+    Path base = getPath((curId / limitPerDir), limitPerDir, Type.DIRECTORY);
+    return new Path(base, name);
+  }
+
+  /**
+   * Gets a file path using the given configuration provided total files and
+   * files per directory
+   * 
+   * @return path
+   */
+  Path getFile() {
+    int fileLimit = config.getTotalFiles();
+    int dirLimit = config.getDirSize();
+    int startPoint = 1 + rnd.nextInt(fileLimit);
+    return getPath(startPoint, dirLimit, Type.FILE);
+  }
+
+  /**
+   * Gets a directory path using the given configuration provided total files
+   * and files per directory
+   * 
+   * @return path
+   */
+  Path getDirectory() {
+    int fileLimit = config.getTotalFiles();
+    int dirLimit = config.getDirSize();
+    int startPoint = rnd.nextInt(fileLimit);
+    return getPath(startPoint, dirLimit, Type.DIRECTORY);
+  }
+
+}

+ 76 - 0
src/test/org/apache/hadoop/fs/slive/Range.java

@@ -0,0 +1,76 @@
+/**
+ * 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.slive;
+
+import java.util.Random;
+
+/**
+ * Class that represents a numeric minimum and a maximum
+ * 
+ * @param <T>
+ *          the type of number being used
+ */
+class Range<T extends Number> {
+
+  private static final String SEP = ",";
+
+  private T min;
+  private T max;
+
+  Range(T min, T max) {
+    this.min = min;
+    this.max = max;
+  }
+
+  /**
+   * @return the minimum value
+   */
+  T getLower() {
+    return min;
+  }
+
+  /**
+   * @return the maximum value
+   */
+  T getUpper() {
+    return max;
+  }
+
+  public String toString() {
+    return min + SEP + max;
+  }
+
+  /**
+   * Gets a long number between two values
+   * 
+   * @param rnd
+   * @param range
+   * 
+   * @return long
+   */
+  static long betweenPositive(Random rnd, Range<Long> range) {
+    if (range.getLower().equals(range.getUpper())) {
+      return range.getLower();
+    }
+    long nextRnd = rnd.nextLong();
+    long normRange = (range.getUpper() - range.getLower() + 1);
+    return Math.abs(nextRnd % normRange) + range.getLower();
+  }
+
+}

+ 141 - 0
src/test/org/apache/hadoop/fs/slive/ReadOp.java

@@ -0,0 +1,141 @@
+/**
+ * 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.slive;
+
+import java.io.DataInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.slive.DataVerifier.VerifyOutput;
+import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
+
+/**
+ * Operation which selects a random file and selects a random read size (from
+ * the read size option) and reads from the start of that file to the read size
+ * (or the full file) and verifies the bytes that were written there.
+ * 
+ * This operation will capture statistics on success the time taken to read that
+ * file and the number of successful readings that occurred as well as the
+ * number of bytes read and the number of chunks verified and the number of
+ * chunks which failed verification and on failure or error it will capture the
+ * number of failures and the amount of time taken to fail
+ */
+class ReadOp extends Operation {
+  private static final Log LOG = LogFactory.getLog(ReadOp.class);
+
+  ReadOp(ConfigExtractor cfg, Random rnd) {
+    super(ReadOp.class.getSimpleName(), cfg, rnd);
+  }
+
+  /**
+   * Gets the file name to read
+   * 
+   * @return Path
+   */
+  protected Path getReadFile() {
+    Path fn = getFinder().getFile();
+    return fn;
+  }
+
+  @Override // Operation
+  List<OperationOutput> run(FileSystem fs) {
+    List<OperationOutput> out = super.run(fs);
+    DataInputStream is = null;
+    try {
+      Path fn = getReadFile();
+      Range<Long> readSizeRange = getConfig().getReadSize();
+      long readSize = 0;
+      String readStrAm = "";
+      if (getConfig().shouldReadFullFile()) {
+        readSize = Long.MAX_VALUE;
+        readStrAm = "full file";
+      } else {
+        readSize = Range.betweenPositive(getRandom(), readSizeRange);
+        readStrAm = Helper.toByteInfo(readSize);
+      }
+      long timeTaken = 0;
+      long chunkSame = 0;
+      long chunkDiff = 0;
+      long bytesRead = 0;
+      long startTime = 0;
+      DataVerifier vf = new DataVerifier();
+      LOG.info("Attempting to read file at " + fn + " of size (" + readStrAm
+          + ")");
+      {
+        // open
+        startTime = Timer.now();
+        is = fs.open(fn);
+        timeTaken += Timer.elapsed(startTime);
+        // read & verify
+        VerifyOutput vo = vf.verifyFile(readSize, is);
+        timeTaken += vo.getReadTime();
+        chunkSame += vo.getChunksSame();
+        chunkDiff += vo.getChunksDifferent();
+        bytesRead += vo.getBytesRead();
+        // capture close time
+        startTime = Timer.now();
+        is.close();
+        is = null;
+        timeTaken += Timer.elapsed(startTime);
+      }
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.OK_TIME_TAKEN, timeTaken));
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.BYTES_READ, bytesRead));
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.SUCCESSES, 1L));
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.CHUNKS_VERIFIED, chunkSame));
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.CHUNKS_UNVERIFIED, chunkDiff));
+      LOG.info("Read " + Helper.toByteInfo(bytesRead) + " of " + fn + " with "
+          + chunkSame + " chunks being same as expected and " + chunkDiff
+          + " chunks being different than expected in " + timeTaken
+          + " milliseconds");
+
+    } catch (FileNotFoundException e) {
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.NOT_FOUND, 1L));
+      LOG.warn("Error with reading", e);
+    } catch (BadFileException e) {
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.BAD_FILES, 1L));
+      LOG.warn("Error reading bad file", e);
+    } catch (IOException e) {
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.FAILURES, 1L));
+      LOG.warn("Error reading", e);
+    } finally {
+      if (is != null) {
+        try {
+          is.close();
+        } catch (IOException e) {
+          LOG.warn("Error closing read stream", e);
+        }
+      }
+    }
+    return out;
+  }
+}

+ 119 - 0
src/test/org/apache/hadoop/fs/slive/RenameOp.java

@@ -0,0 +1,119 @@
+/**
+ * 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.slive;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
+
+/**
+ * Operation which selects a random file and a second random file and attempts
+ * to rename that first file into the second file.
+ * 
+ * This operation will capture statistics on success the time taken to rename
+ * those files and the number of successful renames that occurred and on failure
+ * or error it will capture the number of failures and the amount of time taken
+ * to fail
+ */
+class RenameOp extends Operation {
+
+  /**
+   * Class that holds the src and target for renames
+   */
+  protected static class SrcTarget {
+    private Path src, target;
+
+    SrcTarget(Path src, Path target) {
+      this.src = src;
+      this.target = target;
+    }
+
+    Path getSrc() {
+      return src;
+    }
+
+    Path getTarget() {
+      return target;
+    }
+  }
+
+  private static final Log LOG = LogFactory.getLog(RenameOp.class);
+
+  RenameOp(ConfigExtractor cfg, Random rnd) {
+    super(RenameOp.class.getSimpleName(), cfg, rnd);
+  }
+
+  /**
+   * Gets the file names to rename
+   * 
+   * @return SrcTarget
+   */
+  protected SrcTarget getRenames() {
+    Path src = getFinder().getFile();
+    Path target = getFinder().getFile();
+    return new SrcTarget(src, target);
+  }
+
+  @Override // Operation
+  List<OperationOutput> run(FileSystem fs) {
+    List<OperationOutput> out = super.run(fs);
+    try {
+      // find the files to modify
+      SrcTarget targets = getRenames();
+      Path src = targets.getSrc();
+      Path target = targets.getTarget();
+      // capture results
+      boolean renamedOk = false;
+      long timeTaken = 0;
+      {
+        // rename it
+        long startTime = Timer.now();
+        renamedOk = fs.rename(src, target);
+        timeTaken = Timer.elapsed(startTime);
+      }
+      if (renamedOk) {
+        out.add(new OperationOutput(OutputType.LONG, getType(),
+            ReportWriter.OK_TIME_TAKEN, timeTaken));
+        out.add(new OperationOutput(OutputType.LONG, getType(),
+            ReportWriter.SUCCESSES, 1L));
+        LOG.info("Renamed " + src + " to " + target);
+      } else {
+        out.add(new OperationOutput(OutputType.LONG, getType(),
+            ReportWriter.FAILURES, 1L));
+        LOG.warn("Could not rename " + src + " to " + target);
+      }
+    } catch (FileNotFoundException e) {
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.NOT_FOUND, 1L));
+      LOG.warn("Error with renaming", e);
+    } catch (IOException e) {
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.FAILURES, 1L));
+      LOG.warn("Error with renaming", e);
+    }
+    return out;
+  }
+}

+ 186 - 0
src/test/org/apache/hadoop/fs/slive/ReportWriter.java

@@ -0,0 +1,186 @@
+/**
+ * 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.slive;
+
+import java.io.PrintWriter;
+import java.text.NumberFormat;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Class which provides a report for the given operation output
+ */
+class ReportWriter {
+
+  // simple measurement types
+  // expect long values
+  // these will be reported on + rates by this reporter
+  static final String OK_TIME_TAKEN = "milliseconds_taken";
+  static final String FAILURES = "failures";
+  static final String SUCCESSES = "successes";
+  static final String BYTES_WRITTEN = "bytes_written";
+  static final String FILES_CREATED = "files_created";
+  static final String DIR_ENTRIES = "dir_entries";
+  static final String OP_COUNT = "op_count";
+  static final String CHUNKS_VERIFIED = "chunks_verified";
+  static final String CHUNKS_UNVERIFIED = "chunks_unverified";
+  static final String BYTES_READ = "bytes_read";
+  static final String NOT_FOUND = "files_not_found";
+  static final String BAD_FILES = "bad_files";
+
+  private static final Log LOG = LogFactory.getLog(ReportWriter.class);
+
+  private static final String SECTION_DELIM = "-------------";
+
+  /**
+   * @return String to be used for as a section delimiter
+   */
+  private String getSectionDelimiter() {
+    return SECTION_DELIM;
+  }
+
+  /**
+   * Writes a message the the logging library and the given print writer (if it
+   * is not null)
+   * 
+   * @param msg
+   *          the message to write
+   * @param os
+   *          the print writer if specified to also write to
+   */
+  private void writeMessage(String msg, PrintWriter os) {
+    LOG.info(msg);
+    if (os != null) {
+      os.println(msg);
+    }
+  }
+
+  /**
+   * Provides a simple report showing only the input size, and for each
+   * operation the operation type, measurement type and its values.
+   * 
+   * @param input
+   *          the list of operations to report on
+   * @param os
+   *          any print writer for which output should be written to (along with
+   *          the logging library)
+   */
+  void basicReport(List<OperationOutput> input, PrintWriter os) {
+    writeMessage("Default report for " + input.size() + " operations ", os);
+    writeMessage(getSectionDelimiter(), os);
+    for (OperationOutput data : input) {
+      writeMessage("Operation \"" + data.getOperationType() + "\" measuring \""
+          + data.getMeasurementType() + "\" = " + data.getValue(), os);
+    }
+    writeMessage(getSectionDelimiter(), os);
+  }
+
+  /**
+   * Provides a more detailed report for a given operation. This will output the
+   * keys and values for all input and then sort based on measurement type and
+   * attempt to show rates for various metrics which have expected types to be
+   * able to measure there rate. Currently this will show rates for bytes
+   * written, success count, files created, directory entries, op count and
+   * bytes read if the variable for time taken is available for each measurement
+   * type.
+   * 
+   * @param operation
+   *          the operation that is being reported on.
+   * @param input
+   *          the set of data for that that operation.
+   * @param os
+   *          any print writer for which output should be written to (along with
+   *          the logging library)
+   */
+  void opReport(String operation, List<OperationOutput> input,
+      PrintWriter os) {
+    writeMessage("Basic report for operation type " + operation, os);
+    writeMessage(getSectionDelimiter(), os);
+    for (OperationOutput data : input) {
+      writeMessage("Measurement \"" + data.getMeasurementType() + "\" = "
+          + data.getValue(), os);
+    }
+    // split up into measurement types for rates...
+    Map<String, OperationOutput> combined = new TreeMap<String, OperationOutput>();
+    for (OperationOutput data : input) {
+      if (combined.containsKey(data.getMeasurementType())) {
+        OperationOutput curr = combined.get(data.getMeasurementType());
+        combined.put(data.getMeasurementType(), OperationOutput.merge(curr,
+            data));
+      } else {
+        combined.put(data.getMeasurementType(), data);
+      }
+    }
+    // handle the known types
+    OperationOutput timeTaken = combined.get(OK_TIME_TAKEN);
+    if (timeTaken != null) {
+      Long mTaken = Long.parseLong(timeTaken.getValue().toString());
+      if (mTaken > 0) {
+        NumberFormat formatter = Formatter.getDecimalFormatter();
+        for (String measurementType : combined.keySet()) {
+          Double rate = null;
+          String rateType = "";
+          if (measurementType.equals(BYTES_WRITTEN)) {
+            Long mbWritten = Long.parseLong(combined.get(measurementType)
+                .getValue().toString())
+                / (Constants.MEGABYTES);
+            rate = (double) mbWritten / (double) (mTaken / 1000.0d);
+            rateType = "MB/sec";
+          } else if (measurementType.equals(SUCCESSES)) {
+            Long succ = Long.parseLong(combined.get(measurementType).getValue()
+                .toString());
+            rate = (double) succ / (double) (mTaken / 1000.0d);
+            rateType = "successes/sec";
+          } else if (measurementType.equals(FILES_CREATED)) {
+            Long filesCreated = Long.parseLong(combined.get(measurementType)
+                .getValue().toString());
+            rate = (double) filesCreated / (double) (mTaken / 1000.0d);
+            rateType = "files created/sec";
+          } else if (measurementType.equals(DIR_ENTRIES)) {
+            Long entries = Long.parseLong(combined.get(measurementType)
+                .getValue().toString());
+            rate = (double) entries / (double) (mTaken / 1000.0d);
+            rateType = "directory entries/sec";
+          } else if (measurementType.equals(OP_COUNT)) {
+            Long opCount = Long.parseLong(combined.get(measurementType)
+                .getValue().toString());
+            rate = (double) opCount / (double) (mTaken / 1000.0d);
+            rateType = "operations/sec";
+          } else if (measurementType.equals(BYTES_READ)) {
+            Long mbRead = Long.parseLong(combined.get(measurementType)
+                .getValue().toString())
+                / (Constants.MEGABYTES);
+            rate = (double) mbRead / (double) (mTaken / 1000.0d);
+            rateType = "MB/sec";
+          }
+          if (rate != null) {
+            writeMessage("Rate for measurement \"" + measurementType + "\" = "
+                + formatter.format(rate) + " " + rateType, os);
+          }
+        }
+      }
+    }
+    writeMessage(getSectionDelimiter(), os);
+  }
+
+}

+ 65 - 0
src/test/org/apache/hadoop/fs/slive/RouletteSelector.java

@@ -0,0 +1,65 @@
+/**
+ * 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.slive;
+
+import java.util.List;
+import java.util.Random;
+
+/**
+ * A selection object which simulates a roulette wheel whereby all operation
+ * have a weight and the total value of the wheel is the combined weight and
+ * during selection a random number (0, total weight) is selected and then the
+ * operation that is at that value will be selected. So for a set of operations
+ * with uniform weight they will all have the same probability of being
+ * selected. Operations which choose to have higher weights will have higher
+ * likelihood of being selected (and the same goes for lower weights).
+ */
+class RouletteSelector {
+
+  private Random picker;
+
+  RouletteSelector(Random rnd) {
+    picker = rnd;
+  }
+
+  Operation select(List<OperationWeight> ops) {
+    if (ops.isEmpty()) {
+      return null;
+    }
+    double totalWeight = 0;
+    for (OperationWeight w : ops) {
+      if (w.getWeight() < 0) {
+        throw new IllegalArgumentException("Negative weights not allowed");
+      }
+      totalWeight += w.getWeight();
+    }
+    // roulette wheel selection
+    double sAm = picker.nextDouble() * totalWeight;
+    int index = 0;
+    for (int i = 0; i < ops.size(); ++i) {
+      sAm -= ops.get(i).getWeight();
+      if (sAm <= 0) {
+        index = i;
+        break;
+      }
+    }
+    return ops.get(index).getOperation();
+  }
+
+}

+ 95 - 0
src/test/org/apache/hadoop/fs/slive/SleepOp.java

@@ -0,0 +1,95 @@
+/**
+ * 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.slive;
+
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
+
+/**
+ * Operation which sleeps for a given number of milliseconds according to the
+ * config given, and reports on the sleep time overall
+ */
+class SleepOp extends Operation {
+
+  private static final Log LOG = LogFactory.getLog(SleepOp.class);
+
+  SleepOp(ConfigExtractor cfg, Random rnd) {
+    super(SleepOp.class.getSimpleName(), cfg, rnd);
+  }
+
+  protected long getSleepTime(Range<Long> sleepTime) {
+    long sleepMs = Range.betweenPositive(getRandom(), sleepTime);
+    return sleepMs;
+  }
+
+  /**
+   * Sleep for a random amount of time between a given positive range
+   * 
+   * @param sleepTime
+   *          positive long range for times to choose
+   * 
+   * @return output data on operation
+   */
+  List<OperationOutput> run(Range<Long> sleepTime) {
+    List<OperationOutput> out = super.run(null);
+    try {
+      if (sleepTime != null) {
+        long sleepMs = getSleepTime(sleepTime);
+        long startTime = Timer.now();
+        sleep(sleepMs);
+        long elapsedTime = Timer.elapsed(startTime);
+        out.add(new OperationOutput(OutputType.LONG, getType(),
+            ReportWriter.OK_TIME_TAKEN, elapsedTime));
+        out.add(new OperationOutput(OutputType.LONG, getType(),
+            ReportWriter.SUCCESSES, 1L));
+      }
+    } catch (InterruptedException e) {
+      out.add(new OperationOutput(OutputType.LONG, getType(),
+          ReportWriter.FAILURES, 1L));
+      LOG.warn("Error with sleeping", e);
+    }
+    return out;
+  }
+
+  @Override // Operation
+  List<OperationOutput> run(FileSystem fs) {
+    Range<Long> sleepTime = getConfig().getSleepRange();
+    return run(sleepTime);
+  }
+
+  /**
+   * Sleeps the current thread for X milliseconds
+   * 
+   * @param ms
+   *          milliseconds to sleep for
+   * 
+   * @throws InterruptedException
+   */
+  private void sleep(long ms) throws InterruptedException {
+    if (ms <= 0) {
+      return;
+    }
+    Thread.sleep(ms);
+  }
+}

+ 197 - 0
src/test/org/apache/hadoop/fs/slive/SliveMapper.java

@@ -0,0 +1,197 @@
+/**
+ * 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.slive;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.slive.OperationOutput.OutputType;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * The slive class which sets up the mapper to be used which itself will receive
+ * a single dummy key and value and then in a loop run the various operations
+ * that have been selected and upon operation completion output the collected
+ * output from that operation (and repeat until finished).
+ */
+@SuppressWarnings("deprecation")
+public class SliveMapper extends MapReduceBase implements
+    Mapper<Object, Object, Text, Text> {
+
+  private static final Log LOG = LogFactory.getLog(SliveMapper.class);
+
+  private static final String OP_TYPE = SliveMapper.class.getSimpleName();
+
+  private FileSystem filesystem;
+  private ConfigExtractor config;
+  private int taskId;
+  private String taskIdKey="mapred.task.id";
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see
+   * org.apache.hadoop.mapred.MapReduceBase#configure(org.apache.hadoop.mapred
+   * .JobConf)
+   */
+  @Override // MapReduceBase
+  public void configure(JobConf conf) {
+    try {
+      filesystem = FileSystem.get(conf);
+    } catch (Exception e) {
+      throw new RuntimeException(
+          "Unable to get the filesystem from provided configuration", e);
+    }
+    try {
+      config = new ConfigExtractor(conf);
+      ConfigExtractor.dumpOptions(config);
+    } catch (Exception e) {
+      LOG.error("Unable to setup slive " + StringUtils.stringifyException(e));
+      throw new RuntimeException("Unable to setup slive configuration", e);
+    }
+    this.taskId = TaskAttemptID.forName(conf.get(taskIdKey)).getTaskID()
+        .getId();
+  }
+
+  /**
+   * Fetches the config this object uses
+   * 
+   * @return ConfigExtractor
+   */
+  private ConfigExtractor getConfig() {
+    return config;
+  }
+
+  /**
+   * Logs to the given reporter and logs to the internal logger at info level
+   * 
+   * @param r
+   *          the reporter to set status on
+   * @param msg
+   *          the message to log
+   */
+  private void logAndSetStatus(Reporter r, String msg) {
+    r.setStatus(msg);
+    LOG.info(msg);
+  }
+
+  /**
+   * Runs the given operation and reports on its results
+   * 
+   * @param op
+   *          the operation to run
+   * @param reporter
+   *          the status reporter to notify
+   * @param output
+   *          the output to write to
+   * @throws IOException
+   */
+  private void runOperation(Operation op, Reporter reporter,
+      OutputCollector<Text, Text> output, long opNum) throws IOException {
+    if (op == null) {
+      return;
+    }
+    logAndSetStatus(reporter, "Running operation #" + opNum + " (" + op + ")");
+    List<OperationOutput> opOut = op.run(filesystem);
+    logAndSetStatus(reporter, "Finished operation #" + opNum + " (" + op + ")");
+    if (opOut != null && !opOut.isEmpty()) {
+      for (OperationOutput outData : opOut) {
+        output.collect(outData.getKey(), outData.getOutputValue());
+      }
+    }
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see org.apache.hadoop.mapred.Mapper#map(java.lang.Object,
+   * java.lang.Object, org.apache.hadoop.mapred.OutputCollector,
+   * org.apache.hadoop.mapred.Reporter)
+   */
+  @Override // Mapper
+  public void map(Object key, Object value, OutputCollector<Text, Text> output,
+      Reporter reporter) throws IOException {
+    logAndSetStatus(reporter, "Running slive mapper for dummy key " + key
+        + " and dummy value " + value);
+    //Add taskID to randomSeed to deterministically seed rnd.
+    Random rnd = config.getRandomSeed() != null ?
+      new Random(this.taskId + config.getRandomSeed()) : new Random();
+    WeightSelector selector = new WeightSelector(config, rnd);
+    long startTime = Timer.now();
+    long opAm = 0;
+    long sleepOps = 0;
+    int duration = getConfig().getDurationMilliseconds();
+    Range<Long> sleepRange = getConfig().getSleepRange();
+    Operation sleeper = null;
+    if (sleepRange != null) {
+      sleeper = new SleepOp(getConfig(), rnd);
+    }
+    while (Timer.elapsed(startTime) < duration) {
+      try {
+        logAndSetStatus(reporter, "Attempting to select operation #"
+            + (opAm + 1));
+        int currElapsed = (int) (Timer.elapsed(startTime));
+        Operation op = selector.select(currElapsed, duration);
+        if (op == null) {
+          // no ops left
+          break;
+        } else {
+          // got a good op
+          ++opAm;
+          runOperation(op, reporter, output, opAm);
+        }
+        // do a sleep??
+        if (sleeper != null) {
+          // these don't count against the number of operations
+          ++sleepOps;
+          runOperation(sleeper, reporter, output, sleepOps);
+        }
+      } catch (Exception e) {
+        logAndSetStatus(reporter, "Failed at running due to "
+            + StringUtils.stringifyException(e));
+        if (getConfig().shouldExitOnFirstError()) {
+          break;
+        }
+      }
+    }
+    // write out any accumulated mapper stats
+    {
+      long timeTaken = Timer.elapsed(startTime);
+      OperationOutput opCount = new OperationOutput(OutputType.LONG, OP_TYPE,
+          ReportWriter.OP_COUNT, opAm);
+      output.collect(opCount.getKey(), opCount.getOutputValue());
+      OperationOutput overallTime = new OperationOutput(OutputType.LONG,
+          OP_TYPE, ReportWriter.OK_TIME_TAKEN, timeTaken);
+      output.collect(overallTime.getKey(), overallTime.getOutputValue());
+      logAndSetStatus(reporter, "Finished " + opAm + " operations in "
+          + timeTaken + " milliseconds");
+    }
+  }
+}

+ 39 - 0
src/test/org/apache/hadoop/fs/slive/SlivePartitioner.java

@@ -0,0 +1,39 @@
+/**
+ * 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.slive;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Partitioner;
+
+/**
+ * The partitioner partitions the map output according to the operation type.
+ * The partition number is the hash of the operation type modular the total
+ * number of the reducers.
+ */
+@SuppressWarnings("deprecation")
+public class SlivePartitioner implements Partitioner<Text, Text> {
+  @Override // JobConfigurable
+  public void configure(JobConf conf) {}
+
+  @Override // Partitioner
+  public int getPartition(Text key, Text value, int numPartitions) {
+    OperationOutput oo = new OperationOutput(key, value);
+    return (oo.getOperationType().hashCode() & Integer.MAX_VALUE) % numPartitions;
+  }
+}

+ 124 - 0
src/test/org/apache/hadoop/fs/slive/SliveReducer.java

@@ -0,0 +1,124 @@
+/**
+ * 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.slive;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * The slive reducer which iterates over the given input values and merges them
+ * together into a final output value.
+ */
+@SuppressWarnings("deprecation")
+public class SliveReducer extends MapReduceBase implements
+    Reducer<Text, Text, Text, Text> {
+
+  private static final Log LOG = LogFactory.getLog(SliveReducer.class);
+
+  private ConfigExtractor config;
+
+  /**
+   * Logs to the given reporter and logs to the internal logger at info level
+   * 
+   * @param r
+   *          the reporter to set status on
+   * @param msg
+   *          the message to log
+   */
+  private void logAndSetStatus(Reporter r, String msg) {
+    r.setStatus(msg);
+    LOG.info(msg);
+  }
+
+  /**
+   * Fetches the config this object uses
+   * 
+   * @return ConfigExtractor
+   */
+  private ConfigExtractor getConfig() {
+    return config;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see org.apache.hadoop.mapred.Reducer#reduce(java.lang.Object,
+   * java.util.Iterator, org.apache.hadoop.mapred.OutputCollector,
+   * org.apache.hadoop.mapred.Reporter)
+   */
+  @Override // Reducer
+  public void reduce(Text key, Iterator<Text> values,
+      OutputCollector<Text, Text> output, Reporter reporter) throws IOException {
+    OperationOutput collector = null;
+    int reduceAm = 0;
+    int errorAm = 0;
+    logAndSetStatus(reporter, "Iterating over reduction values for key " + key);
+    while (values.hasNext()) {
+      Text value = values.next();
+      try {
+        OperationOutput val = new OperationOutput(key, value);
+        if (collector == null) {
+          collector = val;
+        } else {
+          collector = OperationOutput.merge(collector, val);
+        }
+        LOG.info("Combined " + val + " into/with " + collector);
+        ++reduceAm;
+      } catch (Exception e) {
+        ++errorAm;
+        logAndSetStatus(reporter, "Error iterating over reduction input "
+            + value + " due to : " + StringUtils.stringifyException(e));
+        if (getConfig().shouldExitOnFirstError()) {
+          break;
+        }
+      }
+    }
+    logAndSetStatus(reporter, "Reduced " + reduceAm + " values with " + errorAm
+        + " errors");
+    if (collector != null) {
+      logAndSetStatus(reporter, "Writing output " + collector.getKey() + " : "
+          + collector.getOutputValue());
+      output.collect(collector.getKey(), collector.getOutputValue());
+    }
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see
+   * org.apache.hadoop.mapred.MapReduceBase#configure(org.apache.hadoop.mapred
+   * .JobConf)
+   */
+  @Override // MapReduceBase
+  public void configure(JobConf conf) {
+    config = new ConfigExtractor(conf);
+    ConfigExtractor.dumpOptions(config);
+  }
+
+}

+ 328 - 0
src/test/org/apache/hadoop/fs/slive/SliveTest.java

@@ -0,0 +1,328 @@
+/**
+ * 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.slive;
+
+import java.io.BufferedReader;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.slive.ArgumentParser.ParsedOutput;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TextOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * Slive test entry point + main program
+ * 
+ * This program will output a help message given -help which can be used to
+ * determine the program options and configuration which will affect the program
+ * runtime. The program will take these options, either from configuration or
+ * command line and process them (and merge) and then establish a job which will
+ * thereafter run a set of mappers & reducers and then the output of the
+ * reduction will be reported on.
+ * 
+ * The number of maps is specified by "slive.maps".
+ * The number of reduces is specified by "slive.reduces".
+ */
+@SuppressWarnings("deprecation")
+public class SliveTest implements Tool {
+
+  private static final Log LOG = LogFactory.getLog(SliveTest.class);
+
+  // ensures the hdfs configurations are loaded if they exist
+  static {
+    Configuration.addDefaultResource("hdfs-default.xml");
+    Configuration.addDefaultResource("hdfs-site.xml");
+  }
+
+  private Configuration base;
+
+  public SliveTest(Configuration base) {
+    this.base = base;
+  }
+
+  public int run(String[] args) {
+    ParsedOutput parsedOpts = null;
+    try {
+      ArgumentParser argHolder = new ArgumentParser(args);
+      parsedOpts = argHolder.parse();
+      if (parsedOpts.shouldOutputHelp()) {
+        parsedOpts.outputHelp();
+        return 1;
+      }
+    } catch (Exception e) {
+      LOG.error("Unable to parse arguments due to error: ", e);
+      return 1;
+    }
+    LOG.info("Running with option list " + Helper.stringifyArray(args, " "));
+    ConfigExtractor config = null;
+    try {
+      ConfigMerger cfgMerger = new ConfigMerger();
+      Configuration cfg = cfgMerger.getMerged(parsedOpts,
+                                              new Configuration(base));
+      if (cfg != null) {
+        config = new ConfigExtractor(cfg);
+      }
+    } catch (Exception e) {
+      LOG.error("Unable to merge config due to error: ", e);
+      return 1;
+    }
+    if (config == null) {
+      LOG.error("Unable to merge config & options!");
+      return 1;
+    }
+    try {
+      LOG.info("Options are:");
+      ConfigExtractor.dumpOptions(config);
+    } catch (Exception e) {
+      LOG.error("Unable to dump options due to error: ", e);
+      return 1;
+    }
+    boolean jobOk = false;
+    try {
+      LOG.info("Running job:");
+      runJob(config);
+      jobOk = true;
+    } catch (Exception e) {
+      LOG.error("Unable to run job due to error: ", e);
+    }
+    if (jobOk) {
+      try {
+        LOG.info("Reporting on job:");
+        writeReport(config);
+      } catch (Exception e) {
+        LOG.error("Unable to report on job due to error: ", e);
+      }
+    }
+    // attempt cleanup (not critical)
+    boolean cleanUp = getBool(parsedOpts
+        .getValue(ConfigOption.CLEANUP.getOpt()));
+    if (cleanUp) {
+      try {
+        LOG.info("Cleaning up job:");
+        cleanup(config);
+      } catch (Exception e) {
+        LOG.error("Unable to cleanup job due to error: ", e);
+      }
+    }
+    // all mostly worked
+    if (jobOk) {
+      return 0;
+    }
+    // maybe didn't work
+    return 1;
+  }
+
+  /**
+   * Checks if a string is a boolean or not and what type
+   * 
+   * @param val
+   *          val to check
+   * @return boolean
+   */
+  private boolean getBool(String val) {
+    if (val == null) {
+      return false;
+    }
+    String cleanupOpt = val.toLowerCase().trim();
+    if (cleanupOpt.equals("true") || cleanupOpt.equals("1")) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Sets up a job conf for the given job using the given config object. Ensures
+   * that the correct input format is set, the mapper and and reducer class and
+   * the input and output keys and value classes along with any other job
+   * configuration.
+   * 
+   * @param config
+   * @return JobConf representing the job to be ran
+   * @throws IOException
+   */
+  private JobConf getJob(ConfigExtractor config) throws IOException {
+    JobConf job = new JobConf(config.getConfig(), SliveTest.class);
+    job.setInputFormat(DummyInputFormat.class);
+    FileOutputFormat.setOutputPath(job, config.getOutputPath());
+    job.setMapperClass(SliveMapper.class);
+    job.setPartitionerClass(SlivePartitioner.class);
+    job.setReducerClass(SliveReducer.class);
+    job.setOutputKeyClass(Text.class);
+    job.setOutputValueClass(Text.class);
+    job.setOutputFormat(TextOutputFormat.class);
+    TextOutputFormat.setCompressOutput(job, false);
+    job.setNumReduceTasks(config.getReducerAmount());
+    job.setNumMapTasks(config.getMapAmount());
+    return job;
+  }
+
+  /**
+   * Runs the job given the provided config
+   * 
+   * @param config
+   *          the config to run the job with
+   * 
+   * @throws IOException
+   *           if can not run the given job
+   */
+  private void runJob(ConfigExtractor config) throws IOException {
+    JobClient.runJob(getJob(config));
+  }
+
+  /**
+   * Attempts to write the report to the given output using the specified
+   * config. It will open up the expected reducer output file and read in its
+   * contents and then split up by operation output and sort by operation type
+   * and then for each operation type it will generate a report to the specified
+   * result file and the console.
+   * 
+   * @param cfg
+   *          the config specifying the files and output
+   * 
+   * @throws Exception
+   *           if files can not be opened/closed/read or invalid format
+   */
+  private void writeReport(ConfigExtractor cfg) throws Exception {
+    Path dn = cfg.getOutputPath();
+    LOG.info("Writing report using contents of " + dn);
+    FileSystem fs = FileSystem.get(cfg.getConfig());
+    FileStatus[] reduceFiles = fs.listStatus(dn);
+    BufferedReader fileReader = null;
+    PrintWriter reportWriter = null;
+    try {
+      List<OperationOutput> noOperations = new ArrayList<OperationOutput>();
+      Map<String, List<OperationOutput>> splitTypes = new TreeMap<String, List<OperationOutput>>();
+      for(FileStatus fn : reduceFiles) {
+        if(!fn.getPath().getName().startsWith("part")) continue;
+        fileReader = new BufferedReader(new InputStreamReader(
+            new DataInputStream(fs.open(fn.getPath()))));
+        String line;
+        while ((line = fileReader.readLine()) != null) {
+          String pieces[] = line.split("\t", 2);
+          if (pieces.length == 2) {
+            OperationOutput data = new OperationOutput(pieces[0], pieces[1]);
+            String op = (data.getOperationType());
+            if (op != null) {
+              List<OperationOutput> opList = splitTypes.get(op);
+              if (opList == null) {
+                opList = new ArrayList<OperationOutput>();
+              }
+              opList.add(data);
+              splitTypes.put(op, opList);
+            } else {
+              noOperations.add(data);
+            }
+          } else {
+            throw new IOException("Unparseable line " + line);
+          }
+        }
+        fileReader.close();
+        fileReader = null;
+      }
+      File resFile = null;
+      if (cfg.getResultFile() != null) {
+        resFile = new File(cfg.getResultFile());
+      }
+      if (resFile != null) {
+        LOG.info("Report results being placed to logging output and to file "
+            + resFile.getCanonicalPath());
+        reportWriter = new PrintWriter(new FileOutputStream(resFile));
+      } else {
+        LOG.info("Report results being placed to logging output");
+      }
+      ReportWriter reporter = new ReportWriter();
+      if (!noOperations.isEmpty()) {
+        reporter.basicReport(noOperations, reportWriter);
+      }
+      for (String opType : splitTypes.keySet()) {
+        reporter.opReport(opType, splitTypes.get(opType), reportWriter);
+      }
+    } finally {
+      if (fileReader != null) {
+        fileReader.close();
+      }
+      if (reportWriter != null) {
+        reportWriter.close();
+      }
+    }
+  }
+
+  /**
+   * Cleans up the base directory by removing it
+   * 
+   * @param cfg
+   *          ConfigExtractor which has location of base directory
+   * 
+   * @throws IOException
+   */
+  private void cleanup(ConfigExtractor cfg) throws IOException {
+    FileSystem fs = FileSystem.get(cfg.getConfig());
+    Path base = cfg.getBaseDirectory();
+    if (base != null) {
+      LOG.info("Attempting to recursively delete " + base);
+      fs.delete(base, true);
+    }
+  }
+
+  /**
+   * The main program entry point. Sets up and parses the command line options,
+   * then merges those options and then dumps those options and the runs the
+   * corresponding map/reduce job that those operations represent and then
+   * writes the report for the output of the run that occurred.
+   * 
+   * @param args
+   *          command line options
+   */
+  public static void main(String[] args) throws Exception {
+    Configuration startCfg = new Configuration(true);
+    SliveTest runner = new SliveTest(startCfg);
+    int ec = ToolRunner.run(runner, args);
+    System.exit(ec);
+  }
+
+  @Override // Configurable
+  public Configuration getConf() {
+    return this.base;
+  }
+
+  @Override // Configurable
+  public void setConf(Configuration conf) {
+    this.base = conf;
+  }
+}

+ 535 - 0
src/test/org/apache/hadoop/fs/slive/TestSlive.java

@@ -0,0 +1,535 @@
+/**
+ * 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.slive;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.slive.ArgumentParser.ParsedOutput;
+import org.apache.hadoop.fs.slive.Constants.OperationType;
+import org.apache.hadoop.fs.slive.DataVerifier.VerifyOutput;
+import org.apache.hadoop.fs.slive.DataWriter.GenerateOutput;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Junit 4 test for slive
+ */
+public class TestSlive {
+
+  private static final Log LOG = LogFactory.getLog(TestSlive.class);
+
+  private static final Random rnd = new Random(1L);
+
+  private static final String TEST_DATA_PROP = "test.build.data";
+
+  private static Configuration getBaseConfig() {
+    Configuration conf = new Configuration();
+    return conf;
+  }
+
+  /** gets the test write location according to the coding guidelines */
+  private static File getWriteLoc() {
+    String writeLoc = System.getProperty(TEST_DATA_PROP, "build/test/data/");
+    return new File(writeLoc, "slive");
+  }
+
+  /** gets where the MR job places its data + output + results */
+  private static File getFlowLocation() {
+    return new File(getWriteLoc(), "flow");
+  }
+
+  /** gets the test directory which is created by the mkdir op */
+  private static File getTestDir() {
+    return new File(getWriteLoc(), "slivedir");
+  }
+
+  /**
+   * gets the test file location
+   * which is used for reading, appending and created
+   */
+  private static File getTestFile() {
+    return new File(getWriteLoc(), "slivefile");
+  }
+
+  /**
+   * gets the rename file which is used in combination
+   * with the test file to do a rename operation
+   */
+  private static File getTestRenameFile() {
+    return new File(getWriteLoc(), "slivefile1");
+  }
+
+  /** gets the MR result file name */
+  private static File getResultFile() {
+    return new File(getWriteLoc(), "sliveresfile");
+  }
+
+  private static File getImaginaryFile() {
+    return new File(getWriteLoc(), "slivenofile");
+  }
+
+  /** gets the test program arguments used for merging and main MR running */
+  private String[] getTestArgs(boolean sleep) {
+    List<String> args = new LinkedList<String>();
+    // setup the options
+    {
+      args.add("-" + ConfigOption.WRITE_SIZE.getOpt());
+      args.add("1M,2M");
+      args.add("-" + ConfigOption.OPS.getOpt());
+      args.add(Constants.OperationType.values().length + "");
+      args.add("-" + ConfigOption.MAPS.getOpt());
+      args.add("2");
+      args.add("-" + ConfigOption.REDUCES.getOpt());
+      args.add("2");
+      args.add("-" + ConfigOption.APPEND_SIZE.getOpt());
+      args.add("1M,2M");
+      args.add("-" + ConfigOption.BLOCK_SIZE.getOpt());
+      args.add("1M,2M");
+      args.add("-" + ConfigOption.REPLICATION_AM.getOpt());
+      args.add("1,1");
+      if (sleep) {
+        args.add("-" + ConfigOption.SLEEP_TIME.getOpt());
+        args.add("10,10");
+      }
+      args.add("-" + ConfigOption.RESULT_FILE.getOpt());
+      args.add(getResultFile().toString());
+      args.add("-" + ConfigOption.BASE_DIR.getOpt());
+      args.add(getFlowLocation().toString());
+      args.add("-" + ConfigOption.DURATION.getOpt());
+      args.add("10");
+      args.add("-" + ConfigOption.DIR_SIZE.getOpt());
+      args.add("10");
+      args.add("-" + ConfigOption.FILES.getOpt());
+      args.add("10");
+    }
+    return args.toArray(new String[args.size()]);
+  }
+
+  @Test
+  public void testFinder() throws Exception {
+    ConfigExtractor extractor = getTestConfig(false);
+    PathFinder fr = new PathFinder(extractor, rnd);
+    // should only be able to select 10 files
+    // attempt for a given amount of iterations
+    int maxIterations = 10000;
+    Set<Path> files = new HashSet<Path>();
+    for (int i = 0; i < maxIterations; i++) {
+      files.add(fr.getFile());
+    }
+    assertTrue(files.size() == 10);
+    Set<Path> dirs = new HashSet<Path>();
+    for (int i = 0; i < maxIterations; i++) {
+      dirs.add(fr.getDirectory());
+    }
+    assertTrue(dirs.size() == 10);
+  }
+
+  @Test
+  public void testSelection() throws Exception {
+    ConfigExtractor extractor = getTestConfig(false);
+    WeightSelector selector = new WeightSelector(extractor, rnd);
+    // should be 1 of each type - uniform
+    int expected = OperationType.values().length;
+    Operation op = null;
+    Set<String> types = new HashSet<String>();
+    FileSystem fs = FileSystem.get(extractor.getConfig());
+    while (true) {
+      op = selector.select(1, 1);
+      if (op == null) {
+        break;
+      }
+      // doesn't matter if they work or not
+      op.run(fs);
+      types.add(op.getType());
+    }
+    assertEquals(types.size(), expected);
+  }
+
+  // gets the config merged with the arguments
+  private ConfigExtractor getTestConfig(boolean sleep) throws Exception {
+    ArgumentParser parser = new ArgumentParser(getTestArgs(sleep));
+    ParsedOutput out = parser.parse();
+    assertTrue(!out.shouldOutputHelp());
+    ConfigMerger merge = new ConfigMerger();
+    Configuration cfg = merge.getMerged(out, getBaseConfig());
+    ConfigExtractor extractor = new ConfigExtractor(cfg);
+    return extractor;
+  }
+
+  @Before
+  public void ensureDeleted() throws Exception {
+    rDelete(getTestFile());
+    rDelete(getTestDir());
+    rDelete(getTestRenameFile());
+    rDelete(getResultFile());
+    rDelete(getFlowLocation());
+    rDelete(getImaginaryFile());
+  }
+
+  /** cleans up a file or directory recursively if need be */
+  private void rDelete(File place) throws Exception {
+    if (place.isFile()) {
+      LOG.info("Deleting file " + place);
+      assertTrue(place.delete());
+    } else if (place.isDirectory()) {
+      deleteDir(place);
+    }
+  }
+
+  /** deletes a dir and its contents */
+  private void deleteDir(File dir) throws Exception {
+    String fns[] = dir.list();
+    // delete contents first
+    for (String afn : fns) {
+      File fn = new File(dir, afn);
+      rDelete(fn);
+    }
+    LOG.info("Deleting directory " + dir);
+    // now delete the dir
+    assertTrue(dir.delete());
+  }
+
+  @Test
+  public void testArguments() throws Exception {
+    ConfigExtractor extractor = getTestConfig(true);
+    assertEquals(extractor.getOpCount().intValue(), Constants.OperationType
+        .values().length);
+    assertEquals(extractor.getMapAmount().intValue(), 2);
+    assertEquals(extractor.getReducerAmount().intValue(), 2);
+    Range<Long> apRange = extractor.getAppendSize();
+    assertEquals(apRange.getLower().intValue(), Constants.MEGABYTES * 1);
+    assertEquals(apRange.getUpper().intValue(), Constants.MEGABYTES * 2);
+    Range<Long> wRange = extractor.getWriteSize();
+    assertEquals(wRange.getLower().intValue(), Constants.MEGABYTES * 1);
+    assertEquals(wRange.getUpper().intValue(), Constants.MEGABYTES * 2);
+    Range<Long> bRange = extractor.getBlockSize();
+    assertEquals(bRange.getLower().intValue(), Constants.MEGABYTES * 1);
+    assertEquals(bRange.getUpper().intValue(), Constants.MEGABYTES * 2);
+    String resfile = extractor.getResultFile();
+    assertEquals(resfile, getResultFile().toString());
+    int durationMs = extractor.getDurationMilliseconds();
+    assertEquals(durationMs, 10 * 1000);
+  }
+
+  @Test
+  public void testDataWriting() throws Exception {
+    long byteAm = 100;
+    File fn = getTestFile();
+    DataWriter writer = new DataWriter(rnd);
+    FileOutputStream fs = new FileOutputStream(fn);
+    GenerateOutput ostat = writer.writeSegment(byteAm, fs);
+    LOG.info(ostat);
+    fs.close();
+    assertTrue(ostat.getBytesWritten() == byteAm);
+    DataVerifier vf = new DataVerifier();
+    FileInputStream fin = new FileInputStream(fn);
+    VerifyOutput vfout = vf.verifyFile(byteAm, new DataInputStream(fin));
+    LOG.info(vfout);
+    fin.close();
+    assertEquals(vfout.getBytesRead(), byteAm);
+    assertTrue(vfout.getChunksDifferent() == 0);
+  }
+
+  @Test
+  public void testRange() {
+    Range<Long> r = new Range<Long>(10L, 20L);
+    assertEquals(r.getLower().longValue(), 10L);
+    assertEquals(r.getUpper().longValue(), 20L);
+  }
+
+  @Test
+  public void testCreateOp() throws Exception {
+    // setup a valid config
+    ConfigExtractor extractor = getTestConfig(false);
+    final Path fn = new Path(getTestFile().getCanonicalPath());
+    CreateOp op = new CreateOp(extractor, rnd) {
+      protected Path getCreateFile() {
+        return fn;
+      }
+    };
+    runOperationOk(extractor, op, true);
+  }
+
+  @Test
+  public void testOpFailures() throws Exception {
+    ConfigExtractor extractor = getTestConfig(false);
+    final Path fn = new Path(getImaginaryFile().getCanonicalPath());
+    ReadOp rop = new ReadOp(extractor, rnd) {
+      protected Path getReadFile() {
+        return fn;
+      }
+    };
+    runOperationBad(extractor, rop);
+
+    DeleteOp dop = new DeleteOp(extractor, rnd) {
+      protected Path getDeleteFile() {
+        return fn;
+      }
+    };
+    runOperationBad(extractor, dop);
+
+    RenameOp reop = new RenameOp(extractor, rnd) {
+      protected SrcTarget getRenames() {
+        return new SrcTarget(fn, fn);
+      }
+    };
+    runOperationBad(extractor, reop);
+
+    AppendOp aop = new AppendOp(extractor, rnd) {
+      protected Path getAppendFile() {
+        return fn;
+      }
+    };
+
+    runOperationBad(extractor, aop);
+  }
+
+  private void runOperationBad(ConfigExtractor cfg, Operation op)
+      throws Exception {
+    FileSystem fs = FileSystem.get(cfg.getConfig());
+    List<OperationOutput> data = op.run(fs);
+    assertTrue(!data.isEmpty());
+    boolean foundFail = false;
+    for (OperationOutput d : data) {
+      if (d.getMeasurementType().equals(ReportWriter.FAILURES)) {
+        foundFail = true;
+      }
+      if (d.getMeasurementType().equals(ReportWriter.NOT_FOUND)) {
+        foundFail = true;
+      }
+    }
+    assertTrue(foundFail);
+  }
+
+  private void runOperationOk(ConfigExtractor cfg, Operation op, boolean checkOk)
+      throws Exception {
+    FileSystem fs = FileSystem.get(cfg.getConfig());
+    List<OperationOutput> data = op.run(fs);
+    assertTrue(!data.isEmpty());
+    if (checkOk) {
+      boolean foundSuc = false;
+      boolean foundOpCount = false;
+      boolean foundTime = false;
+      for (OperationOutput d : data) {
+        assertTrue(!d.getMeasurementType().equals(ReportWriter.FAILURES));
+        if (d.getMeasurementType().equals(ReportWriter.SUCCESSES)) {
+          foundSuc = true;
+        }
+        if (d.getMeasurementType().equals(ReportWriter.OP_COUNT)) {
+          foundOpCount = true;
+        }
+        if (d.getMeasurementType().equals(ReportWriter.OK_TIME_TAKEN)) {
+          foundTime = true;
+        }
+      }
+      assertTrue(foundSuc);
+      assertTrue(foundOpCount);
+      assertTrue(foundTime);
+    }
+  }
+
+  @Test
+  public void testDelete() throws Exception {
+    ConfigExtractor extractor = getTestConfig(false);
+    final Path fn = new Path(getTestFile().getCanonicalPath());
+    // ensure file created before delete
+    CreateOp op = new CreateOp(extractor, rnd) {
+      protected Path getCreateFile() {
+        return fn;
+      }
+    };
+    runOperationOk(extractor, op, true);
+    // now delete
+    DeleteOp dop = new DeleteOp(extractor, rnd) {
+      protected Path getDeleteFile() {
+        return fn;
+      }
+    };
+    runOperationOk(extractor, dop, true);
+  }
+
+  @Test
+  public void testRename() throws Exception {
+    ConfigExtractor extractor = getTestConfig(false);
+    final Path src = new Path(getTestFile().getCanonicalPath());
+    final Path tgt = new Path(getTestRenameFile().getCanonicalPath());
+    // ensure file created before rename
+    CreateOp op = new CreateOp(extractor, rnd) {
+      protected Path getCreateFile() {
+        return src;
+      }
+    };
+    runOperationOk(extractor, op, true);
+    RenameOp rop = new RenameOp(extractor, rnd) {
+      protected SrcTarget getRenames() {
+        return new SrcTarget(src, tgt);
+      }
+    };
+    runOperationOk(extractor, rop, true);
+  }
+
+  @Test
+  public void testMRFlow() throws Exception {
+    ConfigExtractor extractor = getTestConfig(false);
+    SliveTest s = new SliveTest(getBaseConfig());
+    int ec = ToolRunner.run(s, getTestArgs(false));
+    assertTrue(ec == 0);
+    String resFile = extractor.getResultFile();
+    File fn = new File(resFile);
+    assertTrue(fn.exists());
+    // can't validate completely since operations may fail (mainly anyone but
+    // create +mkdir) since they may not find there files
+  }
+
+  @Test
+  public void testRead() throws Exception {
+    ConfigExtractor extractor = getTestConfig(false);
+    final Path fn = new Path(getTestFile().getCanonicalPath());
+    // ensure file created before read
+    CreateOp op = new CreateOp(extractor, rnd) {
+      protected Path getCreateFile() {
+        return fn;
+      }
+    };
+    runOperationOk(extractor, op, true);
+    ReadOp rop = new ReadOp(extractor, rnd) {
+      protected Path getReadFile() {
+        return fn;
+      }
+    };
+    runOperationOk(extractor, rop, true);
+  }
+
+  @Test
+  public void testSleep() throws Exception {
+    ConfigExtractor extractor = getTestConfig(true);
+    SleepOp op = new SleepOp(extractor, rnd);
+    runOperationOk(extractor, op, true);
+  }
+
+  @Test
+  public void testList() throws Exception {
+    // ensure dir made
+    ConfigExtractor extractor = getTestConfig(false);
+    final Path dir = new Path(getTestDir().getCanonicalPath());
+    MkdirOp op = new MkdirOp(extractor, rnd) {
+      protected Path getDirectory() {
+        return dir;
+      }
+    };
+    runOperationOk(extractor, op, true);
+    // list it
+    ListOp lop = new ListOp(extractor, rnd) {
+      protected Path getDirectory() {
+        return dir;
+      }
+    };
+    runOperationOk(extractor, lop, true);
+  }
+
+  @Test
+  public void testBadChunks() throws Exception {
+    File fn = getTestFile();
+    int byteAm = 10000;
+    FileOutputStream fout = new FileOutputStream(fn);
+    byte[] bytes = new byte[byteAm];
+    rnd.nextBytes(bytes);
+    fout.write(bytes);
+    fout.close();
+    // attempt to read it
+    DataVerifier vf = new DataVerifier();
+    VerifyOutput vout = new VerifyOutput(0, 0, 0, 0);
+    DataInputStream in = null;
+    try {
+      in = new DataInputStream(new FileInputStream(fn));
+      vout = vf.verifyFile(byteAm, in);
+    } catch (Exception e) {
+
+    } finally {
+      if(in != null) in.close();
+    }
+    assertTrue(vout.getChunksSame() == 0);
+  }
+
+  @Test
+  public void testMkdir() throws Exception {
+    ConfigExtractor extractor = getTestConfig(false);
+    final Path dir = new Path(getTestDir().getCanonicalPath());
+    MkdirOp op = new MkdirOp(extractor, rnd) {
+      protected Path getDirectory() {
+        return dir;
+      }
+    };
+    runOperationOk(extractor, op, true);
+  }
+
+  @Test
+  public void testSelector() throws Exception {
+    ConfigExtractor extractor = getTestConfig(false);
+    RouletteSelector selector = new RouletteSelector(rnd);
+    List<OperationWeight> sList = new LinkedList<OperationWeight>();
+    Operation op = selector.select(sList);
+    assertTrue(op == null);
+    CreateOp cop = new CreateOp(extractor, rnd);
+    sList.add(new OperationWeight(cop, 1.0d));
+    AppendOp aop = new AppendOp(extractor, rnd);
+    sList.add(new OperationWeight(aop, 0.01d));
+    op = selector.select(sList);
+    assertTrue(op == cop);
+  }
+
+  @Test
+  public void testAppendOp() throws Exception {
+    // setup a valid config
+    ConfigExtractor extractor = getTestConfig(false);
+    // ensure file created before append
+    final Path fn = new Path(getTestFile().getCanonicalPath());
+    CreateOp op = new CreateOp(extractor, rnd) {
+      protected Path getCreateFile() {
+        return fn;
+      }
+    };
+    runOperationOk(extractor, op, true);
+    // local file system (ChecksumFileSystem) currently doesn't support append -
+    // but we'll leave this test here anyways but can't check the results..
+    AppendOp aop = new AppendOp(extractor, rnd) {
+      protected Path getAppendFile() {
+        return fn;
+      }
+    };
+    runOperationOk(extractor, aop, false);
+  }
+}

+ 55 - 0
src/test/org/apache/hadoop/fs/slive/Timer.java

@@ -0,0 +1,55 @@
+/**
+ * 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.slive;
+
+/**
+ * Simple timer class that abstracts time access
+ */
+class Timer {
+
+  // no construction allowed
+  private Timer() {
+
+  }
+
+  /**
+   * The current time in milliseconds
+   * 
+   * @return long (milliseconds)
+   */
+  static long now() {
+    return System.currentTimeMillis();
+  }
+
+  /**
+   * Calculates how much time in milliseconds elapsed from given start time to
+   * the current time in milliseconds
+   * 
+   * @param startTime
+   * @return elapsed time (milliseconds)
+   */
+  static long elapsed(long startTime) {
+    long elapsedTime = now() - startTime;
+    if (elapsedTime < 0) {
+      elapsedTime = 0;
+    }
+    return elapsedTime;
+  }
+
+}

+ 195 - 0
src/test/org/apache/hadoop/fs/slive/WeightSelector.java

@@ -0,0 +1,195 @@
+/**
+ * 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.slive;
+
+import java.text.NumberFormat;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.TreeMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.slive.Constants.Distribution;
+import org.apache.hadoop.fs.slive.Constants.OperationType;
+import org.apache.hadoop.fs.slive.Weights.UniformWeight;
+import org.apache.hadoop.fs.slive.ObserveableOp.Observer;
+
+/**
+ * This class is the main handler that selects operations to run using the
+ * currently held selection object. It configures and weights each operation and
+ * then hands the operations + weights off to the selector object to determine
+ * which one should be ran. If no operations are left to be ran then it will
+ * return null.
+ */
+class WeightSelector {
+
+  // what a weight calculation means
+  interface Weightable {
+    Double weight(int elapsed, int duration);
+  }
+
+  private static final Log LOG = LogFactory.getLog(WeightSelector.class);
+
+  private static class OperationInfo {
+    Integer amountLeft;
+    Operation operation;
+    Distribution distribution;
+  }
+
+  private Map<OperationType, OperationInfo> operations;
+  private Map<Distribution, Weightable> weights;
+  private RouletteSelector selector;
+  private OperationFactory factory;
+
+  WeightSelector(ConfigExtractor cfg, Random rnd) {
+    selector = new RouletteSelector(rnd);
+    factory = new OperationFactory(cfg, rnd);
+    configureOperations(cfg);
+    configureWeights(cfg);
+  }
+
+  protected RouletteSelector getSelector() {
+    return selector;
+  }
+
+  private void configureWeights(ConfigExtractor e) {
+    weights = new HashMap<Distribution, Weightable>();
+    weights.put(Distribution.UNIFORM, new UniformWeight());
+    // weights.put(Distribution.BEG, new BeginWeight());
+    // weights.put(Distribution.END, new EndWeight());
+    // weights.put(Distribution.MID, new MidWeight());
+  }
+
+  /**
+   * Determines how many initial operations a given operation data should have
+   * 
+   * @param totalAm
+   *          the total amount of operations allowed
+   * 
+   * @param opData
+   *          the given operation information (with a valid percentage >= 0)
+   * 
+   * @return the number of items to allow to run
+   * 
+   * @throws IllegalArgumentException
+   *           if negative operations are determined
+   */
+  static int determineHowMany(int totalAm, OperationData opData,
+      OperationType type) {
+    if (totalAm <= 0) {
+      return 0;
+    }
+    int amLeft = (int) Math.floor(opData.getPercent() * totalAm);
+    if (amLeft < 0) {
+      throw new IllegalArgumentException("Invalid amount " + amLeft
+          + " determined for operation type " + type.name());
+    }
+    return amLeft;
+  }
+
+  /**
+   * Sets up the operation using the given configuration by setting up the
+   * number of operations to perform (and how many are left) and setting up the
+   * operation objects to be used throughout selection.
+   * 
+   * @param cfg
+   *          ConfigExtractor.
+   */
+  private void configureOperations(ConfigExtractor cfg) {
+    operations = new TreeMap<OperationType, OperationInfo>();
+    Map<OperationType, OperationData> opinfo = cfg.getOperations();
+    int totalAm = cfg.getOpCount();
+    int opsLeft = totalAm;
+    NumberFormat formatter = Formatter.getPercentFormatter();
+    for (final OperationType type : opinfo.keySet()) {
+      OperationData opData = opinfo.get(type);
+      OperationInfo info = new OperationInfo();
+      info.distribution = opData.getDistribution();
+      int amLeft = determineHowMany(totalAm, opData, type);
+      opsLeft -= amLeft;
+      LOG
+          .info(type.name() + " has " + amLeft + " initial operations out of "
+              + totalAm + " for its ratio "
+              + formatter.format(opData.getPercent()));
+      info.amountLeft = amLeft;
+      Operation op = factory.getOperation(type);
+      // wrap operation in finalizer so that amount left gets decrements when
+      // its done
+      if (op != null) {
+        Observer fn = new Observer() {
+          public void notifyFinished(Operation op) {
+            OperationInfo opInfo = operations.get(type);
+            if (opInfo != null) {
+              --opInfo.amountLeft;
+            }
+          }
+
+          public void notifyStarting(Operation op) {
+          }
+        };
+        info.operation = new ObserveableOp(op, fn);
+        operations.put(type, info);
+      }
+    }
+    if (opsLeft > 0) {
+      LOG
+          .info(opsLeft
+              + " left over operations found (due to inability to support partial operations)");
+    }
+  }
+
+  /**
+   * Selects an operation from the known operation set or returns null if none
+   * are available by applying the weighting algorithms and then handing off the
+   * weight operations to the selection object.
+   * 
+   * @param elapsed
+   *          the currently elapsed time (milliseconds) of the running program
+   * @param duration
+   *          the maximum amount of milliseconds of the running program
+   * 
+   * @return operation or null if none left
+   */
+  Operation select(int elapsed, int duration) {
+    List<OperationWeight> validOps = new ArrayList<OperationWeight>(operations
+        .size());
+    for (OperationType type : operations.keySet()) {
+      OperationInfo opinfo = operations.get(type);
+      if (opinfo == null || opinfo.amountLeft <= 0) {
+        continue;
+      }
+      Weightable weighter = weights.get(opinfo.distribution);
+      if (weighter != null) {
+        OperationWeight weightOp = new OperationWeight(opinfo.operation,
+            weighter.weight(elapsed, duration));
+        validOps.add(weightOp);
+      } else {
+        throw new RuntimeException("Unable to get weight for distribution "
+            + opinfo.distribution);
+      }
+    }
+    if (validOps.isEmpty()) {
+      return null;
+    }
+    return getSelector().select(validOps);
+  }
+}

+ 126 - 0
src/test/org/apache/hadoop/fs/slive/Weights.java

@@ -0,0 +1,126 @@
+/**
+ * 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.slive;
+
+import org.apache.hadoop.fs.slive.WeightSelector.Weightable;
+
+/**
+ * Class to isolate the various weight algorithms we use.
+ */
+class Weights {
+  private Weights() {
+
+  }
+
+  /**
+   * A weight which always returns the same weight (1/3). Which will have an
+   * overall area of (1/3) unless otherwise provided.
+   */
+  static class UniformWeight implements Weightable {
+
+    private static Double DEFAULT_WEIGHT = (1.0d / 3.0d);
+
+    private Double weight;
+
+    UniformWeight(double w) {
+      weight = w;
+    }
+
+    UniformWeight() {
+      this(DEFAULT_WEIGHT);
+    }
+
+    @Override // Weightable
+    public Double weight(int elapsed, int duration) {
+      return weight;
+    }
+
+  }
+
+  /**
+   * A weight which normalized the elapsed time and the duration to a value
+   * between 0 and 1 and applies the algorithm to form an output using the
+   * function (-2 * (x-0.5)^2) + 0.5 which initially (close to 0) has a value
+   * close to 0 and near input being 1 has a value close to 0 and near 0.5 has a
+   * value close to 0.5 (with overall area 0.3).
+   */
+  static class MidWeight implements Weightable {
+
+    @Override // Weightable
+    public Double weight(int elapsed, int duration) {
+      double normalized = (double) elapsed / (double) duration;
+      double result = (-2.0d * Math.pow(normalized - 0.5, 2)) + 0.5d;
+      if (result < 0) {
+        result = 0;
+      }
+      if (result > 1) {
+        result = 1;
+      }
+      return result;
+    }
+
+  }
+
+  /**
+   * A weight which normalized the elapsed time and the duration to a value
+   * between 0 and 1 and applies the algorithm to form an output using the
+   * function (x)^2 which initially (close to 0) has a value close to 0 and near
+   * input being 1 has a value close to 1 (with overall area 1/3).
+   */
+  static class EndWeight implements Weightable {
+
+    @Override // Weightable
+    public Double weight(int elapsed, int duration) {
+      double normalized = (double) elapsed / (double) duration;
+      double result = Math.pow(normalized, 2);
+      if (result < 0) {
+        result = 0;
+      }
+      if (result > 1) {
+        result = 1;
+      }
+      return result;
+    }
+
+  }
+
+  /**
+   * A weight which normalized the elapsed time and the duration to a value
+   * between 0 and 1 and applies the algorithm to form an output using the
+   * function (x-1)^2 which initially (close to 0) has a value close to 1 and
+   * near input being 1 has a value close to 0 (with overall area 1/3).
+   */
+  static class BeginWeight implements Weightable {
+
+    @Override // Weightable
+    public Double weight(int elapsed, int duration) {
+      double normalized = (double) elapsed / (double) duration;
+      double result = Math.pow((normalized - 1), 2);
+      if (result < 0) {
+        result = 0;
+      }
+      if (result > 1) {
+        result = 1;
+      }
+      return result;
+    }
+
+  }
+
+}