You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by je...@apache.org on 2010/07/15 19:53:52 UTC

svn commit: r964507 [2/3] - in /mahout/trunk: core/src/main/java/org/apache/mahout/clustering/canopy/ core/src/main/java/org/apache/mahout/clustering/dirichlet/ core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/ core/src/main/java/org/apache/...

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyDriver.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyDriver.java?rev=964507&r1=964506&r2=964507&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyDriver.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyDriver.java Thu Jul 15 17:53:51 2010
@@ -18,13 +18,10 @@
 package org.apache.mahout.clustering.meanshift;
 
 import java.io.IOException;
+import java.util.Map;
 
-import org.apache.commons.cli2.CommandLine;
-import org.apache.commons.cli2.Group;
-import org.apache.commons.cli2.Option;
-import org.apache.commons.cli2.OptionException;
-import org.apache.commons.cli2.builder.GroupBuilder;
-import org.apache.commons.cli2.commandline.Parser;
+import org.apache.commons.cli2.builder.ArgumentBuilder;
+import org.apache.commons.cli2.builder.DefaultOptionBuilder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -37,13 +34,17 @@ import org.apache.hadoop.mapreduce.lib.o
 import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
 import org.apache.mahout.clustering.Cluster;
 import org.apache.mahout.clustering.WeightedVectorWritable;
-import org.apache.mahout.common.CommandLineUtil;
+import org.apache.mahout.common.AbstractJob;
 import org.apache.mahout.common.HadoopUtil;
 import org.apache.mahout.common.commandline.DefaultOptionCreator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public final class MeanShiftCanopyDriver {
+public class MeanShiftCanopyDriver extends AbstractJob {
+
+  protected static final String INPUT_IS_CANOPIES_OPTION = "inputIsCanopies";
+
+  protected static final String INPUT_IS_CANOPIES_OPTION_KEY = "--" + INPUT_IS_CANOPIES_OPTION;
 
   private static final Logger log = LoggerFactory.getLogger(MeanShiftCanopyDriver.class);
 
@@ -51,52 +52,158 @@ public final class MeanShiftCanopyDriver
 
   private static final String CONTROL_CONVERGED = "control/converged";
 
-  private MeanShiftCanopyDriver() {
+  protected MeanShiftCanopyDriver() {
   }
 
-  public static void main(String[] args) throws IOException, InterruptedException, ClassNotFoundException {
-    Option inputOpt = DefaultOptionCreator.inputOption().create();
-    Option outputOpt = DefaultOptionCreator.outputOption().create();
-    Option convergenceDeltaOpt = DefaultOptionCreator.convergenceOption().create();
-    Option helpOpt = DefaultOptionCreator.helpOption();
-    Option maxIterOpt = DefaultOptionCreator.maxIterationsOption().create();
-    Option overwriteOutput = DefaultOptionCreator.overwriteOption().create();
-    Option inputIsCanopiesOpt = DefaultOptionCreator.inputIsCanopiesOption().create();
-    Option measureClassOpt = DefaultOptionCreator.distanceMeasureOption().create();
-    Option threshold1Opt = DefaultOptionCreator.t1Option().create();
-    Option threshold2Opt = DefaultOptionCreator.t2Option().create();
-    Option clusteringOpt = DefaultOptionCreator.clusteringOption().create();
-
-    Group group = new GroupBuilder().withName("Options").withOption(inputOpt).withOption(outputOpt)
-        .withOption(overwriteOutput).withOption(measureClassOpt).withOption(helpOpt)
-        .withOption(convergenceDeltaOpt).withOption(threshold1Opt).withOption(threshold2Opt)
-        .withOption(clusteringOpt).withOption(maxIterOpt).withOption(inputIsCanopiesOpt).create();
-
-    try {
-      Parser parser = new Parser();
-      parser.setGroup(group);
-      parser.setHelpOption(helpOpt);
-      CommandLine cmdLine = parser.parse(args);
-      if (cmdLine.hasOption(helpOpt)) {
-        CommandLineUtil.printHelp(group);
-        return;
-      }
-
-      Path input = new Path(cmdLine.getValue(inputOpt).toString());
-      Path output = new Path(cmdLine.getValue(outputOpt).toString());
-      String measureClass = cmdLine.getValue(measureClassOpt).toString();
-      if (cmdLine.hasOption(overwriteOutput)) {
-        HadoopUtil.overwriteOutput(output);
-      }
-      double t1 = Double.parseDouble(cmdLine.getValue(threshold1Opt).toString());
-      double t2 = Double.parseDouble(cmdLine.getValue(threshold2Opt).toString());
-      double convergenceDelta = Double.parseDouble(cmdLine.getValue(convergenceDeltaOpt).toString());
-      int maxIterations = Integer.parseInt(cmdLine.getValue(maxIterOpt).toString());
-      runJob(input, output, measureClass, t1, t2, convergenceDelta, maxIterations,
-             cmdLine.hasOption(inputIsCanopiesOpt), cmdLine.hasOption(clusteringOpt));
-    } catch (OptionException e) {
-      log.error("Exception parsing command line: ", e);
-      CommandLineUtil.printHelp(group);
+  public static void main(String[] args) throws Exception {
+    new MeanShiftCanopyDriver().run(args);
+  }
+
+  /**
+   * Run the job where the input format can be either Vectors or Canopies
+   * 
+   * @param input
+   *          the input pathname String
+   * @param output
+   *          the output pathname String
+   * @param measureClassName
+   *          the DistanceMeasure class name
+   * @param t1
+   *          the T1 distance threshold
+   * @param t2
+   *          the T2 distance threshold
+   * @param convergenceDelta
+   *          the double convergence criteria
+   * @param maxIterations
+   *          an int number of iterations
+   * @param inputIsCanopies 
+              true if the input path already contains MeanShiftCanopies and does not need to be converted from Vectors
+   * @param runClustering 
+   *          true if the input points are to be clustered once the iterations complete
+   * @throws ClassNotFoundException 
+   * @throws InterruptedException 
+   */
+  public static void runJob(Path input,
+                            Path output,
+                            String measureClassName,
+                            double t1,
+                            double t2,
+                            double convergenceDelta,
+                            int maxIterations,
+                            boolean inputIsCanopies,
+                            boolean runClustering) throws IOException, InterruptedException, ClassNotFoundException {
+    new MeanShiftCanopyDriver().job(input,
+                                    output,
+                                    measureClassName,
+                                    t1,
+                                    t2,
+                                    convergenceDelta,
+                                    maxIterations,
+                                    inputIsCanopies,
+                                    runClustering);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.util.Tool#run(java.lang.String[])
+   */
+  @Override
+  public int run(String[] args) throws Exception {
+    addInputOption();
+    addOutputOption();
+    addOption(DefaultOptionCreator.convergenceOption().create());
+    addOption(DefaultOptionCreator.maxIterationsOption().create());
+    addOption(DefaultOptionCreator.overwriteOption().create());
+    addOption(new DefaultOptionBuilder().withLongName(INPUT_IS_CANOPIES_OPTION).withRequired(false).withShortName("ic")
+        .withArgument(new ArgumentBuilder().withName(INPUT_IS_CANOPIES_OPTION).withMinimum(1).withMaximum(1).create())
+        .withDescription("If present, the input directory already contains MeanShiftCanopies").create());
+    addOption(DefaultOptionCreator.distanceMeasureOption().create());
+    addOption(DefaultOptionCreator.t1Option().create());
+    addOption(DefaultOptionCreator.t2Option().create());
+    addOption(DefaultOptionCreator.clusteringOption().create());
+
+    Map<String, String> argMap = parseArguments(args);
+    if (argMap == null) {
+      return -1;
+    }
+
+    Path input = getInputPath();
+    Path output = getOutputPath();
+    if (argMap.containsKey(DefaultOptionCreator.OVERWRITE_OPTION_KEY)) {
+      HadoopUtil.overwriteOutput(output);
+    }
+    String measureClass = argMap.get(DefaultOptionCreator.DISTANCE_MEASURE_OPTION_KEY);
+    double t1 = Double.parseDouble(argMap.get(DefaultOptionCreator.T1_OPTION_KEY));
+    double t2 = Double.parseDouble(argMap.get(DefaultOptionCreator.T2_OPTION_KEY));
+    boolean runClustering = argMap.containsKey(DefaultOptionCreator.CLUSTERING_OPTION_KEY);
+    double convergenceDelta = Double.parseDouble(argMap.get(DefaultOptionCreator.CONVERGENCE_DELTA_OPTION_KEY));
+    int maxIterations = Integer.parseInt(argMap.get(DefaultOptionCreator.MAX_ITERATIONS_OPTION_KEY));
+    boolean inputIsCanopies = argMap.containsKey(INPUT_IS_CANOPIES_OPTION_KEY);
+
+    job(input, output, measureClass, t1, t2, convergenceDelta, maxIterations, inputIsCanopies, runClustering);
+    return 0;
+  }
+
+  /**
+   * Run the job where the input format can be either Vectors or Canopies
+   * 
+   * @param input
+   *          the input pathname String
+   * @param output
+   *          the output pathname String
+   * @param measureClassName
+   *          the DistanceMeasure class name
+   * @param t1
+   *          the T1 distance threshold
+   * @param t2
+   *          the T2 distance threshold
+   * @param convergenceDelta
+   *          the double convergence criteria
+   * @param maxIterations
+   *          an int number of iterations
+   * @param inputIsCanopies 
+              true if the input path already contains MeanShiftCanopies and does not need to be converted from Vectors
+   * @param runClustering 
+   *          true if the input points are to be clustered once the iterations complete
+   * @throws IOException
+   * @throws InterruptedException
+   * @throws ClassNotFoundException
+   */
+  private void job(Path input,
+                   Path output,
+                   String measureClassName,
+                   double t1,
+                   double t2,
+                   double convergenceDelta,
+                   int maxIterations,
+                   boolean inputIsCanopies,
+                   boolean runClustering) throws IOException, InterruptedException, ClassNotFoundException {
+    Path clustersIn = new Path(output, Cluster.INITIAL_CLUSTERS_DIR);
+    if (inputIsCanopies) {
+      clustersIn = input;
+    } else {
+      createCanopyFromVectors(input, clustersIn);
+    }
+
+    // iterate until the clusters converge
+    boolean converged = false;
+    int iteration = 1;
+    while (!converged && (iteration <= maxIterations)) {
+      log.info("Iteration {}", iteration);
+      // point the output to a new directory per iteration
+      Path clustersOut = new Path(output, Cluster.CLUSTERS_DIR + iteration);
+      Path controlOut = new Path(output, CONTROL_CONVERGED);
+      runIteration(clustersIn, clustersOut, controlOut, measureClassName, t1, t2, convergenceDelta);
+      converged = FileSystem.get(new Configuration()).exists(controlOut);
+      // now point the input to the old output directory
+      clustersIn = clustersOut;
+      iteration++;
+    }
+
+    if (runClustering) {
+      // now cluster the points
+      runClustering(inputIsCanopies ? input : new Path(output, Cluster.INITIAL_CLUSTERS_DIR),
+                    clustersIn,
+                    new Path(output, Cluster.CLUSTERED_POINTS_DIR));
     }
   }
 
@@ -121,8 +228,13 @@ public final class MeanShiftCanopyDriver
    * @throws ClassNotFoundException 
    * @throws InterruptedException 
    */
-  static void runIteration(Path input, Path output, Path control, String measureClassName, double t1, double t2,
-      double convergenceDelta) throws IOException, InterruptedException, ClassNotFoundException {
+  private void runIteration(Path input,
+                            Path output,
+                            Path control,
+                            String measureClassName,
+                            double t1,
+                            double t2,
+                            double convergenceDelta) throws IOException, InterruptedException, ClassNotFoundException {
 
     Configuration conf = new Configuration();
     conf.set(MeanShiftCanopyConfigKeys.DISTANCE_MEASURE_KEY, measureClassName);
@@ -148,8 +260,7 @@ public final class MeanShiftCanopyDriver
     job.waitForCompletion(true);
   }
 
-  static void createCanopyFromVectors(Path input, Path output)
-    throws IOException, InterruptedException, ClassNotFoundException {
+  private void createCanopyFromVectors(Path input, Path output) throws IOException, InterruptedException, ClassNotFoundException {
     Configuration conf = new Configuration();
     Job job = new Job(conf);
     job.setOutputKeyClass(Text.class);
@@ -178,8 +289,8 @@ public final class MeanShiftCanopyDriver
    * @throws InterruptedException 
    * @throws IOException 
    */
-  static void runClustering(Path input, Path clustersIn, Path output)
-    throws IOException, InterruptedException, ClassNotFoundException {
+  private void runClustering(Path input, Path clustersIn, Path output) throws IOException, InterruptedException,
+      ClassNotFoundException {
 
     Configuration conf = new Configuration();
     conf.set(STATE_IN_KEY, clustersIn.toString());
@@ -199,67 +310,4 @@ public final class MeanShiftCanopyDriver
 
     job.waitForCompletion(true);
   }
-
-  /**
-   * Run the job where the input format can be either Vectors or Canopies
-   * 
-   * @param input
-   *          the input pathname String
-   * @param output
-   *          the output pathname String
-   * @param measureClassName
-   *          the DistanceMeasure class name
-   * @param t1
-   *          the T1 distance threshold
-   * @param t2
-   *          the T2 distance threshold
-   * @param convergenceDelta
-   *          the double convergence criteria
-   * @param maxIterations
-   *          an int number of iterations
-   * @param inputIsCanopies 
-              true if the input path already contains MeanShiftCanopies and does not need to be converted from Vectors
-   * @param runClustering 
-   *          true if the input points are to be clustered once the iterations complete
-   * @throws ClassNotFoundException 
-   * @throws InterruptedException 
-   */
-  public static void runJob(Path input,
-                            Path output,
-                            String measureClassName,
-                            double t1,
-                            double t2,
-                            double convergenceDelta,
-                            int maxIterations,
-                            boolean inputIsCanopies,
-                            boolean runClustering)
-    throws IOException, InterruptedException, ClassNotFoundException {
-    Path clustersIn = new Path(output, Cluster.INITIAL_CLUSTERS_DIR);
-    if (inputIsCanopies) {
-      clustersIn = input;
-    } else {
-      createCanopyFromVectors(input, clustersIn);
-    }
-
-    // iterate until the clusters converge
-    boolean converged = false;
-    int iteration = 1;
-    while (!converged && (iteration <= maxIterations)) {
-      log.info("Iteration {}", iteration);
-      // point the output to a new directory per iteration
-      Path clustersOut = new Path(output, Cluster.CLUSTERS_DIR + iteration);
-      Path controlOut = new Path(output, CONTROL_CONVERGED);
-      runIteration(clustersIn, clustersOut, controlOut, measureClassName, t1, t2, convergenceDelta);
-      converged = FileSystem.get(new Configuration()).exists(controlOut);
-      // now point the input to the old output directory
-      clustersIn = clustersOut;
-      iteration++;
-    }
-
-    if (runClustering) {
-      // now cluster the points
-      runClustering(inputIsCanopies ? input : new Path(output, Cluster.INITIAL_CLUSTERS_DIR), clustersIn, new Path(output,
-          Cluster.CLUSTERED_POINTS_DIR));
-    }
-  }
 }

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/common/commandline/DefaultOptionCreator.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/common/commandline/DefaultOptionCreator.java?rev=964507&r1=964506&r2=964507&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/common/commandline/DefaultOptionCreator.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/common/commandline/DefaultOptionCreator.java Thu Jul 15 17:53:51 2010
@@ -20,12 +20,68 @@ package org.apache.mahout.common.command
 import org.apache.commons.cli2.Option;
 import org.apache.commons.cli2.builder.ArgumentBuilder;
 import org.apache.commons.cli2.builder.DefaultOptionBuilder;
-import org.apache.mahout.clustering.dirichlet.models.NormalModelDistribution;
 import org.apache.mahout.common.distance.SquaredEuclideanDistanceMeasure;
-import org.apache.mahout.math.RandomAccessSparseVector;
 
 public final class DefaultOptionCreator {
 
+  // private option keywords
+  private static final String THRESHOLD_OPTION = "threshold";
+
+  private static final String EMIT_MOST_LIKELY_OPTION = "emitMostLikely";
+
+  private static final String CLUSTERING_OPTION = "clustering";
+
+  private static final String MAX_REDUCERS_OPTION = "maxRed";
+
+  private static final String CONVERGENCE_DELTA_OPTION = "convergenceDelta";
+
+  private static final String NUM_CLUSTERS_OPTION = "numClusters";
+
+  private static final String MAX_ITERATIONS_OPTION = "maxIter";
+
+  private static final String T2_OPTION = "t2";
+
+  private static final String T1_OPTION = "t1";
+
+  private static final String DISTANCE_MEASURE_OPTION = "distanceMeasure";
+
+  private static final String OVERWRITE_OPTION = "overwrite";
+
+  private static final String OUTPUT_OPTION = "output";
+
+  private static final String CLUSTERS_IN_OPTION = "clusters";
+
+  private static final String INPUT_OPTION = "input";
+
+  // public keys used to access parsed arguments map
+  public static final String T2_OPTION_KEY = "--" + T2_OPTION;
+
+  public static final String T1_OPTION_KEY = "--" + T1_OPTION;
+
+  public static final String DISTANCE_MEASURE_OPTION_KEY = "--" + DISTANCE_MEASURE_OPTION;
+
+  public static final String OVERWRITE_OPTION_KEY = "--" + OVERWRITE_OPTION;
+
+  public static final String CLUSTERING_OPTION_KEY = "--" + CLUSTERING_OPTION;
+
+  public static final String NUM_CLUSTERS_OPTION_KEY = "--" + NUM_CLUSTERS_OPTION;
+
+  public static final String MAX_REDUCERS_OPTION_KEY = "--" + MAX_REDUCERS_OPTION;
+
+  public static final String MAX_ITERATIONS_OPTION_KEY = "--" + MAX_ITERATIONS_OPTION;
+
+  public static final String EMIT_MOST_LIKELY_OPTION_KEY = "--" + EMIT_MOST_LIKELY_OPTION;
+
+  public static final String THRESHOLD_OPTION_KEY = "--" + THRESHOLD_OPTION;
+
+  public static final String CLUSTERS_IN_OPTION_KEY = "--" + CLUSTERS_IN_OPTION;
+
+  public static final String CONVERGENCE_DELTA_OPTION_KEY = "--" + CONVERGENCE_DELTA_OPTION;
+
+  public static final String INPUT_OPTION_KEY = "--" + INPUT_OPTION;
+
+  public static final String OUTPUT_OPTION_KEY = "--" + OUTPUT_OPTION;
+
   private DefaultOptionCreator() {
   }
 
@@ -33,26 +89,25 @@ public final class DefaultOptionCreator 
    * Returns a default command line option for help. Used by all clustering jobs and many others
    * */
   public static Option helpOption() {
-    return new DefaultOptionBuilder().withLongName("help").withDescription("Print out help")
-        .withShortName("h").create();
+    return new DefaultOptionBuilder().withLongName("help").withDescription("Print out help").withShortName("h").create();
   }
 
   /**
    * Returns a default command line option for input directory specification. Used by all clustering jobs plus others
    */
   public static DefaultOptionBuilder inputOption() {
-    return new DefaultOptionBuilder().withLongName("input").withRequired(false).withShortName("i").withArgument(
-        new ArgumentBuilder().withName("input").withMinimum(1).withMaximum(1).create()).withDescription(
-        "Path to job input directory.");
+    return new DefaultOptionBuilder().withLongName(INPUT_OPTION).withRequired(false).withShortName("i")
+        .withArgument(new ArgumentBuilder().withName(INPUT_OPTION).withMinimum(1).withMaximum(1).create())
+        .withDescription("Path to job input directory.");
   }
 
   /**
    * Returns a default command line option for clusters input directory specification. Used by FuzzyKmeans, Kmeans
    */
   public static DefaultOptionBuilder clustersInOption() {
-    return new DefaultOptionBuilder().withLongName("clusters").withRequired(true).withArgument(
-        new ArgumentBuilder().withName("clusters").withMinimum(1).withMaximum(1).create()).withDescription(
-        "The path to the initial clusters directory. Must be a SequenceFile of some type of Cluster")
+    return new DefaultOptionBuilder().withLongName(CLUSTERS_IN_OPTION).withRequired(true).withArgument(new ArgumentBuilder()
+        .withName(CLUSTERS_IN_OPTION).withMinimum(1).withMaximum(1).create())
+        .withDescription("The path to the initial clusters directory. Must be a SequenceFile of some type of Cluster")
         .withShortName("c");
   }
 
@@ -60,17 +115,17 @@ public final class DefaultOptionCreator 
    * Returns a default command line option for output directory specification. Used by all clustering jobs plus others
    */
   public static DefaultOptionBuilder outputOption() {
-    return new DefaultOptionBuilder().withLongName("output").withRequired(false).withShortName("o").withArgument(
-        new ArgumentBuilder().withName("output").withMinimum(1).withMaximum(1).create()).withDescription(
-        "The directory pathname for output.");
+    return new DefaultOptionBuilder().withLongName(OUTPUT_OPTION).withRequired(false).withShortName("o")
+        .withArgument(new ArgumentBuilder().withName(OUTPUT_OPTION).withMinimum(1).withMaximum(1).create())
+        .withDescription("The directory pathname for output.");
   }
 
   /**
    * Returns a default command line option for output directory overwriting. Used by all clustering jobs
    */
   public static DefaultOptionBuilder overwriteOption() {
-    return new DefaultOptionBuilder().withLongName("overwrite").withRequired(false).withDescription(
-        "If present, overwrite the output directory before running job").withShortName("ow");
+    return new DefaultOptionBuilder().withLongName(OVERWRITE_OPTION).withRequired(false)
+        .withDescription("If present, overwrite the output directory before running job").withShortName("ow");
   }
 
   /**
@@ -78,10 +133,9 @@ public final class DefaultOptionCreator 
    * Used by Canopy, FuzzyKmeans, Kmeans, MeanShift
    */
   public static DefaultOptionBuilder distanceMeasureOption() {
-    return new DefaultOptionBuilder().withLongName("distanceMeasure").withRequired(false).withShortName("dm")
-        .withArgument(new ArgumentBuilder().withName("distanceMeasure")
-            .withDefault(SquaredEuclideanDistanceMeasure.class.getName())
-            .withMinimum(1).withMaximum(1).create())
+    return new DefaultOptionBuilder().withLongName(DISTANCE_MEASURE_OPTION).withRequired(false).withShortName("dm")
+        .withArgument(new ArgumentBuilder().withName(DISTANCE_MEASURE_OPTION).withDefault(SquaredEuclideanDistanceMeasure.class
+            .getName()).withMinimum(1).withMaximum(1).create())
         .withDescription("The classname of the DistanceMeasure. Default is SquaredEuclidean");
   }
 
@@ -89,19 +143,16 @@ public final class DefaultOptionCreator 
    * Returns a default command line option for specification of T1. Used by Canopy, MeanShift
    */
   public static DefaultOptionBuilder t1Option() {
-    return new DefaultOptionBuilder().withLongName("t1").withRequired(true).withArgument(
-        new ArgumentBuilder().withName("t1").withMinimum(1).withMaximum(1).create())
-        .withDescription("T1 threshold value").withShortName("t1");
+    return new DefaultOptionBuilder().withLongName(T1_OPTION).withRequired(true).withArgument(new ArgumentBuilder()
+        .withName(T1_OPTION).withMinimum(1).withMaximum(1).create()).withDescription("T1 threshold value").withShortName(T1_OPTION);
   }
 
   /**
    * Returns a default command line option for specification of T2. Used by Canopy, MeanShift
    */
   public static DefaultOptionBuilder t2Option() {
-    return new DefaultOptionBuilder().withLongName("t2").withRequired(true).withArgument(
-        new ArgumentBuilder().withName("t2").withMinimum(1).withMaximum(1).create())
-        .withDescription("T2 threshold value")
-        .withShortName("t2");
+    return new DefaultOptionBuilder().withLongName(T2_OPTION).withRequired(true).withArgument(new ArgumentBuilder()
+        .withName(T2_OPTION).withMinimum(1).withMaximum(1).create()).withDescription("T2 threshold value").withShortName(T2_OPTION);
   }
 
   /**
@@ -110,19 +161,19 @@ public final class DefaultOptionCreator 
    */
   public static DefaultOptionBuilder maxIterationsOption() {
     // default value used by LDA which overrides withRequired(false)
-    return new DefaultOptionBuilder().withLongName("maxIter").withRequired(true).withShortName("x").withArgument(
-        new ArgumentBuilder().withName("maxIter").withDefault("-1").withMinimum(1).withMaximum(1).create())
-        .withDescription("The maximum number of iterations.");
+    return new DefaultOptionBuilder().withLongName(MAX_ITERATIONS_OPTION).withRequired(true).withShortName("x")
+        .withArgument(new ArgumentBuilder().withName(MAX_ITERATIONS_OPTION).withDefault("-1").withMinimum(1).withMaximum(1)
+            .create()).withDescription("The maximum number of iterations.");
   }
 
   /**
    * Returns a default command line option for specification of numbers of clusters to create.
    * Used by Dirichlet, FuzzyKmeans, Kmeans
    */
-  public static DefaultOptionBuilder kOption() {
-    return new DefaultOptionBuilder().withLongName("k").withRequired(false).withArgument(
-        new ArgumentBuilder().withName("k").withMinimum(1).withMaximum(1).create()).withDescription(
-        "The number of clusters to create").withShortName("k");
+  public static DefaultOptionBuilder numClustersOption() {
+    return new DefaultOptionBuilder().withLongName(NUM_CLUSTERS_OPTION).withRequired(false).withArgument(new ArgumentBuilder()
+        .withName("k").withMinimum(1).withMaximum(1).create()).withDescription("The number of clusters to create")
+        .withShortName("k");
   }
 
   /**
@@ -130,49 +181,9 @@ public final class DefaultOptionCreator 
    * Used by FuzzyKmeans, Kmeans, MeanShift
    */
   public static DefaultOptionBuilder convergenceOption() {
-    return new DefaultOptionBuilder().withLongName("convergenceDelta").withRequired(false)
-        .withShortName("cd").withArgument(new ArgumentBuilder().withName("convergenceDelta")
-            .withDefault("0.5").withMinimum(1).withMaximum(1).create())
-        .withDescription("The convergence delta value. Default is 0.5");
-  }
-
-  /**
-   * Returns a default command line option for alpha specification. Used by Dirichlet
-   */
-  public static DefaultOptionBuilder alphaOption() {
-    return new DefaultOptionBuilder().withLongName("alpha").withRequired(false).withShortName("m").withArgument(
-        new ArgumentBuilder().withName("alpha").withDefault("1.0").withMinimum(1).withMaximum(1).create())
-        .withDescription("The alpha0 value for the DirichletDistribution. Defaults to 1.0");
-  }
-
-  /**
-   * Returns a default command line option for model distribution class specification. Used by Dirichlet
-   */
-  public static DefaultOptionBuilder modelDistributionOption() {
-    return new DefaultOptionBuilder().withLongName("modelDistClass").withRequired(false).withShortName("md")
-        .withArgument(new ArgumentBuilder().withName("modelDistClass")
-            .withDefault(NormalModelDistribution.class.getName()).withMinimum(1).withMaximum(1).create())
-        .withDescription("The ModelDistribution class name. " + "Defaults to NormalModelDistribution");
-  }
-
-  /**
-   * Returns a default command line option for model prototype class specification. Used by Dirichlet
-   */
-  public static DefaultOptionBuilder modelPrototypeOption() {
-    return new DefaultOptionBuilder().withLongName("modelPrototypeClass").withRequired(false).withShortName("mp")
-        .withArgument(new ArgumentBuilder().withName("prototypeClass")
-            .withDefault(RandomAccessSparseVector.class.getName()).withMinimum(1)
-            .withMaximum(1).create()).withDescription(
-            "The ModelDistribution prototype Vector class name. Defaults to RandomAccessSparseVector");
-  }
-
-  /**
-   * Returns a default command line option for specifying the number of Mappers. Used by FuzzyKmeans
-   */
-  public static DefaultOptionBuilder numMappersOption() {
-    return new DefaultOptionBuilder().withLongName("numMap").withRequired(false).withArgument(
-        new ArgumentBuilder().withName("numMap").withDefault("10").withMinimum(1).withMaximum(1).create())
-        .withDescription("The number of map tasks. Defaults to 10").withShortName("u");
+    return new DefaultOptionBuilder().withLongName(CONVERGENCE_DELTA_OPTION).withRequired(false).withShortName("cd")
+        .withArgument(new ArgumentBuilder().withName(CONVERGENCE_DELTA_OPTION).withDefault("0.5").withMinimum(1).withMaximum(1)
+            .create()).withDescription("The convergence delta value. Default is 0.5");
   }
 
   /**
@@ -180,8 +191,8 @@ public final class DefaultOptionCreator 
    * Used by Dirichlet, FuzzyKmeans, Kmeans and LDA
    */
   public static DefaultOptionBuilder numReducersOption() {
-    return new DefaultOptionBuilder().withLongName("maxRed").withRequired(false).withShortName("r").withArgument(
-        new ArgumentBuilder().withName("maxRed").withDefault("2").withMinimum(1).withMaximum(1).create())
+    return new DefaultOptionBuilder().withLongName(MAX_REDUCERS_OPTION).withRequired(false).withShortName("r")
+        .withArgument(new ArgumentBuilder().withName(MAX_REDUCERS_OPTION).withDefault("2").withMinimum(1).withMaximum(1).create())
         .withDescription("The number of reduce tasks. Defaults to 2");
   }
 
@@ -189,19 +200,18 @@ public final class DefaultOptionCreator 
    * Returns a default command line option for clustering specification. Used by all clustering except LDA
    */
   public static DefaultOptionBuilder clusteringOption() {
-    return new DefaultOptionBuilder().withLongName("clustering").withRequired(false).withDescription(
-        "If present, run clustering after the iterations have taken place").withShortName("cl");
+    return new DefaultOptionBuilder().withLongName(CLUSTERING_OPTION).withRequired(false)
+        .withDescription("If present, run clustering after the iterations have taken place").withShortName("cl");
   }
 
   /**
    * Returns a default command line option for specifying the emitMostLikely flag. Used by Dirichlet and FuzzyKmeans
    */
   public static DefaultOptionBuilder emitMostLikelyOption() {
-    return new DefaultOptionBuilder().withLongName("emitMostLikely").withRequired(false).withShortName("e")
-        .withArgument(new ArgumentBuilder().withName("emitMostLikely").withDefault("true")
-            .withMinimum(1).withMaximum(1).create()).withDescription(
-            "True if clustering should emit the most likely point only, "
-                + "false for threshold clustering. Default is true");
+    return new DefaultOptionBuilder().withLongName(EMIT_MOST_LIKELY_OPTION).withRequired(false).withShortName("e")
+        .withArgument(new ArgumentBuilder().withName(EMIT_MOST_LIKELY_OPTION).withDefault("true").withMinimum(1).withMaximum(1)
+            .create()).withDescription("True if clustering should emit the most likely point only, "
+            + "false for threshold clustering. Default is true");
   }
 
   /**
@@ -209,59 +219,9 @@ public final class DefaultOptionCreator 
    * Used by Dirichlet and FuzzyKmeans
    */
   public static DefaultOptionBuilder thresholdOption() {
-    return new DefaultOptionBuilder().withLongName("threshold").withRequired(false).withShortName("t")
-        .withArgument(new ArgumentBuilder().withName("threshold").withDefault("0")
-            .withMinimum(1).withMaximum(1).create())
+    return new DefaultOptionBuilder().withLongName(THRESHOLD_OPTION).withRequired(false).withShortName("t")
+        .withArgument(new ArgumentBuilder().withName(THRESHOLD_OPTION).withDefault("0").withMinimum(1).withMaximum(1).create())
         .withDescription("The pdf threshold used for cluster determination. Default is 0");
   }
 
-  /**
-   * Returns a default command line option for specifying the FuzzyKMeans coefficient normalization factor, 'm'
-   */
-  public static DefaultOptionBuilder mOption() {
-    return new DefaultOptionBuilder().withLongName("m").withRequired(true).withArgument(
-        new ArgumentBuilder().withName("m").withMinimum(1).withMaximum(1).create()).withDescription(
-        "coefficient normalization factor, must be greater than 1").withShortName("m");
-  }
-
-  /**
-   * Returns a default command line option for specifying that the MeanShift input directory already
-   * contains Canopies vs. Vectors
-   */
-  public static DefaultOptionBuilder inputIsCanopiesOption() {
-    return new DefaultOptionBuilder().withLongName("inputIsCanopies").withRequired(false)
-        .withShortName("ic").withArgument(new ArgumentBuilder().withName("inputIsCanopies")
-            .withMinimum(1).withMaximum(1).create()).withDescription(
-        "If present, the input directory already contains MeanShiftCanopies");
-  }
-
-  /**
-   * Returns a default command line option for specifying the LDA number of topics option
-   */
-  public static DefaultOptionBuilder numTopicsOption() {
-    return new DefaultOptionBuilder().withLongName("numTopics").withRequired(true).withArgument(
-        new ArgumentBuilder().withName("numTopics").withMinimum(1).withMaximum(1).create()).withDescription(
-        "The total number of topics in the corpus").withShortName("k");
-  }
-
-  /**
-   * Returns a default command line option for specifying the LDA number of words option
-   */
-  public static DefaultOptionBuilder numWordsOption() {
-    return new DefaultOptionBuilder().withLongName("numWords").withRequired(true).withArgument(
-        new ArgumentBuilder().withName("numWords").withMinimum(1).withMaximum(1).create()).withDescription(
-        "The total number of words in the corpus (can be approximate, needs to exceed the actual value)")
-        .withShortName("v");
-  }
-
-  /**
-   * Returns a default command line option for specifying the LDA topic smoothing option
-   */
-  public static DefaultOptionBuilder topicSmoothingOption() {
-    return new DefaultOptionBuilder().withLongName("topicSmoothing").withRequired(false).withArgument(
-        new ArgumentBuilder().withName("topicSmoothing").withDefault(-1.0)
-            .withMinimum(0).withMaximum(1).create()).withDescription(
-        "Topic smoothing parameter. Default is 50/numTopics.").withShortName("a");
-  }
-
 }

Modified: mahout/trunk/core/src/test/java/org/apache/mahout/clustering/canopy/TestCanopyCreation.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/canopy/TestCanopyCreation.java?rev=964507&r1=964506&r2=964507&view=diff
==============================================================================
--- mahout/trunk/core/src/test/java/org/apache/mahout/clustering/canopy/TestCanopyCreation.java (original)
+++ mahout/trunk/core/src/test/java/org/apache/mahout/clustering/canopy/TestCanopyCreation.java Thu Jul 15 17:53:51 2010
@@ -37,6 +37,7 @@ import org.apache.mahout.clustering.Clus
 import org.apache.mahout.clustering.WeightedVectorWritable;
 import org.apache.mahout.common.DummyRecordWriter;
 import org.apache.mahout.common.MahoutTestCase;
+import org.apache.mahout.common.commandline.DefaultOptionCreator;
 import org.apache.mahout.common.distance.DistanceMeasure;
 import org.apache.mahout.common.distance.EuclideanDistanceMeasure;
 import org.apache.mahout.common.distance.ManhattanDistanceMeasure;
@@ -510,9 +511,13 @@ public class TestCanopyCreation extends 
     Configuration conf = new Configuration();
     ClusteringTestUtils.writePointsToFile(points, getTestTempFilePath("testdata/file1"), fs, conf);
     ClusteringTestUtils.writePointsToFile(points, getTestTempFilePath("testdata/file2"), fs, conf);
-    // now run the Job
+    // now run the Job using the run() command. Others can use runJob().
     Path output = getTestTempDirPath("output");
-    CanopyDriver.runJob(getTestTempDirPath("testdata"), output, EuclideanDistanceMeasure.class.getName(), 3.1, 2.1, true);
+    String[] args = { DefaultOptionCreator.INPUT_OPTION_KEY, getTestTempDirPath("testdata").toString(),
+        DefaultOptionCreator.OUTPUT_OPTION_KEY, output.toString(), DefaultOptionCreator.DISTANCE_MEASURE_OPTION_KEY,
+        EuclideanDistanceMeasure.class.getName(), DefaultOptionCreator.T1_OPTION_KEY, "3.1", DefaultOptionCreator.T2_OPTION_KEY,
+        "2.1", DefaultOptionCreator.CLUSTERING_OPTION_KEY, DefaultOptionCreator.OVERWRITE_OPTION_KEY  };
+    new CanopyDriver().run(args);
     Path path = new Path(output, "clusteredPoints/part-m-00000");
     SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, conf);
     int count = 0;

Modified: mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestMapReduce.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestMapReduce.java?rev=964507&r1=964506&r2=964507&view=diff
==============================================================================
--- mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestMapReduce.java (original)
+++ mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestMapReduce.java Thu Jul 15 17:53:51 2010
@@ -40,6 +40,7 @@ import org.apache.mahout.clustering.diri
 import org.apache.mahout.common.DummyRecordWriter;
 import org.apache.mahout.common.MahoutTestCase;
 import org.apache.mahout.common.RandomUtils;
+import org.apache.mahout.common.commandline.DefaultOptionCreator;
 import org.apache.mahout.math.DenseVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.math.VectorWritable;
@@ -227,19 +228,16 @@ public class TestMapReduce extends Mahou
     generateSamples(100, 0, 2, 0.3);
     generateSamples(100, 2, 2, 1);
     ClusteringTestUtils.writePointsToFile(sampleData, getTestTempFilePath("input/data.txt"), fs, conf);
-    // Now run the driver
-    int maxIterations = 5;
-    DirichletDriver.runJob(getTestTempDirPath("input"),
-                           getTestTempDirPath("output"),
-                           "org.apache.mahout.clustering.dirichlet.models.SampledNormalDistribution",
-                           "org.apache.mahout.math.DenseVector",
-                           20,
-                           maxIterations,
-                           1.0,
-                           1,
-                           false,
-                           true,
-                           0);
+    // Now run the driver using the run() method. Others can use runJob() as before
+    Integer maxIterations = 5;
+    String[] args = { DefaultOptionCreator.INPUT_OPTION_KEY, getTestTempDirPath("input").toString(),
+        DefaultOptionCreator.OUTPUT_OPTION_KEY, getTestTempDirPath("output").toString(),
+        DirichletDriver.MODEL_DISTRIBUTION_CLASS_OPTION_KEY,
+        "org.apache.mahout.clustering.dirichlet.models.SampledNormalDistribution",
+        DirichletDriver.MODEL_PROTOTYPE_CLASS_OPTION_KEY, "org.apache.mahout.math.DenseVector",
+        DefaultOptionCreator.NUM_CLUSTERS_OPTION_KEY, "20", DefaultOptionCreator.MAX_ITERATIONS_OPTION_KEY,
+        maxIterations.toString(), DirichletDriver.ALPHA_OPTION_KEY, "1.0", DefaultOptionCreator.OVERWRITE_OPTION_KEY };
+    new DirichletDriver().run(args);
     // and inspect results
     List<List<DirichletCluster<VectorWritable>>> clusters = new ArrayList<List<DirichletCluster<VectorWritable>>>();
     Configuration conf = new Configuration();

Modified: mahout/trunk/core/src/test/java/org/apache/mahout/clustering/fuzzykmeans/TestFuzzyKmeansClustering.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/fuzzykmeans/TestFuzzyKmeansClustering.java?rev=964507&r1=964506&r2=964507&view=diff
==============================================================================
--- mahout/trunk/core/src/test/java/org/apache/mahout/clustering/fuzzykmeans/TestFuzzyKmeansClustering.java (original)
+++ mahout/trunk/core/src/test/java/org/apache/mahout/clustering/fuzzykmeans/TestFuzzyKmeansClustering.java Thu Jul 15 17:53:51 2010
@@ -37,6 +37,7 @@ import org.apache.mahout.clustering.Weig
 import org.apache.mahout.clustering.kmeans.TestKmeansClustering;
 import org.apache.mahout.common.DummyRecordWriter;
 import org.apache.mahout.common.MahoutTestCase;
+import org.apache.mahout.common.commandline.DefaultOptionCreator;
 import org.apache.mahout.common.distance.DistanceMeasure;
 import org.apache.mahout.common.distance.EuclideanDistanceMeasure;
 import org.apache.mahout.math.Vector;
@@ -208,21 +209,31 @@ public class TestFuzzyKmeansClustering e
       }
       writer.close();
 
-      // now run the Job
+      // now run the Job using the run() command line options.
       Path output = getTestTempDirPath("output");
-      FuzzyKMeansDriver.runJob(pointsPath,
+/*      FuzzyKMeansDriver.runJob(pointsPath,
                                clustersPath,
                                output,
                                EuclideanDistanceMeasure.class.getName(),
                                0.001,
                                2,
-                               1,
                                k + 1,
                                2,
                                false,
                                true,
                                0);
-
+*/
+      String[] args = { DefaultOptionCreator.INPUT_OPTION_KEY, pointsPath.toString(), 
+          DefaultOptionCreator.CLUSTERS_IN_OPTION_KEY, clustersPath.toString(), 
+          DefaultOptionCreator.OUTPUT_OPTION_KEY, output.toString(),
+          DefaultOptionCreator.DISTANCE_MEASURE_OPTION_KEY, EuclideanDistanceMeasure.class.getName(),
+          DefaultOptionCreator.CONVERGENCE_DELTA_OPTION_KEY, "0.001", 
+          DefaultOptionCreator.MAX_ITERATIONS_OPTION_KEY, "2",
+          FuzzyKMeansDriver.M_OPTION_KEY, "2.0", 
+          DefaultOptionCreator.CLUSTERING_OPTION_KEY,
+          DefaultOptionCreator.EMIT_MOST_LIKELY_OPTION_KEY,
+          DefaultOptionCreator.OVERWRITE_OPTION_KEY };
+      new FuzzyKMeansDriver().run(args);
       SequenceFile.Reader reader = new SequenceFile.Reader(fs, new Path(output, "clusteredPoints/part-m-00000"), conf);
       IntWritable key = new IntWritable();
       WeightedVectorWritable out = new WeightedVectorWritable();

Modified: mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java?rev=964507&r1=964506&r2=964507&view=diff
==============================================================================
--- mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java (original)
+++ mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java Thu Jul 15 17:53:51 2010
@@ -38,6 +38,7 @@ import org.apache.mahout.clustering.cano
 import org.apache.mahout.common.DummyOutputCollector;
 import org.apache.mahout.common.DummyRecordWriter;
 import org.apache.mahout.common.MahoutTestCase;
+import org.apache.mahout.common.commandline.DefaultOptionCreator;
 import org.apache.mahout.common.distance.DistanceMeasure;
 import org.apache.mahout.common.distance.EuclideanDistanceMeasure;
 import org.apache.mahout.common.distance.ManhattanDistanceMeasure;
@@ -376,7 +377,14 @@ public class TestKmeansClustering extend
       writer.close();
       // now run the Job
       Path outputPath = getTestTempDirPath("output");
-      KMeansDriver.runJob(pointsPath, clustersPath, outputPath, EuclideanDistanceMeasure.class.getName(), 0.001, 10, k + 1, true);
+      //KMeansDriver.runJob(pointsPath, clustersPath, outputPath, EuclideanDistanceMeasure.class.getName(), 0.001, 10, k + 1, true);
+      String[] args = { DefaultOptionCreator.INPUT_OPTION_KEY, pointsPath.toString(), DefaultOptionCreator.CLUSTERS_IN_OPTION_KEY,
+          clustersPath.toString(), DefaultOptionCreator.OUTPUT_OPTION_KEY, outputPath.toString(),
+          DefaultOptionCreator.DISTANCE_MEASURE_OPTION_KEY, EuclideanDistanceMeasure.class.getName(),
+          DefaultOptionCreator.CONVERGENCE_DELTA_OPTION_KEY, "0.001", DefaultOptionCreator.MAX_ITERATIONS_OPTION_KEY, "2",
+          DefaultOptionCreator.CLUSTERING_OPTION_KEY, DefaultOptionCreator.OVERWRITE_OPTION_KEY };
+      new KMeansDriver().run(args);
+
       // now compare the expected clusters with actual
       Path clusteredPointsPath = new Path(outputPath, "clusteredPoints");
       // assertEquals("output dir files?", 4, outFiles.length);

Modified: mahout/trunk/core/src/test/java/org/apache/mahout/clustering/meanshift/TestMeanShift.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/meanshift/TestMeanShift.java?rev=964507&r1=964506&r2=964507&view=diff
==============================================================================
--- mahout/trunk/core/src/test/java/org/apache/mahout/clustering/meanshift/TestMeanShift.java (original)
+++ mahout/trunk/core/src/test/java/org/apache/mahout/clustering/meanshift/TestMeanShift.java Thu Jul 15 17:53:51 2010
@@ -33,6 +33,7 @@ import org.apache.hadoop.mapreduce.Reduc
 import org.apache.mahout.clustering.ClusteringTestUtils;
 import org.apache.mahout.common.DummyRecordWriter;
 import org.apache.mahout.common.MahoutTestCase;
+import org.apache.mahout.common.commandline.DefaultOptionCreator;
 import org.apache.mahout.common.distance.DistanceMeasure;
 import org.apache.mahout.common.distance.EuclideanDistanceMeasure;
 import org.apache.mahout.math.DenseVector;
@@ -136,13 +137,13 @@ public class TestMeanShift extends Mahou
     }
     assertTrue(true);
   }
-  
+
   /**
    * Test the MeanShiftCanopyClusterer's reference implementation. Should produce the same final output as above.
    */
   public void testClustererReferenceImplementation() {
     List<Vector> points = new ArrayList<Vector>();
-    for (Vector v: raw)
+    for (Vector v : raw)
       points.add(v);
     List<MeanShiftCanopy> canopies = MeanShiftCanopyClusterer.clusterPoints(points, new EuclideanDistanceMeasure(), 0.5, 4, 1, 10);
     printCanopies(canopies);
@@ -308,9 +309,15 @@ public class TestMeanShift extends Mahou
     }
     ClusteringTestUtils.writePointsToFile(points, getTestTempFilePath("testdata/file1"), fs, conf);
     ClusteringTestUtils.writePointsToFile(points, getTestTempFilePath("testdata/file2"), fs, conf);
-    // now run the Job
+    // now run the Job using the run() command. Other tests can continue to use runJob().
     Path output = getTestTempDirPath("output");
-    MeanShiftCanopyDriver.runJob(input, output, EuclideanDistanceMeasure.class.getName(), 4, 1, 0.5, 10, false, false);
+    //MeanShiftCanopyDriver.runJob(input, output, EuclideanDistanceMeasure.class.getName(), 4, 1, 0.5, 10, false, false);
+    String[] args = { DefaultOptionCreator.INPUT_OPTION_KEY, getTestTempDirPath("testdata").toString(),
+        DefaultOptionCreator.OUTPUT_OPTION_KEY, output.toString(), DefaultOptionCreator.DISTANCE_MEASURE_OPTION_KEY,
+        EuclideanDistanceMeasure.class.getName(), DefaultOptionCreator.T1_OPTION_KEY, "4", DefaultOptionCreator.T2_OPTION_KEY, "1",
+        DefaultOptionCreator.CLUSTERING_OPTION_KEY, DefaultOptionCreator.MAX_ITERATIONS_OPTION_KEY, "4",
+        DefaultOptionCreator.CONVERGENCE_DELTA_OPTION_KEY, "0.5", DefaultOptionCreator.OVERWRITE_OPTION_KEY  };
+    new MeanShiftCanopyDriver().run(args);
     Path outPart = new Path(output, "clusters-3/part-r-00000");
     SequenceFile.Reader reader = new SequenceFile.Reader(fs, outPart, conf);
     Text key = new Text();

Modified: mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/Job.java
URL: http://svn.apache.org/viewvc/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/Job.java?rev=964507&r1=964506&r2=964507&view=diff
==============================================================================
--- mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/Job.java (original)
+++ mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/Job.java Thu Jul 15 17:53:51 2010
@@ -18,79 +18,34 @@
 package org.apache.mahout.clustering.syntheticcontrol.canopy;
 
 import java.io.IOException;
+import java.util.Map;
 
-import org.apache.commons.cli2.CommandLine;
-import org.apache.commons.cli2.Group;
-import org.apache.commons.cli2.Option;
-import org.apache.commons.cli2.OptionException;
-import org.apache.commons.cli2.builder.ArgumentBuilder;
-import org.apache.commons.cli2.builder.DefaultOptionBuilder;
-import org.apache.commons.cli2.builder.GroupBuilder;
-import org.apache.commons.cli2.commandline.Parser;
 import org.apache.hadoop.fs.Path;
 import org.apache.mahout.clustering.canopy.CanopyDriver;
 import org.apache.mahout.clustering.syntheticcontrol.Constants;
-import org.apache.mahout.common.CommandLineUtil;
 import org.apache.mahout.common.HadoopUtil;
+import org.apache.mahout.common.commandline.DefaultOptionCreator;
+import org.apache.mahout.utils.clustering.ClusterDumper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public final class Job {
-
-  private static final Logger log = LoggerFactory.getLogger(Job.class);
+public final class Job extends CanopyDriver {
 
   private Job() {
+    super();
   }
 
+  private static final Logger log = LoggerFactory.getLogger(Job.class);
+
   public static void main(String[] args) throws Exception {
-    DefaultOptionBuilder obuilder = new DefaultOptionBuilder();
-    ArgumentBuilder abuilder = new ArgumentBuilder();
-    GroupBuilder gbuilder = new GroupBuilder();
-
-    Option inputOpt = obuilder.withLongName("input").withRequired(false).withArgument(
-        abuilder.withName("input").withMinimum(1).withMaximum(1).create()).withDescription(
-        "The Path for input Vectors. Must be a SequenceFile of Writable, Vector").withShortName("i").create();
-    Option outputOpt = obuilder.withLongName("output").withRequired(false).withArgument(
-        abuilder.withName("output").withMinimum(1).withMaximum(1).create()).withDescription("The Path to put the output in")
-        .withShortName("o").create();
-
-    Option measureClassOpt = obuilder.withLongName("distance").withRequired(false).withArgument(
-        abuilder.withName("distance").withMinimum(1).withMaximum(1).create()).withDescription(
-        "The Distance Measure to use.  Default is SquaredEuclidean").withShortName("m").create();
-
-    Option t1Opt = obuilder.withLongName("t1").withRequired(false).withArgument(
-        abuilder.withName("t1").withMinimum(1).withMaximum(1).create()).withDescription("t1").withShortName("t1").create();
-    Option t2Opt = obuilder.withLongName("t2").withRequired(false).withArgument(
-        abuilder.withName("t2").withMinimum(1).withMaximum(1).create()).withDescription("t2").withShortName("t2").create();
-
-    Option helpOpt = obuilder.withLongName("help").withDescription("Print out help").withShortName("h").create();
-
-    Group group = gbuilder.withName("Options").withOption(inputOpt).withOption(outputOpt).withOption(measureClassOpt).withOption(
-        t1Opt).withOption(t2Opt).withOption(helpOpt).create();
-
-    try {
-      Parser parser = new Parser();
-      parser.setGroup(group);
-      parser.setHelpOption(helpOpt);
-      CommandLine cmdLine = parser.parse(args);
-
-      if (cmdLine.hasOption(helpOpt)) {
-        CommandLineUtil.printHelp(group);
-        return;
-      }
-
-      Path input = new Path(cmdLine.getValue(inputOpt, "testdata").toString());
-      Path output = new Path(cmdLine.getValue(outputOpt, "output").toString());
-      String measureClass = cmdLine.getValue(measureClassOpt, "org.apache.mahout.common.distance.EuclideanDistanceMeasure")
-          .toString();
-
-      double t1 = Double.parseDouble(cmdLine.getValue(t1Opt, "80").toString());
-      double t2 = Double.parseDouble(cmdLine.getValue(t2Opt, "55").toString());
-
-      runJob(input, output, measureClass, t1, t2);
-    } catch (OptionException e) {
-      Job.log.error("Exception", e);
-      CommandLineUtil.printHelp(group);
+    if (args.length > 0) {
+      log.info("Running with only user-supplied arguments");
+      new Job().run(args);
+    } else {
+      log.info("Running with default arguments");
+      Path output = new Path("output");
+      HadoopUtil.overwriteOutput(output);
+      new Job().job(new Path("testdata"), output, "org.apache.mahout.common.distance.EuclideanDistanceMeasure", 80, 55);
     }
   }
 
@@ -115,14 +70,43 @@ public final class Job {
    * @throws ClassNotFoundException 
    * @throws InterruptedException 
    */
-  private static void runJob(Path input, Path output, String measureClassName, double t1, double t2) throws IOException,
+  private void job(Path input, Path output, String measureClassName, double t1, double t2) throws IOException,
       InstantiationException, IllegalAccessException, InterruptedException, ClassNotFoundException {
-    HadoopUtil.overwriteOutput(output);
-
     Path directoryContainingConvertedInput = new Path(output, Constants.DIRECTORY_CONTAINING_CONVERTED_INPUT);
     InputDriver.runJob(input, directoryContainingConvertedInput, "org.apache.mahout.math.RandomAccessSparseVector");
     CanopyDriver.runJob(directoryContainingConvertedInput, output, measureClassName, t1, t2, true);
+    // run ClusterDumper
+    ClusterDumper clusterDumper = new ClusterDumper(new Path(output, "clusters-0"),
+                                                    new Path(output, "clusteredPoints"));
+    clusterDumper.printClusters(null);
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+
+    addInputOption();
+    addOutputOption();
+    addOption(DefaultOptionCreator.distanceMeasureOption().create());
+    addOption(DefaultOptionCreator.t1Option().create());
+    addOption(DefaultOptionCreator.t2Option().create());
+    addOption(DefaultOptionCreator.overwriteOption().create());
+
+    Map<String, String> argMap = parseArguments(args);
+    if (argMap == null) {
+      return -1;
+    }
+
+    Path input = getInputPath();
+    Path output = getOutputPath();
+    if (argMap.containsKey(DefaultOptionCreator.OVERWRITE_OPTION_KEY)) {
+      HadoopUtil.overwriteOutput(output);
+    }
+    String measureClass = argMap.get(DefaultOptionCreator.DISTANCE_MEASURE_OPTION_KEY);
+    double t1 = Double.parseDouble(argMap.get(DefaultOptionCreator.T1_OPTION_KEY));
+    double t2 = Double.parseDouble(argMap.get(DefaultOptionCreator.T2_OPTION_KEY));
 
+    job(input, output, measureClass, t1, t2);
+    return 0;
   }
 
 }

Modified: mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/dirichlet/Job.java
URL: http://svn.apache.org/viewvc/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/dirichlet/Job.java?rev=964507&r1=964506&r2=964507&view=diff
==============================================================================
--- mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/dirichlet/Job.java (original)
+++ mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/dirichlet/Job.java Thu Jul 15 17:53:51 2010
@@ -21,94 +21,104 @@ import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
-import org.apache.commons.cli2.CommandLine;
-import org.apache.commons.cli2.Group;
-import org.apache.commons.cli2.Option;
-import org.apache.commons.cli2.OptionException;
 import org.apache.commons.cli2.builder.ArgumentBuilder;
 import org.apache.commons.cli2.builder.DefaultOptionBuilder;
-import org.apache.commons.cli2.builder.GroupBuilder;
-import org.apache.commons.cli2.commandline.Parser;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.mahout.clustering.dirichlet.DirichletCluster;
 import org.apache.mahout.clustering.dirichlet.DirichletDriver;
 import org.apache.mahout.clustering.dirichlet.DirichletMapper;
 import org.apache.mahout.clustering.dirichlet.models.Model;
+import org.apache.mahout.clustering.dirichlet.models.NormalModelDistribution;
 import org.apache.mahout.clustering.syntheticcontrol.Constants;
 import org.apache.mahout.clustering.syntheticcontrol.canopy.InputDriver;
-import org.apache.mahout.common.CommandLineUtil;
 import org.apache.mahout.common.HadoopUtil;
 import org.apache.mahout.common.commandline.DefaultOptionCreator;
+import org.apache.mahout.math.RandomAccessSparseVector;
 import org.apache.mahout.math.VectorWritable;
+import org.apache.mahout.utils.clustering.ClusterDumper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public final class Job {
-  
+public final class Job extends DirichletDriver {
+
+  private Job() {
+    super();
+  }
+
   private static final Logger log = LoggerFactory.getLogger(Job.class);
-  
-  private Job() { }
-  
+
   public static void main(String[] args) throws Exception {
-    DefaultOptionBuilder obuilder = new DefaultOptionBuilder();
-    ArgumentBuilder abuilder = new ArgumentBuilder();
-    GroupBuilder gbuilder = new GroupBuilder();
-    
-    Option inputOpt = DefaultOptionCreator.inputOption().withRequired(false).create();
-    Option outputOpt = DefaultOptionCreator.outputOption().withRequired(false).create();
-    Option maxIterOpt = DefaultOptionCreator.maxIterationsOption().withRequired(false).create();
-    Option topicsOpt = DefaultOptionCreator.kOption().withRequired(false).create();
-    
-    Option redOpt = obuilder.withLongName("reducerNum").withRequired(false).withArgument(
-      abuilder.withName("r").withMinimum(1).withMaximum(1).create()).withDescription(
-      "The number of reducers to use.").withShortName("r").create();
-    
-    Option vectorOpt = obuilder.withLongName("vector").withRequired(false).withArgument(
-      abuilder.withName("v").withMinimum(1).withMaximum(1).create()).withDescription(
-      "The vector implementation to use.").withShortName("v").create();
-    
-    Option mOpt = obuilder.withLongName("alpha").withRequired(false).withShortName("m").withArgument(
-      abuilder.withName("alpha").withMinimum(1).withMaximum(1).create()).withDescription(
-      "The alpha0 value for the DirichletDistribution.").create();
-    
-    Option modelOpt = obuilder.withLongName("modelClass").withRequired(false).withShortName("d")
-        .withArgument(abuilder.withName("modelClass").withMinimum(1).withMaximum(1).create())
-        .withDescription("The ModelDistribution class name.").create();
-    Option helpOpt = DefaultOptionCreator.helpOption();
-    
-    Group group = gbuilder.withName("Options").withOption(inputOpt).withOption(outputOpt)
-        .withOption(modelOpt).withOption(maxIterOpt).withOption(mOpt).withOption(topicsOpt)
-        .withOption(redOpt).withOption(helpOpt).create();
-    
-    try {
-      Parser parser = new Parser();
-      parser.setGroup(group);
-      CommandLine cmdLine = parser.parse(args);
-      if (cmdLine.hasOption(helpOpt)) {
-        CommandLineUtil.printHelp(group);
-        return;
-      }
-      
-      Path input = new Path(cmdLine.getValue(inputOpt, "testdata").toString());
-      Path output = new Path(cmdLine.getValue(outputOpt, "output").toString());
-      String modelFactory = cmdLine.getValue(modelOpt,
-        "org.apache.mahout.clustering.syntheticcontrol.dirichlet.NormalScModelDistribution").toString();
-      int numModels = Integer.parseInt(cmdLine.getValue(topicsOpt, "10").toString());
-      int maxIterations = Integer.parseInt(cmdLine.getValue(maxIterOpt, "5").toString());
-      double alpha0 = Double.parseDouble(cmdLine.getValue(mOpt, "1.0").toString());
-      int numReducers = Integer.parseInt(cmdLine.getValue(redOpt, "1").toString());
-      String vectorClassName = cmdLine.getValue(vectorOpt, "org.apache.mahout.math.RandomAccessSparseVector")
-          .toString();
-      runJob(input, output, modelFactory, numModels, maxIterations, alpha0, numReducers,
-            vectorClassName);
-    } catch (OptionException e) {
-      log.error("Exception parsing command line: ", e);
-      CommandLineUtil.printHelp(group);
+    if (args.length > 0) {
+      log.info("Running with only user-supplied arguments");
+      new Job().run(args);
+    } else {
+      log.info("Running with default arguments");
+      Path output = new Path("output");
+      HadoopUtil.overwriteOutput(output);
+      new Job().job(new Path("testdata"),
+                    output,
+                    "org.apache.mahout.clustering.syntheticcontrol.dirichlet.NormalScModelDistribution",
+                    "org.apache.mahout.math.RandomAccessSparseVector",
+                    10,
+                    5,
+                    1.0,
+                    1,
+                    false,
+                    0.001);
+    }
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.util.Tool#run(java.lang.String[])
+   */
+  public int run(String[] args) throws IOException, ClassNotFoundException, InstantiationException, IllegalAccessException,
+      NoSuchMethodException, InvocationTargetException, InterruptedException {
+    addInputOption();
+    addOutputOption();
+    addOption(DefaultOptionCreator.maxIterationsOption().create());
+    addOption(DefaultOptionCreator.numClustersOption().withRequired(true).create());
+    addOption(DefaultOptionCreator.overwriteOption().create());
+    addOption(new DefaultOptionBuilder().withLongName(ALPHA_OPTION).withRequired(false).withShortName("m")
+        .withArgument(new ArgumentBuilder().withName(ALPHA_OPTION).withDefault("1.0").withMinimum(1).withMaximum(1).create())
+        .withDescription("The alpha0 value for the DirichletDistribution. Defaults to 1.0").create());
+    addOption(new DefaultOptionBuilder().withLongName(MODEL_DISTRIBUTION_CLASS_OPTION).withRequired(false).withShortName("md")
+        .withArgument(new ArgumentBuilder().withName(MODEL_DISTRIBUTION_CLASS_OPTION).withDefault(NormalModelDistribution.class
+            .getName()).withMinimum(1).withMaximum(1).create()).withDescription("The ModelDistribution class name. "
+            + "Defaults to NormalModelDistribution").create());
+    addOption(new DefaultOptionBuilder().withLongName(MODEL_PROTOTYPE_CLASS_OPTION).withRequired(false).withShortName("mp")
+        .withArgument(new ArgumentBuilder().withName("prototypeClass").withDefault(RandomAccessSparseVector.class.getName())
+            .withMinimum(1).withMaximum(1).create())
+        .withDescription("The ModelDistribution prototype Vector class name. Defaults to RandomAccessSparseVector").create());
+    addOption(DefaultOptionCreator.emitMostLikelyOption().create());
+    addOption(DefaultOptionCreator.thresholdOption().create());
+    addOption(DefaultOptionCreator.numReducersOption().create());
+
+    Map<String, String> argMap = parseArguments(args);
+    if (argMap == null) {
+      return -1;
+    }
+
+    Path input = getInputPath();
+    Path output = getOutputPath();
+    if (argMap.containsKey(DefaultOptionCreator.OVERWRITE_OPTION_KEY)) {
+      HadoopUtil.overwriteOutput(output);
     }
+    String modelFactory = argMap.get(MODEL_DISTRIBUTION_CLASS_OPTION_KEY);
+    String modelPrototype = argMap.get(MODEL_PROTOTYPE_CLASS_OPTION_KEY);
+    int numModels = Integer.parseInt(argMap.get(DefaultOptionCreator.NUM_CLUSTERS_OPTION_KEY));
+    int numReducers = Integer.parseInt(argMap.get(DefaultOptionCreator.MAX_REDUCERS_OPTION_KEY));
+    int maxIterations = Integer.parseInt(argMap.get(DefaultOptionCreator.MAX_ITERATIONS_OPTION_KEY));
+    boolean emitMostLikely = Boolean.parseBoolean(argMap.get(DefaultOptionCreator.EMIT_MOST_LIKELY_OPTION_KEY));
+    double threshold = Double.parseDouble(argMap.get(DefaultOptionCreator.THRESHOLD_OPTION_KEY));
+    double alpha0 = Double.parseDouble(argMap.get(ALPHA_OPTION_KEY));
+
+    job(input, output, modelFactory, modelPrototype, numModels, maxIterations, alpha0, numReducers, emitMostLikely, threshold);
+    return 0;
   }
-  
+
   /**
    * Run the job using supplied arguments, deleting the output directory if it exists beforehand
    * 
@@ -126,32 +136,43 @@ public final class Job {
    *          the alpha0 value for the DirichletDistribution
    * @param numReducers
    *          the desired number of reducers
+   * @param emitMostLikely 
+   * @param threshold 
    * @throws InterruptedException 
    * @throws SecurityException 
    */
-  public static void runJob(Path input,
-                            Path output,
-                            String modelFactory,
-                            int numModels,
-                            int maxIterations,
-                            double alpha0,
-                            int numReducers,
-                            String vectorClassName) throws IOException,
-                                                   ClassNotFoundException,
-                                                   InstantiationException,
-                                                   IllegalAccessException,
-                                                   NoSuchMethodException,
-                                                   InvocationTargetException, SecurityException, InterruptedException {
-    HadoopUtil.overwriteOutput(output);
-
+  private void job(Path input,
+                   Path output,
+                   String modelFactory,
+                   String modelPrototype,
+                   int numModels,
+                   int maxIterations,
+                   double alpha0,
+                   int numReducers,
+                   boolean emitMostLikely,
+                   double threshold) throws IOException, ClassNotFoundException, InstantiationException, IllegalAccessException,
+      NoSuchMethodException, InvocationTargetException, SecurityException, InterruptedException {
     Path directoryContainingConvertedInput = new Path(output, Constants.DIRECTORY_CONTAINING_CONVERTED_INPUT);
-    InputDriver.runJob(input, directoryContainingConvertedInput, vectorClassName);
-    DirichletDriver.runJob(directoryContainingConvertedInput, output, modelFactory,
-      vectorClassName, numModels, maxIterations, alpha0, numReducers, true, true, 0);
+    InputDriver.runJob(input, directoryContainingConvertedInput, modelPrototype);
+    DirichletDriver.runJob(directoryContainingConvertedInput,
+                           output,
+                           modelFactory,
+                           modelPrototype,
+                           numModels,
+                           maxIterations,
+                           alpha0,
+                           numReducers,
+                           true,
+                           emitMostLikely,
+                           threshold);
+    // run ClusterDumper
+    ClusterDumper clusterDumper = new ClusterDumper(new Path(output, "clusters-" + maxIterations), new Path(output,
+                                                                                                            "clusteredPoints"));
+    clusterDumper.printClusters(null);
   }
-  
+
   /**
-   * Prints out all of the clusters during each iteration
+   * Prints out all of the clusters for each iteration
    * 
    * @param output
    *          the String output directory
@@ -177,8 +198,7 @@ public final class Job {
                                   int prototypeSize,
                                   int numIterations,
                                   int numModels,
-                                  double alpha0) throws NoSuchMethodException,
-                                                 InvocationTargetException {
+                                  double alpha0) throws NoSuchMethodException, InvocationTargetException {
     List<List<DirichletCluster<VectorWritable>>> clusters = new ArrayList<List<DirichletCluster<VectorWritable>>>();
     Configuration conf = new Configuration();
     conf.set(DirichletDriver.MODEL_FACTORY_KEY, modelDistribution);
@@ -190,10 +210,10 @@ public final class Job {
       conf.set(DirichletDriver.PROTOTYPE_SIZE_KEY, Integer.toString(prototypeSize));
       clusters.add(DirichletMapper.getDirichletState(conf).getClusters());
     }
-    printResults(clusters, 0);
-    
+    printClusters(clusters, 0);
+
   }
-  
+
   /**
    * Actually prints out the clusters
    * 
@@ -202,7 +222,7 @@ public final class Job {
    * @param significant
    *          the minimum number of samples to enable printing a model
    */
-  private static void printResults(List<List<DirichletCluster<VectorWritable>>> clusters, int significant) {
+  private static void printClusters(List<List<DirichletCluster<VectorWritable>>> clusters, int significant) {
     int row = 0;
     StringBuilder result = new StringBuilder();
     for (List<DirichletCluster<VectorWritable>> r : clusters) {

Added: mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/fuzzykmeans/Job.java
URL: http://svn.apache.org/viewvc/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/fuzzykmeans/Job.java?rev=964507&view=auto
==============================================================================
--- mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/fuzzykmeans/Job.java (added)
+++ mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/fuzzykmeans/Job.java Thu Jul 15 17:53:51 2010
@@ -0,0 +1,190 @@
+/**
+ * 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.mahout.clustering.syntheticcontrol.fuzzykmeans;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.commons.cli2.builder.ArgumentBuilder;
+import org.apache.commons.cli2.builder.DefaultOptionBuilder;
+import org.apache.hadoop.fs.Path;
+import org.apache.mahout.clustering.Cluster;
+import org.apache.mahout.clustering.canopy.CanopyDriver;
+import org.apache.mahout.clustering.fuzzykmeans.FuzzyKMeansDriver;
+import org.apache.mahout.clustering.kmeans.RandomSeedGenerator;
+import org.apache.mahout.clustering.syntheticcontrol.Constants;
+import org.apache.mahout.clustering.syntheticcontrol.canopy.InputDriver;
+import org.apache.mahout.common.HadoopUtil;
+import org.apache.mahout.common.commandline.DefaultOptionCreator;
+import org.apache.mahout.common.distance.SquaredEuclideanDistanceMeasure;
+import org.apache.mahout.utils.clustering.ClusterDumper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public final class Job extends FuzzyKMeansDriver {
+
+  private static final Logger log = LoggerFactory.getLogger(Job.class);
+
+  private Job() {
+    super();
+  }
+
+  public static void main(String[] args) throws Exception {
+    if (args.length > 0) {
+      log.info("Running with only user-supplied arguments");
+      new Job().run(args);
+    } else {
+      log.info("Running with default arguments");
+      Path output = new Path("output");
+      HadoopUtil.overwriteOutput(output);
+      new Job().job(new Path("testdata"),
+                    output,
+                    "org.apache.mahout.common.distance.EuclideanDistanceMeasure",
+                    80,
+                    55,
+                    10,
+                    1,
+                    (float) 2,
+                    0.5,
+                    true);
+
+    }
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+
+    addInputOption();
+    addOutputOption();
+    addOption(DefaultOptionCreator.distanceMeasureOption().create());
+    addOption(DefaultOptionCreator.clustersInOption()
+        .withDescription("The input centroids, as Vectors.  Must be a SequenceFile of Writable, Cluster/Canopy.  "
+            + "If k is also specified, then a random set of vectors will be selected" + " and written out to this path first")
+        .create());
+    addOption(DefaultOptionCreator.numClustersOption()
+        .withDescription("The k in k-Means.  If specified, then a random selection of k Vectors will be chosen"
+            + " as the Centroid and written to the clusters input path.").create());
+    addOption(DefaultOptionCreator.convergenceOption().create());
+    addOption(DefaultOptionCreator.maxIterationsOption().create());
+    addOption(DefaultOptionCreator.overwriteOption().create());
+    addOption(DefaultOptionCreator.numReducersOption().create());
+    addOption(DefaultOptionCreator.clusteringOption().create());
+    addOption(DefaultOptionCreator.t1Option().create());
+    addOption(DefaultOptionCreator.t2Option().create());
+
+    Map<String, String> argMap = parseArguments(args);
+    if (argMap == null) {
+      return -1;
+    }
+
+    Path input = getInputPath();
+    Path clusters = new Path(argMap.get(DefaultOptionCreator.CLUSTERS_IN_OPTION_KEY));
+    Path output = getOutputPath();
+    String measureClass = argMap.get(DefaultOptionCreator.DISTANCE_MEASURE_OPTION_KEY);
+    if (measureClass == null) {
+      measureClass = SquaredEuclideanDistanceMeasure.class.getName();
+    }
+    double convergenceDelta = Double.parseDouble(argMap.get(DefaultOptionCreator.CONVERGENCE_DELTA_OPTION_KEY));
+    int numReduceTasks = Integer.parseInt(argMap.get(DefaultOptionCreator.MAX_REDUCERS_OPTION_KEY));
+    int maxIterations = Integer.parseInt(argMap.get(DefaultOptionCreator.MAX_ITERATIONS_OPTION_KEY));
+    float fuzziness = Float.parseFloat(argMap.get(M_OPTION_KEY));
+
+    addOption(new DefaultOptionBuilder().withLongName(M_OPTION).withRequired(true).withArgument(new ArgumentBuilder()
+        .withName(M_OPTION).withMinimum(1).withMaximum(1).create())
+        .withDescription("coefficient normalization factor, must be greater than 1").withShortName(M_OPTION).create());
+    if (argMap.containsKey(DefaultOptionCreator.OVERWRITE_OPTION_KEY)) {
+      HadoopUtil.overwriteOutput(output);
+    }
+    if (argMap.containsKey(DefaultOptionCreator.NUM_CLUSTERS_OPTION_KEY)) {
+      clusters = RandomSeedGenerator.buildRandom(input, clusters, Integer.parseInt(argMap
+          .get(DefaultOptionCreator.NUM_CLUSTERS_OPTION_KEY)));
+    }
+    boolean runClustering = argMap.containsKey(DefaultOptionCreator.CLUSTERING_OPTION_KEY);
+    double t1 = Double.parseDouble(argMap.get(DefaultOptionCreator.T1_OPTION_KEY));
+    double t2 = Double.parseDouble(argMap.get(DefaultOptionCreator.T2_OPTION_KEY));
+    job(input, output, measureClass, t1, t2, maxIterations, numReduceTasks, fuzziness, convergenceDelta, runClustering);
+    return 0;
+  }
+
+  /**
+   * Run the kmeans clustering job on an input dataset using the given distance measure, t1, t2 and iteration
+   * parameters. All output data will be written to the output directory, which will be initially deleted if
+   * it exists. The clustered points will reside in the path <output>/clustered-points. By default, the job
+   * expects the a file containing synthetic_control.data as obtained from
+   * http://archive.ics.uci.edu/ml/datasets/Synthetic+Control+Chart+Time+Series resides in a directory named
+   * "testdata", and writes output to a directory named "output".
+   * 
+   * @param input
+   *          the String denoting the input directory path
+   * @param output
+   *          the String denoting the output directory path
+   * @param measureClass
+   *          the canopy T1 threshold
+   * @param t2
+   *          the canopy T2 threshold
+   * @param maxIterations 
+   *          the int maximum number of iterations
+   * @param numReducerTasks 
+   *          the int number of reducer tasks
+   * @param fuzziness 
+   *          the float "m" fuzziness coefficient
+   * @param convergenceDelta
+   *          the double convergence criteria for iterations
+   * @param runClustering
+   *          the int maximum number of iterations
+   * @param output2
+   *          the String class name of the DistanceMeasure to use
+   * @throws IllegalAccessException 
+   * @throws InstantiationException 
+   * @throws ClassNotFoundException 
+   * @throws InterruptedException 
+   */
+  private void job(Path input,
+                   Path output,
+                   String measureClass,
+                   double t1,
+                   double t2,
+                   int maxIterations,
+                   int numReducerTasks,
+                   float fuzziness,
+                   double convergenceDelta,
+                   boolean runClustering) throws IOException, InstantiationException, IllegalAccessException, InterruptedException,
+      ClassNotFoundException {
+
+    Path directoryContainingConvertedInput = new Path(output, Constants.DIRECTORY_CONTAINING_CONVERTED_INPUT);
+    log.info("Preparing Input");
+    InputDriver.runJob(input, directoryContainingConvertedInput, "org.apache.mahout.math.RandomAccessSparseVector");
+    log.info("Running Canopy to get initial clusters");
+    CanopyDriver.runJob(directoryContainingConvertedInput, output, measureClass, t1, t2, false);
+    log.info("Running FuzzyKMeans");
+    FuzzyKMeansDriver.runJob(directoryContainingConvertedInput,
+                             new Path(output, Cluster.INITIAL_CLUSTERS_DIR),
+                             output,
+                             measureClass,
+                             convergenceDelta,
+                             maxIterations,
+                             numReducerTasks,
+                             fuzziness,
+                             true,
+                             false,
+                             0.0);
+    // run ClusterDumper
+    ClusterDumper clusterDumper = new ClusterDumper(new Path(output, "clusters-3"), new Path(output, "clusteredPoints"));
+    clusterDumper.printClusters(null);
+  }
+}