You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by sr...@apache.org on 2010/05/06 13:43:04 UTC

svn commit: r941676 [2/3] - in /lucene/mahout/trunk: core/src/main/java/org/apache/mahout/classifier/bayes/ core/src/main/java/org/apache/mahout/classifier/bayes/mapreduce/bayes/ core/src/main/java/org/apache/mahout/classifier/bayes/mapreduce/cbayes/ c...

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/RandomSeedGenerator.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/RandomSeedGenerator.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/RandomSeedGenerator.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/RandomSeedGenerator.java Thu May  6 11:43:01 2010
@@ -29,6 +29,7 @@ import org.apache.hadoop.io.SequenceFile
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.mahout.common.HadoopUtil;
 import org.apache.mahout.common.RandomUtils;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.math.VectorWritable;
@@ -49,31 +50,22 @@ public final class RandomSeedGenerator {
   
   private RandomSeedGenerator() {}
   
-  public static Path buildRandom(String input, String output, int k) throws IOException,
+  public static Path buildRandom(Path input, Path output, int k) throws IOException,
                                                                     IllegalAccessException,
                                                                     InstantiationException {
     // delete the output directory
     JobConf conf = new JobConf(RandomSeedGenerator.class);
-    Path outPath = new Path(output);
-    FileSystem fs = FileSystem.get(outPath.toUri(), conf);
-    if (fs.exists(outPath)) {
-      fs.delete(outPath, true);
-    }
-    fs.mkdirs(outPath);
-    Path outFile = new Path(outPath, "part-randomSeed");
-    if (fs.exists(outFile)) {
-      log.warn("Deleting {}", outFile);
-      fs.delete(outFile, false);
-    }
+    FileSystem fs = FileSystem.get(output.toUri(), conf);
+    HadoopUtil.overwriteOutput(output);
+    Path outFile = new Path(output, "part-randomSeed");
     boolean newFile = fs.createNewFile(outFile);
     if (newFile) {
       Path inputPathPattern;
-      Path inputPath = new Path(input);
-      
-      if (fs.getFileStatus(inputPath).isDir()) {
-        inputPathPattern = new Path(inputPath.toString() + "/*");
+
+      if (fs.getFileStatus(input).isDir()) {
+        inputPathPattern = new Path(input, "*");
       } else {
-        inputPathPattern = inputPath;
+        inputPathPattern = input;
       }
       
       FileStatus[] inputFiles = fs.globStatus(inputPathPattern);

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/lda/LDADriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/lda/LDADriver.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/lda/LDADriver.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/lda/LDADriver.java Thu May  6 11:43:01 2010
@@ -118,8 +118,8 @@ public final class LDADriver {
         CommandLineUtil.printHelp(group);
         return;
       }
-      String input = cmdLine.getValue(inputOpt).toString();
-      String output = cmdLine.getValue(outputOpt).toString();
+      Path input = new Path(cmdLine.getValue(inputOpt).toString());
+      Path output = new Path(cmdLine.getValue(outputOpt).toString());
       
       int maxIterations = -1;
       if (cmdLine.hasOption(maxIterOpt)) {
@@ -180,15 +180,15 @@ public final class LDADriver {
    *          the number of Reducers desired
    * @throws IOException
    */
-  public static void runJob(String input,
-                            String output,
+  public static void runJob(Path input,
+                            Path output,
                             int numTopics,
                             int numWords,
                             double topicSmoothing,
                             int maxIterations,
                             int numReducers) throws IOException, InterruptedException, ClassNotFoundException {
     
-    String stateIn = output + "/state-0";
+    Path stateIn = new Path(output, "state-0");
     writeInitialState(stateIn, numTopics, numWords);
     double oldLL = Double.NEGATIVE_INFINITY;
     boolean converged = false;
@@ -196,7 +196,7 @@ public final class LDADriver {
     for (int iteration = 0; ((maxIterations < 1) || (iteration < maxIterations)) && !converged; iteration++) {
       log.info("Iteration {}", iteration);
       // point the output to a new directory per iteration
-      String stateOut = output + "/state-" + (iteration + 1);
+      Path stateOut = new Path(output, "state-" + (iteration + 1));
       double ll = runIteration(input, stateIn, stateOut, numTopics, numWords, topicSmoothing, numReducers);
       double relChange = (oldLL - ll) / oldLL;
       
@@ -211,17 +211,16 @@ public final class LDADriver {
     }
   }
   
-  private static void writeInitialState(String statePath, int numTopics, int numWords) throws IOException {
-    Path dir = new Path(statePath);
+  private static void writeInitialState(Path statePath, int numTopics, int numWords) throws IOException {
     Configuration job = new Configuration();
-    FileSystem fs = dir.getFileSystem(job);
+    FileSystem fs = statePath.getFileSystem(job);
     
     DoubleWritable v = new DoubleWritable();
     
     Random random = RandomUtils.getRandom();
     
     for (int k = 0; k < numTopics; ++k) {
-      Path path = new Path(dir, "part-" + k);
+      Path path = new Path(statePath, "part-" + k);
       SequenceFile.Writer writer = new SequenceFile.Writer(fs, job, path, IntPairWritable.class,
           DoubleWritable.class);
       
@@ -242,15 +241,14 @@ public final class LDADriver {
     }
   }
   
-  private static double findLL(String statePath, Configuration job) throws IOException {
-    Path dir = new Path(statePath);
-    FileSystem fs = dir.getFileSystem(job);
+  private static double findLL(Path statePath, Configuration job) throws IOException {
+    FileSystem fs = statePath.getFileSystem(job);
     
     double ll = 0.0;
     
     IntPairWritable key = new IntPairWritable();
     DoubleWritable value = new DoubleWritable();
-    for (FileStatus status : fs.globStatus(new Path(dir, "part-*"))) {
+    for (FileStatus status : fs.globStatus(new Path(statePath, "part-*"))) {
       Path path = status.getPath();
       SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, job);
       while (reader.next(key, value)) {
@@ -279,9 +277,9 @@ public final class LDADriver {
    * @param numReducers
    *          the number of Reducers desired
    */
-  public static double runIteration(String input,
-                                    String stateIn,
-                                    String stateOut,
+  public static double runIteration(Path input,
+                                    Path stateIn,
+                                    Path stateOut,
                                     int numTopics,
                                     int numWords,
                                     double topicSmoothing,
@@ -289,7 +287,7 @@ public final class LDADriver {
                                                     InterruptedException,
                                                     ClassNotFoundException {
     Configuration conf = new Configuration();
-    conf.set(STATE_IN_KEY, stateIn);
+    conf.set(STATE_IN_KEY, stateIn.toString());
     conf.set(NUM_TOPICS_KEY, Integer.toString(numTopics));
     conf.set(NUM_WORDS_KEY, Integer.toString(numWords));
     conf.set(TOPIC_SMOOTHING_KEY, Double.toString(topicSmoothing));
@@ -298,9 +296,8 @@ public final class LDADriver {
     
     job.setOutputKeyClass(IntPairWritable.class);
     job.setOutputValueClass(DoubleWritable.class);
-    FileInputFormat.addInputPaths(job, input);
-    Path outPath = new Path(stateOut);
-    FileOutputFormat.setOutputPath(job, outPath);
+    FileInputFormat.addInputPaths(job, input.toString());
+    FileOutputFormat.setOutputPath(job, stateOut);
     
     job.setMapperClass(LDAMapper.class);
     job.setReducerClass(LDAReducer.class);

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyDriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyDriver.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyDriver.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyDriver.java Thu May  6 11:43:01 2010
@@ -53,7 +53,7 @@ public final class MeanShiftCanopyDriver
 
   public static final String STATE_IN_KEY = "org.apache.mahout.clustering.meanshift.stateInKey";
 
-  protected static final String CONTROL_CONVERGED = "/control/converged";
+  protected static final String CONTROL_CONVERGED = "control/converged";
 
   private MeanShiftCanopyDriver() {
   }
@@ -104,15 +104,15 @@ public final class MeanShiftCanopyDriver
         runClustering = Boolean.parseBoolean(cmdLine.getValue(clusteringOpt).toString());
       }
 
-      String input = cmdLine.getValue(inputOpt).toString();
-      String output = cmdLine.getValue(outputOpt).toString();
+      Path input = new Path(cmdLine.getValue(inputOpt).toString());
+      Path output = new Path(cmdLine.getValue(outputOpt).toString());
       String measureClassName = cmdLine.getValue(modelOpt).toString();
       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());
       boolean inputIsCanopies = Boolean.parseBoolean(cmdLine.getValue(inputIsCanopiesOpt).toString());
-      createCanopyFromVectors(input, output + "/intial-canopies");
+      createCanopyFromVectors(input, new Path(output, "intial-canopies"));
       runJob(input, output, measureClassName, t1, t2, convergenceDelta, maxIterations, inputIsCanopies, runClustering);
     } catch (OptionException e) {
       log.error("Exception parsing command line: ", e);
@@ -138,7 +138,7 @@ public final class MeanShiftCanopyDriver
    * @param convergenceDelta
    *          the double convergence criteria
    */
-  static void runIteration(String input, String output, String control, String measureClassName, double t1, double t2,
+  static void runIteration(Path input, Path output, Path control, String measureClassName, double t1, double t2,
       double convergenceDelta) {
 
     Configurable client = new JobClient();
@@ -147,9 +147,8 @@ public final class MeanShiftCanopyDriver
     conf.setOutputKeyClass(Text.class);
     conf.setOutputValueClass(MeanShiftCanopy.class);
 
-    FileInputFormat.setInputPaths(conf, new Path(input));
-    Path outPath = new Path(output);
-    FileOutputFormat.setOutputPath(conf, outPath);
+    FileInputFormat.setInputPaths(conf, input);
+    FileOutputFormat.setOutputPath(conf, output);
 
     conf.setMapperClass(MeanShiftCanopyMapper.class);
     conf.setReducerClass(MeanShiftCanopyReducer.class);
@@ -160,7 +159,7 @@ public final class MeanShiftCanopyDriver
     conf.set(MeanShiftCanopyConfigKeys.CLUSTER_CONVERGENCE_KEY, String.valueOf(convergenceDelta));
     conf.set(MeanShiftCanopyConfigKeys.T1_KEY, String.valueOf(t1));
     conf.set(MeanShiftCanopyConfigKeys.T2_KEY, String.valueOf(t2));
-    conf.set(MeanShiftCanopyConfigKeys.CONTROL_PATH_KEY, control);
+    conf.set(MeanShiftCanopyConfigKeys.CONTROL_PATH_KEY, control.toString());
 
     client.setConf(conf);
     try {
@@ -170,15 +169,7 @@ public final class MeanShiftCanopyDriver
     }
   }
 
-  /**
-   * Run the job
-   * 
-   * @param input
-   *          the input pathname String
-   * @param output
-   *          the output pathname String
-   */
-  static void createCanopyFromVectors(String input, String output) {
+  static void createCanopyFromVectors(Path input, Path output) {
 
     Configurable client = new JobClient();
     JobConf conf = new JobConf(MeanShiftCanopyDriver.class);
@@ -186,9 +177,8 @@ public final class MeanShiftCanopyDriver
     conf.setOutputKeyClass(Text.class);
     conf.setOutputValueClass(MeanShiftCanopy.class);
 
-    FileInputFormat.setInputPaths(conf, new Path(input));
-    Path outPath = new Path(output);
-    FileOutputFormat.setOutputPath(conf, outPath);
+    FileInputFormat.setInputPaths(conf, input);
+    FileOutputFormat.setOutputPath(conf, output);
 
     conf.setMapperClass(MeanShiftCanopyCreatorMapper.class);
     conf.setNumReduceTasks(0);
@@ -213,7 +203,7 @@ public final class MeanShiftCanopyDriver
    * @param output
    *          the directory pathname for output clustered points
    */
-  static void runClustering(String input, String clustersIn, String output) {
+  static void runClustering(Path input, Path clustersIn, Path output) {
 
     JobConf conf = new JobConf(FuzzyKMeansDriver.class);
     conf.setJobName("Mean Shift Clustering");
@@ -221,9 +211,8 @@ public final class MeanShiftCanopyDriver
     conf.setOutputKeyClass(IntWritable.class);
     conf.setOutputValueClass(WeightedVectorWritable.class);
 
-    FileInputFormat.setInputPaths(conf, new Path(input));
-    Path outPath = new Path(output);
-    FileOutputFormat.setOutputPath(conf, outPath);
+    FileInputFormat.setInputPaths(conf, input);
+    FileOutputFormat.setOutputPath(conf, output);
 
     conf.setMapperClass(MeanShiftCanopyClusterMapper.class);
 
@@ -233,7 +222,7 @@ public final class MeanShiftCanopyDriver
     // uncomment it to run locally
     // conf.set("mapred.job.tracker", "local");
     conf.setNumReduceTasks(0);
-    conf.set(STATE_IN_KEY, clustersIn);
+    conf.set(STATE_IN_KEY, clustersIn.toString());
     try {
       JobClient.runJob(conf);
     } catch (IOException e) {
@@ -263,12 +252,12 @@ public final class MeanShiftCanopyDriver
    * @param runClustering 
    *          true if the input points are to be clustered once the iterations complete
    */
-  public static void runJob(String input, String output, String measureClassName, double t1, double t2, double convergenceDelta,
+  public static void runJob(Path input, Path output, String measureClassName, double t1, double t2, double convergenceDelta,
       int maxIterations, boolean inputIsCanopies, boolean runClustering) throws IOException {
     // delete the output directory
     Configuration conf = new JobConf(MeanShiftCanopyDriver.class);
 
-    String clustersIn = output + Cluster.INITIAL_CLUSTERS_DIR;
+    Path clustersIn = new Path(output, Cluster.INITIAL_CLUSTERS_DIR);
     if (inputIsCanopies) {
       clustersIn = input;
     } else {
@@ -281,10 +270,10 @@ public final class MeanShiftCanopyDriver
     while (!converged && (iteration <= maxIterations)) {
       log.info("Iteration {}", iteration);
       // point the output to a new directory per iteration
-      String clustersOut = output + Cluster.CLUSTERS_DIR + iteration;
-      String controlOut = output + CONTROL_CONVERGED;
+      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(conf).exists(new Path(controlOut));
+      converged = FileSystem.get(conf).exists(controlOut);
       // now point the input to the old output directory
       clustersIn = clustersOut;
       iteration++;
@@ -292,8 +281,8 @@ public final class MeanShiftCanopyDriver
 
     if (runClustering) {
       // now cluster the points
-      MeanShiftCanopyDriver.runClustering((inputIsCanopies ? input : output + Cluster.INITIAL_CLUSTERS_DIR), clustersIn, output
-          + Cluster.CLUSTERED_POINTS_DIR);
+      MeanShiftCanopyDriver.runClustering((inputIsCanopies ? input : new Path(output, Cluster.INITIAL_CLUSTERS_DIR)),
+                                          clustersIn, new Path(output, Cluster.CLUSTERED_POINTS_DIR));
     }
   }
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/HadoopUtil.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/HadoopUtil.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/HadoopUtil.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/HadoopUtil.java Thu May  6 11:43:01 2010
@@ -33,37 +33,27 @@ public final class HadoopUtil {
   
   private HadoopUtil() { }
   
-  public static void overwriteOutput(String output) throws IOException {
+  public static void overwriteOutput(Path output) throws IOException {
     Configuration conf = new JobConf(KMeansDriver.class);
-    Path outPath = new Path(output);
-    FileSystem fs = FileSystem.get(outPath.toUri(), conf);
-    if (fs.exists(outPath)) {
-      log.warn("Deleting {}", outPath);
-      fs.delete(outPath, true);
-    }
-    log.warn("Creating dir {}", outPath);
-    fs.mkdirs(outPath);
-  }
-  
-  public static void deletePath(String output, FileSystem fs) throws IOException {
-    Path outPath = new Path(output);
-    if (fs.exists(outPath)) {
-      log.warn("Deleting {}", outPath);
-      fs.delete(outPath, true);
+    FileSystem fs = FileSystem.get(output.toUri(), conf);
+    //boolean wasFile = fs.isFile(output);
+    if (fs.exists(output)) {
+      log.info("Deleting {}", output);
+      fs.delete(output, true);
     }
+    //if (!wasFile) {
+    //  log.info("Creating dir {}", output);
+    //  fs.mkdirs(output);
+    //}
   }
   
   public static void deletePaths(Iterable<Path> paths, FileSystem fs) throws IOException {
     for (Path path : paths) {
       if (fs.exists(path)) {
-        log.warn("Deleting {}", path);
+        log.info("Deleting {}", path);
         fs.delete(path, true);
       }
     }
   }
-  
-  public static void rename(Path from, Path to, FileSystem fs) throws IOException {
-    log.warn("Renaming " + from.toUri() + " to " + to.toUri());
-    fs.rename(from, to);
-  }
+
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/Builder.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/Builder.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/Builder.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/Builder.java Thu May  6 11:43:01 2010
@@ -29,6 +29,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.mahout.common.HadoopUtil;
 import org.apache.mahout.common.StringUtils;
 import org.apache.mahout.df.DecisionForest;
 import org.apache.mahout.df.builder.TreeBuilder;
@@ -291,10 +292,7 @@ public abstract class Builder {
     if (isOutput(job)) {
       log.debug("Parsing the output...");
       DecisionForest forest = parseOutput(job, callback);
-      
-      // delete the output path
-      fs.delete(outputPath, true);
-      
+      HadoopUtil.overwriteOutput(outputPath);
       return forest;
     }
     

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/Builder.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/Builder.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/Builder.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/Builder.java Thu May  6 11:43:01 2010
@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Job;
+import org.apache.mahout.common.HadoopUtil;
 import org.apache.mahout.common.StringUtils;
 import org.apache.mahout.df.DecisionForest;
 import org.apache.mahout.df.builder.TreeBuilder;
@@ -337,10 +338,7 @@ public abstract class Builder {
     if (isOutput(conf)) {
       log.debug("Parsing the output...");
       DecisionForest forest = parseOutput(job, callback);
-      
-      // delete the output path
-      fs.delete(outputPath, true);
-      
+      HadoopUtil.overwriteOutput(outputPath);
       return forest;
     }
     

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/Classifier.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/Classifier.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/Classifier.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/Classifier.java Thu May  6 11:43:01 2010
@@ -17,6 +17,7 @@
 
 package org.apache.mahout.df.mapreduce;
 
+import org.apache.mahout.common.HadoopUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.mahout.df.DecisionForest;
@@ -122,8 +123,7 @@ public class Classifier {
 
     parseOutput(job);
 
-    // delete the output path
-    fs.delete(mappersOutputPath, true);
+    HadoopUtil.overwriteOutput(mappersOutputPath);
   }
 
   /**

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/tools/FrequenciesJob.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/tools/FrequenciesJob.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/tools/FrequenciesJob.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/tools/FrequenciesJob.java Thu May  6 11:43:01 2010
@@ -40,6 +40,7 @@ import org.apache.hadoop.mapreduce.lib.i
 import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.mahout.common.HadoopUtil;
 import org.apache.mahout.df.DFUtils;
 import org.apache.mahout.df.data.DataConverter;
 import org.apache.mahout.df.data.Dataset;
@@ -115,9 +116,8 @@ public class FrequenciesJob {
     job.waitForCompletion(true);
     
     int[][] counts = parseOutput(job);
-    
-    // delete the output path
-    fs.delete(outputPath, true);
+
+    HadoopUtil.overwriteOutput(outputPath);
     
     return counts;
   }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/FPGrowthDriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/FPGrowthDriver.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/FPGrowthDriver.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/FPGrowthDriver.java Thu May  6 11:43:01 2010
@@ -152,11 +152,11 @@ public final class FPGrowthDriver {
         encoding = (String) cmdLine.getValue(encodingOpt);
       }
       params.set("encoding", encoding);
-      String inputDir = (String) cmdLine.getValue(inputDirOpt);
-      String outputDir = (String) cmdLine.getValue(outputOpt);
+      Path inputDir = new Path(cmdLine.getValue(inputDirOpt).toString());
+      Path outputDir = new Path(cmdLine.getValue(outputOpt).toString());
       
-      params.set("input", inputDir);
-      params.set("output", outputDir);
+      params.set("input", inputDir.toString());
+      params.set("output", outputDir.toString());
       
       String classificationMethod = (String) cmdLine.getValue(methodOpt);
       if (classificationMethod.equalsIgnoreCase("sequential")) {

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/PFPGrowth.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/PFPGrowth.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/PFPGrowth.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/PFPGrowth.java Thu May  6 11:43:01 2010
@@ -42,6 +42,7 @@ import org.apache.hadoop.mapreduce.lib.i
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
 import org.apache.hadoop.util.GenericsUtil;
+import org.apache.mahout.common.HadoopUtil;
 import org.apache.mahout.common.Pair;
 import org.apache.mahout.common.Parameters;
 import org.apache.mahout.fpm.pfpgrowth.convertors.string.TopKStringPatterns;
@@ -119,9 +120,10 @@ public final class PFPGrowth {
     LongWritable value = new LongWritable();
     int minSupport = Integer.valueOf(params.get("minSupport", "3"));
     Configuration conf = new Configuration();
-    
-    FileSystem fs = FileSystem.get(new Path(params.get("output") + "/parallelcounting").toUri(), conf);
-    FileStatus[] outputFiles = fs.globStatus(new Path(params.get("output") + "/parallelcounting/part-*"));
+
+    Path parallelCountingPath = new Path(params.get("output"), "parallelcounting");
+    FileSystem fs = FileSystem.get(parallelCountingPath.toUri(), conf);
+    FileStatus[] outputFiles = fs.globStatus(new Path(parallelCountingPath, "part-*"));
     
     PriorityQueue<Pair<String,Long>> queue = new PriorityQueue<Pair<String,Long>>(11,
         new Comparator<Pair<String,Long>>() {
@@ -141,10 +143,9 @@ public final class PFPGrowth {
       SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, conf);
       // key is feature value is count
       while (reader.next(key, value)) {
-        if (value.get() < minSupport) {
-          continue;
+        if (value.get() >= minSupport) {
+          queue.add(new Pair<String, Long>(key.toString(), value.get()));
         }
-        queue.add(new Pair<String,Long>(key.toString(), value.get()));
       }
     }
     List<Pair<String,Long>> fList = new ArrayList<Pair<String,Long>>();
@@ -164,9 +165,10 @@ public final class PFPGrowth {
   public static List<Pair<String,TopKStringPatterns>> readFrequentPattern(Parameters params) throws IOException {
     
     Configuration conf = new Configuration();
-    
-    FileSystem fs = FileSystem.get(new Path(params.get("output") + "/frequentPatterns").toUri(), conf);
-    FileStatus[] outputFiles = fs.globStatus(new Path(params.get("output") + "/frequentPatterns/part-*"));
+
+    Path frequentPatternsPath = new Path(params.get("output"), "frequentPatterns");
+    FileSystem fs = FileSystem.get(frequentPatternsPath.toUri(), conf);
+    FileStatus[] outputFiles = fs.globStatus(new Path(frequentPatternsPath, "part-*"));
     
     List<Pair<String,TopKStringPatterns>> ret = new ArrayList<Pair<String,TopKStringPatterns>>();
     for (FileStatus fileStatus : outputFiles) {
@@ -223,7 +225,7 @@ public final class PFPGrowth {
     job.setOutputValueClass(TopKStringPatterns.class);
     
     FileInputFormat.addInputPath(job, new Path(input));
-    Path outPath = new Path(params.get("output") + "/frequentPatterns");
+    Path outPath = new Path(params.get("output"), "frequentPatterns");
     FileOutputFormat.setOutputPath(job, outPath);
     
     job.setInputFormatClass(SequenceFileInputFormat.class);
@@ -231,11 +233,8 @@ public final class PFPGrowth {
     job.setCombinerClass(AggregatorReducer.class);
     job.setReducerClass(AggregatorReducer.class);
     job.setOutputFormatClass(SequenceFileOutputFormat.class);
-    
-    FileSystem dfs = FileSystem.get(outPath.toUri(), conf);
-    if (dfs.exists(outPath)) {
-      dfs.delete(outPath, true);
-    }
+
+    HadoopUtil.overwriteOutput(outPath);
     job.waitForCompletion(true);
   }
   
@@ -301,13 +300,10 @@ public final class PFPGrowth {
     job.setOutputValueClass(LongWritable.class);
     
     FileInputFormat.addInputPath(job, new Path(input));
-    Path outPath = new Path(params.get("output") + "/parallelcounting");
+    Path outPath = new Path(params.get("output"), "parallelcounting");
     FileOutputFormat.setOutputPath(job, outPath);
     
-    FileSystem dfs = FileSystem.get(outPath.toUri(), conf);
-    if (dfs.exists(outPath)) {
-      dfs.delete(outPath, true);
-    }
+    HadoopUtil.overwriteOutput(outPath);
     
     job.setInputFormatClass(TextInputFormat.class);
     job.setMapperClass(ParallelCountingMapper.class);
@@ -351,10 +347,7 @@ public final class PFPGrowth {
     Path outPath = new Path(params.get("output") + "/sortedoutput");
     FileOutputFormat.setOutputPath(job, outPath);
     
-    FileSystem dfs = FileSystem.get(outPath.toUri(), conf);
-    if (dfs.exists(outPath)) {
-      dfs.delete(outPath, true);
-    }
+    HadoopUtil.overwriteOutput(outPath);
     
     job.setInputFormatClass(TextInputFormat.class);
     job.setMapperClass(TransactionSortingMapper.class);
@@ -392,13 +385,10 @@ public final class PFPGrowth {
     job.setOutputValueClass(TopKStringPatterns.class);
     
     FileInputFormat.addInputPath(job, new Path(input));
-    Path outPath = new Path(params.get("output") + "/fpgrowth");
+    Path outPath = new Path(new Path(params.get("output")), "fpgrowth");
     FileOutputFormat.setOutputPath(job, outPath);
     
-    FileSystem dfs = FileSystem.get(outPath.toUri(), conf);
-    if (dfs.exists(outPath)) {
-      dfs.delete(outPath, true);
-    }
+    HadoopUtil.overwriteOutput(outPath);
     
     job.setInputFormatClass(SequenceFileInputFormat.class);
     job.setMapperClass(ParallelFPGrowthMapper.class);

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/ga/watchmaker/MahoutEvaluator.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/ga/watchmaker/MahoutEvaluator.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/ga/watchmaker/MahoutEvaluator.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/ga/watchmaker/MahoutEvaluator.java Thu May  6 11:43:01 2010
@@ -33,6 +33,7 @@ import org.apache.hadoop.mapred.JobClien
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.mahout.common.HadoopUtil;
 import org.apache.mahout.common.StringUtils;
 import org.uncommons.watchmaker.framework.FitnessEvaluator;
 
@@ -59,7 +60,7 @@ public class MahoutEvaluator {
     JobConf conf = new JobConf(MahoutEvaluator.class);
     FileSystem fs = FileSystem.get(conf);
     Path inpath = prepareInput(fs, population);
-    Path outpath = OutputUtils.prepareOutput(fs);
+    Path outpath = new Path("output");
     
     configureJob(conf, evaluator, inpath, outpath);
     JobClient.runJob(conf);
@@ -78,16 +79,8 @@ public class MahoutEvaluator {
    */
   private static Path prepareInput(FileSystem fs, List<?> population) throws IOException {
     Path inpath = new Path(fs.getWorkingDirectory(), "input");
-    
-    // Delete the input if it already exists
-    if (fs.exists(inpath)) {
-      fs.delete(inpath, true);
-    }
-    
-    fs.mkdirs(inpath);
-    
+    HadoopUtil.overwriteOutput(inpath);
     storePopulation(fs, new Path(inpath, "population"), population);
-    
     return inpath;
   }
   

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/ga/watchmaker/OutputUtils.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/ga/watchmaker/OutputUtils.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/ga/watchmaker/OutputUtils.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/ga/watchmaker/OutputUtils.java Thu May  6 11:43:01 2010
@@ -38,23 +38,6 @@ public final class OutputUtils {
   }
   
   /**
-   * Removes the output directory if it already exists.
-   * 
-   * @param fs
-   *          <code>FileSystem</code> to use
-   * @return output <code>Path</code>
-   */
-  public static Path prepareOutput(FileSystem fs) throws IOException {
-    Path outpath = new Path(fs.getWorkingDirectory(), "output");
-    
-    if (fs.exists(outpath)) {
-      fs.delete(outpath, true);
-    }
-    
-    return outpath;
-  }
-  
-  /**
    * Lists all files in the output <code>Path</code>
    * 
    * @param fs

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/DistributedRowMatrix.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/DistributedRowMatrix.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/DistributedRowMatrix.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/DistributedRowMatrix.java Thu May  6 11:43:01 2010
@@ -152,7 +152,7 @@ public class DistributedRowMatrix implem
   }
 
   public DistributedRowMatrix transpose() {
-    Path outputPath = new Path(rowPath.getParent(), "transpose-" + (byte)System.nanoTime());
+    Path outputPath = new Path(rowPath.getParent(), "transpose-" + (System.nanoTime() & 0xFF));
     try {
       JobConf conf = TransposeJob.buildTransposeJobConf(rowPath, outputPath, numRows);
       JobClient.runJob(conf);

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/file/FileDataModelTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/file/FileDataModelTest.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/file/FileDataModelTest.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/cf/taste/impl/model/file/FileDataModelTest.java Thu May  6 11:43:01 2010
@@ -61,25 +61,11 @@ public final class FileDataModelTest ext
 
   private DataModel model;
   private File testFile;
-  private File tmpLoc;
 
   @Override
   public void setUp() throws Exception {
     super.setUp();
-    File tmpDir = new File(System.getProperty("java.io.tmpdir"));
-    tmpLoc = new File(tmpDir, "fileDataModel");
-    tmpLoc.deleteOnExit();
-    if (tmpLoc.exists()) {
-      if (tmpLoc.isFile()) {
-        throw new IOException("Temp directory is a file");
-      }
-    } else {
-      if (!tmpLoc.mkdirs()) {
-        throw new IOException("Could not create temp directory");
-      }
-    }
-    testFile = File.createTempFile("test", ".txt", tmpLoc);
-    testFile.deleteOnExit();
+    testFile = getTestTempFile("test.txt");
     PrintWriter writer =
         new PrintWriter(new OutputStreamWriter(new FileOutputStream(testFile), Charset.forName("UTF-8")));
     try {
@@ -92,13 +78,6 @@ public final class FileDataModelTest ext
     model = new FileDataModel(testFile);
   }
 
-  @Override
-  public void tearDown() throws Exception {
-    testFile.delete();
-    tmpLoc.delete();
-    super.tearDown();
-  }
-
   public void testFile() throws Exception {
     UserSimilarity userSimilarity = new PearsonCorrelationSimilarity(model);
     UserNeighborhood neighborhood = new NearestNUserNeighborhood(3, userSimilarity, model);

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/classifier/ClassifierData.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/classifier/ClassifierData.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/classifier/ClassifierData.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/classifier/ClassifierData.java Thu May  6 11:43:01 2010
@@ -17,16 +17,6 @@
 
 package org.apache.mahout.classifier;
 
-import java.io.BufferedWriter;
-import java.io.FileWriter;
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
-
 /**
  * Class containing sample docs from ASF websites under mahout, lucene and spamassasin projects
  *
@@ -108,23 +98,5 @@ public final class ClassifierData {
 
 
   private ClassifierData() { }
-  
-  public static void writeDataToFile(String file, String[][] content) throws IOException {
-    BufferedWriter writer = new BufferedWriter(new FileWriter(file));
-    for (String[] entry : content) {
-      writer.write(entry[0] + '\t' + entry[1] + '\n');
-    }
-    writer.close();
-  }
 
-  public static void writeDataToSequenceFile(String file,
-                                             String[][] content,
-                                             FileSystem fs,
-                                             Configuration conf) throws IOException {
-    SequenceFile.Writer writer = new SequenceFile.Writer(fs, conf, new Path(file), Text.class, Text.class);
-    for (String[] entry : content) {
-      writer.append(new Text(entry[0]), new Text(entry[1]));
-    }
-    writer.close();
-  }
 }

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/classifier/bayes/BayesClassifierSelfTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/classifier/bayes/BayesClassifierSelfTest.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/classifier/bayes/BayesClassifierSelfTest.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/classifier/bayes/BayesClassifierSelfTest.java Thu May  6 11:43:01 2010
@@ -17,12 +17,21 @@
 
 package org.apache.mahout.classifier.bayes;
 
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileWriter;
 import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.nio.charset.Charset;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
 import org.apache.mahout.classifier.ClassifierData;
 import org.apache.mahout.classifier.ClassifierResult;
 import org.apache.mahout.classifier.ResultAnalyzer;
@@ -43,17 +52,31 @@ public class BayesClassifierSelfTest ext
   @Override
   protected void setUp() throws Exception {
     super.setUp();
-    ClassifierData.writeDataToFile("testdata/bayesinput", ClassifierData.DATA);
+
+    File tempInputFile = getTestTempFile("bayesinput");
+    BufferedWriter writer = new BufferedWriter(
+        new OutputStreamWriter(new FileOutputStream(tempInputFile), Charset.forName("UTF-8")));
+    for (String[] entry : ClassifierData.DATA) {
+      writer.write(entry[0] + '\t' + entry[1] + '\n');
+    }
+    writer.close();
+
+    Path input = getTestTempFilePath("bayesinput");
+    Configuration conf = new Configuration();
+    FileSystem fs = input.getFileSystem(conf);
+    fs.copyFromLocalFile(new Path(tempInputFile.getAbsolutePath()), input);
   }
   
   public void testSelfTestBayes() throws InvalidDatastoreException, IOException {
     BayesParameters params = new BayesParameters(1);
     params.set("alpha_i", "1.0");
     params.set("dataSource", "hdfs");
-    TrainClassifier.trainNaiveBayes("testdata/bayesinput", "testdata/bayesmodel", params);
+    Path bayesInputPath = getTestTempFilePath("bayesinput");
+    Path bayesModelPath = getTestTempDirPath("bayesmodel");
+    TrainClassifier.trainNaiveBayes(bayesInputPath, bayesModelPath, params);
     
     params.set("verbose", "true");
-    params.set("basePath", "testdata/bayesmodel");
+    params.set("basePath", bayesModelPath.toString());
     params.set("classifierType", "bayes");
     params.set("dataSource", "hdfs");
     params.set("defaultCat", "unknown");
@@ -82,10 +105,10 @@ public class BayesClassifierSelfTest ext
         assertEquals(i == j ? 4 : 0, matrix[i][j]);
       }
     }
-    params.set("testDirPath", "testdata/bayesinput");
+    params.set("testDirPath", bayesInputPath.toString());
     TestClassifier.classifyParallel(params);
     Configuration conf = new Configuration();
-    Path outputFiles = new Path("testdata/bayesinput-output/part*");
+    Path outputFiles = getTestTempFilePath("bayesinput-output/part*");
     FileSystem fs = FileSystem.get(outputFiles.toUri(), conf);
     matrix = BayesClassifierDriver.readResult(fs, outputFiles, conf, params).getConfusionMatrix();
     for (int i = 0; i < 3; i++) {
@@ -99,10 +122,12 @@ public class BayesClassifierSelfTest ext
     BayesParameters params = new BayesParameters(1);
     params.set("alpha_i", "1.0");
     params.set("dataSource", "hdfs");
-    TrainClassifier.trainCNaiveBayes("testdata/bayesinput", "testdata/cbayesmodel", params);
+    Path bayesInputPath = getTestTempFilePath("bayesinput");
+    Path bayesModelPath = getTestTempDirPath("cbayesmodel");
+    TrainClassifier.trainCNaiveBayes(bayesInputPath, bayesModelPath, params);
     
     params.set("verbose", "true");
-    params.set("basePath", "testdata/cbayesmodel");
+    params.set("basePath", bayesModelPath.toString());
     params.set("classifierType", "cbayes");
     params.set("dataSource", "hdfs");
     params.set("defaultCat", "unknown");
@@ -130,10 +155,10 @@ public class BayesClassifierSelfTest ext
         assertEquals(i == j ? 4 : 0, matrix[i][j]);
       }
     }
-    params.set("testDirPath", "testdata/bayesinput");
+    params.set("testDirPath", bayesInputPath.toString());
     TestClassifier.classifyParallel(params);
     Configuration conf = new Configuration();
-    Path outputFiles = new Path("testdata/bayesinput-output/part*");
+    Path outputFiles = getTestTempFilePath("bayesinput-output/part*");
     FileSystem fs = FileSystem.get(outputFiles.toUri(), conf);
     matrix = BayesClassifierDriver.readResult(fs, outputFiles, conf, params).getConfusionMatrix();
     for (int i = 0; i < 3; i++) {

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/ClusteringTestUtils.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/ClusteringTestUtils.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/ClusteringTestUtils.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/ClusteringTestUtils.java Thu May  6 11:43:01 2010
@@ -33,18 +33,17 @@ public class ClusteringTestUtils {
   }
 
   public static void writePointsToFile(Iterable<VectorWritable> points,
-                                       String fileName,
+                                       Path path,
                                        FileSystem fs,
                                        Configuration conf) throws IOException {
-    writePointsToFile(points, false, fileName, fs, conf);
+    writePointsToFile(points, false, path, fs, conf);
   }
 
   public static void writePointsToFile(Iterable<VectorWritable> points,
                                        boolean intWritable,
-                                       String fileName,
+                                       Path path,
                                        FileSystem fs,
                                        Configuration conf) throws IOException {
-    Path path = new Path(fileName);
     SequenceFile.Writer writer = new SequenceFile.Writer(fs,
                                                          conf,
                                                          path,

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/canopy/TestCanopyCreation.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/canopy/TestCanopyCreation.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/canopy/TestCanopyCreation.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/canopy/TestCanopyCreation.java Thu May  6 11:43:01 2010
@@ -46,7 +46,9 @@ import org.apache.mahout.math.VectorWrit
 
 public class TestCanopyCreation extends MahoutTestCase {
 
-  private static final double[][] raw = { { 1, 1 }, { 2, 1 }, { 1, 2 }, { 2, 2 }, { 3, 3 }, { 4, 4 }, { 5, 4 }, { 4, 5 }, { 5, 5 } };
+  private static final double[][] raw = {
+      { 1, 1 }, { 2, 1 }, { 1, 2 }, { 2, 2 }, { 3, 3 }, 
+      { 4, 4 }, { 5, 4 }, { 4, 5 }, { 5, 5 } };
 
   private List<Canopy> referenceManhattan;
 
@@ -321,19 +323,17 @@ public class TestCanopyCreation extends 
    */
   public void testCanopyGenManhattanMR() throws Exception {
     List<VectorWritable> points = getPointsWritable();
-    File testData = new File("testdata");
-    if (!testData.exists()) {
-      testData.mkdir();
-    }
     JobConf job = new JobConf(CanopyDriver.class);
     job.setMapOutputValueClass(points.get(0).getClass());
-    ClusteringTestUtils.writePointsToFile(points, "testdata/file1", fs, job);
-    ClusteringTestUtils.writePointsToFile(points, "testdata/file2", fs, job);
+    ClusteringTestUtils.writePointsToFile(points, getTestTempFilePath("testdata/file1"), fs, job);
+    ClusteringTestUtils.writePointsToFile(points, getTestTempFilePath("testdata/file2"), fs, job);
     // now run the Canopy Driver
-    CanopyDriver.runJob("testdata", "output", ManhattanDistanceMeasure.class.getName(), 3.1, 2.1, false);
+    Path output = getTestTempDirPath("output");
+    CanopyDriver.runJob(getTestTempDirPath("testdata"), output,
+                        ManhattanDistanceMeasure.class.getName(), 3.1, 2.1, false);
 
     // verify output from sequence file
-    Path path = new Path("output/clusters-0/part-00000");
+    Path path = new Path(output, "clusters-0/part-00000");
     FileSystem fs = FileSystem.get(path.toUri(), job);
     SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, job);
     Text key = new Text();
@@ -358,18 +358,15 @@ public class TestCanopyCreation extends 
    */
   public void testCanopyGenEuclideanMR() throws Exception {
     List<VectorWritable> points = getPointsWritable();
-    File testData = new File("testdata");
-    if (!testData.exists()) {
-      testData.mkdir();
-    }
     JobConf job = new JobConf(CanopyDriver.class);
-    ClusteringTestUtils.writePointsToFile(points, "testdata/file1", fs, job);
-    ClusteringTestUtils.writePointsToFile(points, "testdata/file2", fs, job);
+    ClusteringTestUtils.writePointsToFile(points, getTestTempFilePath("testdata/file1"), fs, job);
+    ClusteringTestUtils.writePointsToFile(points, getTestTempFilePath("testdata/file2"), fs, job);
     // now run the Canopy Driver
-    CanopyDriver.runJob("testdata", "output", EuclideanDistanceMeasure.class.getName(), 3.1, 2.1, false);
+    Path output = getTestTempDirPath("output");
+    CanopyDriver.runJob(getTestTempDirPath("testdata"), output, EuclideanDistanceMeasure.class.getName(), 3.1, 2.1, false);
 
     // verify output from sequence file
-    Path path = new Path("output/clusters-0/part-00000");
+    Path path = new Path(output, "clusters-0/part-00000");
     FileSystem fs = FileSystem.get(path.toUri(), job);
     SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, job);
     Text key = new Text();
@@ -467,18 +464,14 @@ public class TestCanopyCreation extends 
    */
   public void testClusteringManhattanMR() throws Exception {
     List<VectorWritable> points = getPointsWritable();
-    File testData = new File("testdata");
-    if (!testData.exists()) {
-      testData.mkdir();
-    }
     JobConf conf = new JobConf();
     conf.setMapOutputValueClass(points.get(0).getClass());
-    ClusteringTestUtils.writePointsToFile(points, "testdata/file1", fs, conf);
-    ClusteringTestUtils.writePointsToFile(points, "testdata/file2", fs, conf);
+    ClusteringTestUtils.writePointsToFile(points, getTestTempFilePath("testdata/file1"), fs, conf);
+    ClusteringTestUtils.writePointsToFile(points, getTestTempFilePath("testdata/file2"), fs, conf);
     // now run the Job
-    CanopyDriver.runJob("testdata", "output", ManhattanDistanceMeasure.class.getName(), 3.1, 2.1, true);
-    // TODO: change
-    Path path = new Path("output/clusteredPoints/part-00000");
+    Path output = getTestTempDirPath("output");
+    CanopyDriver.runJob(getTestTempDirPath("testdata"), output, ManhattanDistanceMeasure.class.getName(), 3.1, 2.1, true);
+    Path path = new Path(output, "clusteredPoints/part-00000");
     SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, conf);
     int count = 0;
     /*
@@ -500,16 +493,13 @@ public class TestCanopyCreation extends 
    */
   public void testClusteringEuclideanMR() throws Exception {
     List<VectorWritable> points = getPointsWritable();
-    File testData = new File("testdata");
-    if (!testData.exists()) {
-      testData.mkdir();
-    }
     Configuration conf = new Configuration();
-    ClusteringTestUtils.writePointsToFile(points, "testdata/file1", fs, conf);
-    ClusteringTestUtils.writePointsToFile(points, "testdata/file2", fs, conf);
+    ClusteringTestUtils.writePointsToFile(points, getTestTempFilePath("testdata/file1"), fs, conf);
+    ClusteringTestUtils.writePointsToFile(points, getTestTempFilePath("testdata/file2"), fs, conf);
     // now run the Job
-    CanopyDriver.runJob("testdata", "output", EuclideanDistanceMeasure.class.getName(), 3.1, 2.1, true);
-    Path path = new Path("output/clusteredPoints/part-00000");
+    Path output = getTestTempDirPath("output");
+    CanopyDriver.runJob(getTestTempDirPath("testdata"), output, EuclideanDistanceMeasure.class.getName(), 3.1, 2.1, true);
+    Path path = new Path(output, "clusteredPoints/part-00000");
     SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, conf);
     int count = 0;
     IntWritable canopyId = new IntWritable(0);
@@ -524,20 +514,17 @@ public class TestCanopyCreation extends 
   /** Story: Clustering algorithm must support arbitrary user defined distance measure */
   public void testUserDefinedDistanceMeasure() throws Exception {
     List<VectorWritable> points = getPointsWritable();
-    File testData = new File("testdata");
-    if (!testData.exists()) {
-      testData.mkdir();
-    }
     Configuration conf = new Configuration();
-    ClusteringTestUtils.writePointsToFile(points, "testdata/file1", fs, conf);
-    ClusteringTestUtils.writePointsToFile(points, "testdata/file2", fs, conf);
+    ClusteringTestUtils.writePointsToFile(points, getTestTempFilePath("testdata/file1"), fs, conf);
+    ClusteringTestUtils.writePointsToFile(points, getTestTempFilePath("testdata/file2"), fs, conf);
     // now run the Canopy Driver. User defined measure happens to be a Manhattan
     // subclass so results are same.
-    CanopyDriver.runJob("testdata", "output", UserDefinedDistanceMeasure.class.getName(), 3.1, 2.1, false);
+    Path output = getTestTempDirPath("output");
+    CanopyDriver.runJob(getTestTempDirPath("testdata"), output, UserDefinedDistanceMeasure.class.getName(), 3.1, 2.1, false);
 
     // verify output from sequence file
     JobConf job = new JobConf(CanopyDriver.class);
-    Path path = new Path("output/clusters-0/part-00000");
+    Path path = new Path(output, "clusters-0/part-00000");
     FileSystem fs = FileSystem.get(path.toUri(), job);
     SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, job);
     Text key = new Text();

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestMapReduce.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestMapReduce.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestMapReduce.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestMapReduce.java Thu May  6 11:43:01 2010
@@ -209,17 +209,13 @@ public class TestMapReduce extends Mahou
   
   /** Test the Mapper and Reducer using the Driver */
   public void testDriverMRIterations() throws Exception {
-    File f = new File("input");
-    for (File g : f.listFiles()) {
-      g.delete();
-    }
     generateSamples(100, 0, 0, 0.5);
     generateSamples(100, 2, 0, 0.2);
     generateSamples(100, 0, 2, 0.3);
     generateSamples(100, 2, 2, 1);
-    ClusteringTestUtils.writePointsToFile(sampleData, "input/data.txt", fs, conf);
+    ClusteringTestUtils.writePointsToFile(sampleData, getTestTempFilePath("input/data.txt"), fs, conf);
     // Now run the driver
-    DirichletDriver.runJob("input", "output",
+    DirichletDriver.runJob(getTestTempDirPath("input"), getTestTempDirPath("output"),
       "org.apache.mahout.clustering.dirichlet.models.SampledNormalDistribution", 20, 5, 1.0, 1);
     // and inspect results
     List<List<DirichletCluster<VectorWritable>>> clusters = new ArrayList<List<DirichletCluster<VectorWritable>>>();
@@ -255,13 +251,9 @@ public class TestMapReduce extends Mahou
   
   /** Test the Mapper and Reducer using the Driver */
   public void testDriverMnRIterations() throws Exception {
-    File f = new File("input");
-    for (File g : f.listFiles()) {
-      g.delete();
-    }
     generate4Datasets();
     // Now run the driver
-    DirichletDriver.runJob("input", "output",
+    DirichletDriver.runJob(getTestTempDirPath("input"), getTestTempDirPath("output"),
       "org.apache.mahout.clustering.dirichlet.models.SampledNormalDistribution", 20, 3, 1.0, 1);
     // and inspect results
     List<List<DirichletCluster<VectorWritable>>> clusters = new ArrayList<List<DirichletCluster<VectorWritable>>>();
@@ -281,27 +273,23 @@ public class TestMapReduce extends Mahou
   
   private void generate4Datasets() throws IOException {
     generateSamples(500, 0, 0, 0.5);
-    ClusteringTestUtils.writePointsToFile(sampleData, "input/data1.txt", fs, conf);
+    ClusteringTestUtils.writePointsToFile(sampleData, getTestTempFilePath("input/data1.txt"), fs, conf);
     sampleData = new ArrayList<VectorWritable>();
     generateSamples(500, 2, 0, 0.2);
-    ClusteringTestUtils.writePointsToFile(sampleData, "input/data2.txt", fs, conf);
+    ClusteringTestUtils.writePointsToFile(sampleData, getTestTempFilePath("input/data2.txt"), fs, conf);
     sampleData = new ArrayList<VectorWritable>();
     generateSamples(500, 0, 2, 0.3);
-    ClusteringTestUtils.writePointsToFile(sampleData, "input/data3.txt", fs, conf);
+    ClusteringTestUtils.writePointsToFile(sampleData, getTestTempFilePath("input/data3.txt"), fs, conf);
     sampleData = new ArrayList<VectorWritable>();
     generateSamples(500, 2, 2, 1);
-    ClusteringTestUtils.writePointsToFile(sampleData, "input/data4.txt", fs, conf);
+    ClusteringTestUtils.writePointsToFile(sampleData, getTestTempFilePath("input/data4.txt"), fs, conf);
   }
   
   /** Test the Mapper and Reducer using the Driver */
   public void testDriverMnRnIterations() throws Exception {
-    File f = new File("input");
-    for (File g : f.listFiles()) {
-      g.delete();
-    }
     generate4Datasets();
     // Now run the driver
-    DirichletDriver.runJob("input", "output",
+    DirichletDriver.runJob(getTestTempDirPath("input"), getTestTempDirPath("output"),
       "org.apache.mahout.clustering.dirichlet.models.SampledNormalDistribution", 20, 3, 1.0, 2);
     // and inspect results
     List<List<DirichletCluster<VectorWritable>>> clusters = new ArrayList<List<DirichletCluster<VectorWritable>>>();
@@ -326,18 +314,18 @@ public class TestMapReduce extends Mahou
       g.delete();
     }
     generateSamples(500, 0, 0, 0.5, 1.0);
-    ClusteringTestUtils.writePointsToFile(sampleData, "input/data1.txt", fs, conf);
+    ClusteringTestUtils.writePointsToFile(sampleData, getTestTempFilePath("input/data1.txt"), fs, conf);
     sampleData = new ArrayList<VectorWritable>();
     generateSamples(500, 2, 0, 0.2);
-    ClusteringTestUtils.writePointsToFile(sampleData, "input/data2.txt", fs, conf);
+    ClusteringTestUtils.writePointsToFile(sampleData, getTestTempFilePath("input/data2.txt"), fs, conf);
     sampleData = new ArrayList<VectorWritable>();
     generateSamples(500, 0, 2, 0.3);
-    ClusteringTestUtils.writePointsToFile(sampleData, "input/data3.txt", fs, conf);
+    ClusteringTestUtils.writePointsToFile(sampleData, getTestTempFilePath("input/data3.txt"), fs, conf);
     sampleData = new ArrayList<VectorWritable>();
     generateSamples(500, 2, 2, 1);
-    ClusteringTestUtils.writePointsToFile(sampleData, "input/data4.txt", fs, conf);
+    ClusteringTestUtils.writePointsToFile(sampleData, getTestTempFilePath("input/data4.txt"), fs, conf);
     // Now run the driver
-    DirichletDriver.runJob("input", "output",
+    DirichletDriver.runJob(getTestTempDirPath("input"), getTestTempDirPath("output"),
       "org.apache.mahout.clustering.dirichlet.models.AsymmetricSampledNormalDistribution", 20, 3, 1.0, 2);
     // and inspect results
     List<List<DirichletCluster<VectorWritable>>> clusters = new ArrayList<List<DirichletCluster<VectorWritable>>>();

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/fuzzykmeans/TestFuzzyKmeansClustering.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/fuzzykmeans/TestFuzzyKmeansClustering.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/fuzzykmeans/TestFuzzyKmeansClustering.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/fuzzykmeans/TestFuzzyKmeansClustering.java Thu May  6 11:43:01 2010
@@ -171,37 +171,16 @@ public class TestFuzzyKmeansClustering e
 
   public void testFuzzyKMeansMRJob() throws Exception {
     List<VectorWritable> points = TestKmeansClustering.getPointsWritable(TestKmeansClustering.reference);
-    File testData = new File("testdata");
-    if (!testData.exists()) {
-      testData.mkdir();
-    }
-    testData = new File("testdata/points");
-    if (!testData.exists()) {
-      testData.mkdir();
-    }
+
+    Path pointsPath = getTestTempDirPath("points");
+    Path clustersPath = getTestTempDirPath("clusters");
     Configuration conf = new Configuration();
-    ClusteringTestUtils.writePointsToFile(points, "testdata/points/file1", fs, conf);
+    ClusteringTestUtils.writePointsToFile(points, new Path(pointsPath, "file1"), fs, conf);
 
     for (int k = 0; k < points.size(); k++) {
       System.out.println("testKFuzzyKMeansMRJob k= " + k);
       // pick k initial cluster centers at random
-      JobConf job = new JobConf(FuzzyKMeansDriver.class);
-      Path path = new Path("testdata/clusters");
-      FileSystem fs = FileSystem.get(path.toUri(), job);
-      if (fs.exists(path)) {
-        fs.delete(path, true);
-      }
-
-      testData = new File("testdata/clusters");
-      if (!testData.exists()) {
-        testData.mkdir();
-      }
-
-      /*
-       * BufferedWriter writer = new BufferedWriter(new OutputStreamWriter( new
-       * FileOutputStream("testdata/clusters/part-00000"), Charset .forName("UTF-8")));
-       */
-      SequenceFile.Writer writer = new SequenceFile.Writer(fs, conf, new Path("testdata/clusters/part-00000"), Text.class,
+      SequenceFile.Writer writer = new SequenceFile.Writer(fs, conf, new Path(clustersPath, "part-00000"), Text.class,
           SoftCluster.class);
       for (int i = 0; i < k + 1; i++) {
         Vector vec = tweakValue(points.get(i).get());
@@ -217,22 +196,12 @@ public class TestFuzzyKmeansClustering e
       }
       writer.close();
 
-      Path outPath = new Path("output");
-      fs = FileSystem.get(outPath.toUri(), conf);
-      if (fs.exists(outPath)) {
-        fs.delete(outPath, true);
-      }
-      fs.mkdirs(outPath);
       // now run the Job
-      FuzzyKMeansDriver.runJob("testdata/points", "testdata/clusters", "output", EuclideanDistanceMeasure.class.getName(), 0.001,
+      Path output = getTestTempDirPath("output");
+      FuzzyKMeansDriver.runJob(pointsPath, clustersPath, output, EuclideanDistanceMeasure.class.getName(), 0.001,
           2, 1, k + 1, 2, false, true, 0);
 
-      // now compare the expected clusters with actual
-      File outDir = new File("output/clusteredPoints");
-      assertTrue("output dir exists?", outDir.exists());
-      outDir.list();
-      // assertEquals("output dir files?", 4, outFiles.length);
-      SequenceFile.Reader reader = new SequenceFile.Reader(fs, new Path("output/clusteredPoints/part-00000"), conf);
+      SequenceFile.Reader reader = new SequenceFile.Reader(fs, new Path(output, "clusteredPoints/part-00000"), conf);
       IntWritable key = new IntWritable();
       WeightedVectorWritable out = new WeightedVectorWritable();
       while (reader.next(key, out)) {

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java Thu May  6 11:43:01 2010
@@ -35,7 +35,6 @@ import org.apache.mahout.clustering.Weig
 import org.apache.mahout.clustering.canopy.CanopyDriver;
 import org.apache.mahout.common.DummyOutputCollector;
 import org.apache.mahout.common.DummyReporter;
-import org.apache.mahout.common.HadoopUtil;
 import org.apache.mahout.common.MahoutTestCase;
 import org.apache.mahout.common.distance.DistanceMeasure;
 import org.apache.mahout.common.distance.EuclideanDistanceMeasure;
@@ -48,11 +47,15 @@ import org.apache.mahout.math.VectorWrit
 
 public class TestKmeansClustering extends MahoutTestCase {
 
-  public static final double[][] reference = { { 1, 1 }, { 2, 1 }, { 1, 2 }, { 2, 2 }, { 3, 3 }, { 4, 4 }, { 5, 4 }, { 4, 5 },
-      { 5, 5 } };
-
-  private static final int[][] expectedNumPoints = { { 9 }, { 4, 5 }, { 4, 4, 1 }, { 1, 2, 1, 5 }, { 1, 1, 1, 2, 4 },
-      { 1, 1, 1, 1, 1, 4 }, { 1, 1, 1, 1, 1, 2, 2 }, { 1, 1, 1, 1, 1, 1, 2, 1 }, { 1, 1, 1, 1, 1, 1, 1, 1, 1 } };
+  public static final double[][] reference =
+      { { 1, 1 }, { 2, 1 }, { 1, 2 }, { 2, 2 }, { 3, 3 },
+        { 4, 4 }, { 5, 4 }, { 4, 5 }, { 5, 5 } };
+
+  private static final int[][] expectedNumPoints = {
+      { 9 }, { 4, 5 }, { 4, 4, 1 },
+      { 1, 2, 1, 5 }, { 1, 1, 1, 2, 4 },
+      { 1, 1, 1, 1, 1, 4 }, { 1, 1, 1, 1, 1, 2, 2 },
+      { 1, 1, 1, 1, 1, 1, 2, 1 }, { 1, 1, 1, 1, 1, 1, 1, 1, 1 } };
 
   private FileSystem fs;
 
@@ -332,23 +335,17 @@ public class TestKmeansClustering extend
   /** Story: User wishes to run kmeans job on reference data */
   public void testKMeansMRJob() throws Exception {
     List<VectorWritable> points = getPointsWritable(reference);
-    File testData = new File("testdata");
-    if (!testData.exists()) {
-      testData.mkdir();
-    }
-    testData = new File("testdata/points");
-    if (!testData.exists()) {
-      testData.mkdir();
-    }
 
+    Path pointsPath = getTestTempDirPath("points");
+    Path clustersPath = getTestTempDirPath("clusters");
     Configuration conf = new Configuration();
-    ClusteringTestUtils.writePointsToFile(points, "testdata/points/file1", fs, conf);
-    ClusteringTestUtils.writePointsToFile(points, "testdata/points/file2", fs, conf);
+    ClusteringTestUtils.writePointsToFile(points, new Path(pointsPath, "file1"), fs, conf);
+    ClusteringTestUtils.writePointsToFile(points, new Path(pointsPath, "file2"), fs, conf);
     for (int k = 1; k < points.size(); k++) {
       System.out.println("testKMeansMRJob k= " + k);
       // pick k initial cluster centers at random
       JobConf job = new JobConf(KMeansDriver.class);
-      Path path = new Path("testdata/clusters/part-00000");
+      Path path = new Path(clustersPath, "part-00000");
       FileSystem fs = FileSystem.get(path.toUri(), job);
       SequenceFile.Writer writer = new SequenceFile.Writer(fs, job, path, Text.class, Cluster.class);
 
@@ -362,14 +359,13 @@ public class TestKmeansClustering extend
       }
       writer.close();
       // now run the Job
-      HadoopUtil.overwriteOutput("output");
-      KMeansDriver.runJob("testdata/points", "testdata/clusters", "output", EuclideanDistanceMeasure.class.getName(), 0.001, 10,
+      Path outputPath = getTestTempDirPath("output");
+      KMeansDriver.runJob(pointsPath, clustersPath, outputPath, EuclideanDistanceMeasure.class.getName(), 0.001, 10,
           k + 1, true);
       // now compare the expected clusters with actual
-      File outDir = new File("output/clusteredPoints");
-      assertTrue("output dir exists?", outDir.exists());
+      Path clusteredPointsPath = new Path(outputPath, "clusteredPoints");
       // assertEquals("output dir files?", 4, outFiles.length);
-      SequenceFile.Reader reader = new SequenceFile.Reader(fs, new Path("output/clusteredPoints/part-00000"), conf);
+      SequenceFile.Reader reader = new SequenceFile.Reader(fs, new Path(clusteredPointsPath, "part-00000"), conf);
       int[] expect = expectedNumPoints[k];
       DummyOutputCollector<IntWritable, WeightedVectorWritable> collector = new DummyOutputCollector<IntWritable, WeightedVectorWritable>();
       // The key is the clusterId
@@ -380,12 +376,10 @@ public class TestKmeansClustering extend
         collector.collect(clusterId, value);
         clusterId = new IntWritable(0);
         value = new WeightedVectorWritable();
-
       }
       reader.close();
-      if (k == 2)
+      if (k == 2) {
       // cluster 3 is empty so won't appear in output
-      {
         assertEquals("clusters[" + k + ']', expect.length - 1, collector.getKeys().size());
       } else {
         assertEquals("clusters[" + k + ']', expect.length, collector.getKeys().size());
@@ -396,31 +390,26 @@ public class TestKmeansClustering extend
   /** Story: User wants to use canopy clustering to input the initial clusters for kmeans job. */
   public void testKMeansWithCanopyClusterInput() throws Exception {
     List<VectorWritable> points = getPointsWritable(reference);
-    File testData = new File("testdata");
-    if (!testData.exists()) {
-      testData.mkdir();
-    }
-    testData = new File("testdata/points");
-    if (!testData.exists()) {
-      testData.mkdir();
-    }
+
+    Path pointsPath = getTestTempDirPath("points");
     Configuration conf = new Configuration();
-    ClusteringTestUtils.writePointsToFile(points, "testdata/points/file1", fs, conf);
-    ClusteringTestUtils.writePointsToFile(points, "testdata/points/file2", fs, conf);
+    ClusteringTestUtils.writePointsToFile(points, new Path(pointsPath, "file1"), fs, conf);
+    ClusteringTestUtils.writePointsToFile(points, new Path(pointsPath, "file2"), fs, conf);
 
+    Path outputPath = getTestTempDirPath("output");
     // now run the Canopy job
-    CanopyDriver.runJob("testdata/points", "output", ManhattanDistanceMeasure.class.getName(), 3.1, 2.1, false);
+    CanopyDriver.runJob(pointsPath, outputPath, ManhattanDistanceMeasure.class.getName(), 3.1, 2.1, false);
 
     // now run the KMeans job
-    KMeansDriver.runJob("testdata/points", "output/clusters-0", "output", EuclideanDistanceMeasure.class.getName(), 0.001, 10, 1, true);
+    KMeansDriver.runJob(pointsPath, new Path(outputPath, "clusters-0"), outputPath,
+                        EuclideanDistanceMeasure.class.getName(), 0.001, 10, 1, true);
 
     // now compare the expected clusters with actual
-    File outDir = new File("output/clusteredPoints");
-    assertTrue("output dir exists?", outDir.exists());
-    String[] outFiles = outDir.list();
-    assertEquals("output dir files?", 4, outFiles.length);
+    Path clusteredPointsPath = new Path(outputPath, "clusteredPoints");
+    //String[] outFiles = outDir.list();
+    //assertEquals("output dir files?", 4, outFiles.length);
     DummyOutputCollector<IntWritable, WeightedVectorWritable> collector = new DummyOutputCollector<IntWritable, WeightedVectorWritable>();
-    SequenceFile.Reader reader = new SequenceFile.Reader(fs, new Path("output/clusteredPoints/part-00000"), conf);
+    SequenceFile.Reader reader = new SequenceFile.Reader(fs, new Path(clusteredPointsPath, "part-00000"), conf);
 
     // The key is the clusterId
     IntWritable clusterId = new IntWritable(0);

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestRandomSeedGenerator.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestRandomSeedGenerator.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestRandomSeedGenerator.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestRandomSeedGenerator.java Thu May  6 11:43:01 2010
@@ -76,23 +76,15 @@ public class TestRandomSeedGenerator ext
   /** Story: test random seed generation generates 4 clusters with proper ids and data */
   public void testRandomSeedGenerator() throws Exception {
     List<VectorWritable> points = getPoints();
-    File testData = new File("testdata");
-    if (!testData.exists()) {
-      testData.mkdir();
-    }
-    
-    File randomOutput = new File("testdata/random-output");
-    if (!randomOutput.exists()) {
-      randomOutput.mkdir();
-    }
-    
     JobConf job = new JobConf(RandomSeedGenerator.class);
     job.setMapOutputValueClass(VectorWritable.class);
-    ClusteringTestUtils.writePointsToFile(points, "testdata/random-input", fs, job);
+    Path input = getTestTempFilePath("random-input");
+    Path output = getTestTempDirPath("random-output");
+    ClusteringTestUtils.writePointsToFile(points, input, fs, job);
     
-    RandomSeedGenerator.buildRandom("testdata/random-input", "testdata/random-output", 4);
+    RandomSeedGenerator.buildRandom(input, output, 4);
     
-    SequenceFile.Reader reader = new SequenceFile.Reader(fs, new Path("testdata/random-output/part-randomSeed"), job);
+    SequenceFile.Reader reader = new SequenceFile.Reader(fs, new Path(output, "part-randomSeed"), job);
     Writable key = (Writable) reader.getKeyClass().newInstance();
     Cluster value = (Cluster) reader.getValueClass().newInstance();
     

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/meanshift/TestMeanShift.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/meanshift/TestMeanShift.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/meanshift/TestMeanShift.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/meanshift/TestMeanShift.java Thu May  6 11:43:01 2010
@@ -81,19 +81,6 @@ public class TestMeanShift extends Mahou
     }
   }
 
-  private static void rmr(String path) throws Exception {
-    File f = new File(path);
-    if (f.exists()) {
-      if (f.isDirectory()) {
-        String[] contents = f.list();
-        for (String content : contents) {
-          rmr(f.toString() + File.separator + content);
-        }
-      }
-      f.delete();
-    }
-  }
-
   private List<MeanShiftCanopy> getInitialCanopies() {
     int nextCanopyId = 0;
     List<MeanShiftCanopy> canopies = new ArrayList<MeanShiftCanopy>();
@@ -107,8 +94,6 @@ public class TestMeanShift extends Mahou
   protected void setUp() throws Exception {
     super.setUp();
     conf = new Configuration();
-    rmr("output");
-    rmr("testdata");
     raw = new Vector[100];
     for (int i = 0; i < 10; i++) {
       for (int j = 0; j < 10; j++) {
@@ -292,21 +277,19 @@ public class TestMeanShift extends Mahou
    * EuclideanDistanceMeasure.
    */
   public void testCanopyEuclideanMRJob() throws Exception {
-    File testData = new File("testdata");
-    if (!testData.exists()) {
-      testData.mkdir();
-    }
-    FileSystem fs = FileSystem.get(new Path("testdata").toUri(), conf);
+    Path input = getTestTempDirPath("testdata");
+    FileSystem fs = FileSystem.get(input.toUri(), conf);
     List<VectorWritable> points = new ArrayList<VectorWritable>();
     for (Vector v : raw) {
       points.add(new VectorWritable(v));
     }
-    ClusteringTestUtils.writePointsToFile(points, "testdata/file1", fs, conf);
-    ClusteringTestUtils.writePointsToFile(points, "testdata/file2", fs, conf);
+    ClusteringTestUtils.writePointsToFile(points, getTestTempFilePath("testdata/file1"), fs, conf);
+    ClusteringTestUtils.writePointsToFile(points, getTestTempFilePath("testdata/file2"), fs, conf);
     // now run the Job
-    MeanShiftCanopyDriver.runJob("testdata", "output", EuclideanDistanceMeasure.class.getName(), 4, 1, 0.5, 10, false, false);
+    Path output = getTestTempDirPath("output");
+    MeanShiftCanopyDriver.runJob(input, output, EuclideanDistanceMeasure.class.getName(), 4, 1, 0.5, 10, false, false);
     JobConf conf = new JobConf(MeanShiftCanopyDriver.class);
-    Path outPart = new Path("output/clusters-3/part-00000");
+    Path outPart = new Path(output, "clusters-3/part-00000");
     SequenceFile.Reader reader = new SequenceFile.Reader(fs, outPart, conf);
     Text key = new Text();
     MeanShiftCanopy value = new MeanShiftCanopy();

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/common/MahoutTestCase.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/common/MahoutTestCase.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/common/MahoutTestCase.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/common/MahoutTestCase.java Thu May  6 11:43:01 2010
@@ -17,14 +17,68 @@
 
 package org.apache.mahout.common;
 
-import junit.framework.TestCase;
+import java.io.IOException;
 
-public abstract class MahoutTestCase extends TestCase {
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+public abstract class MahoutTestCase extends org.apache.mahout.math.MahoutTestCase {
+
+  private Path testTempDirPath;
+  private FileSystem fs;
 
   @Override
   protected void setUp() throws Exception {
     super.setUp();
     RandomUtils.useTestSeed();
+    testTempDirPath = null;
+    fs = null;
+  }
+
+  @Override
+  protected void tearDown() throws Exception {
+    if (testTempDirPath != null) {
+      fs.delete(testTempDirPath, true);
+      testTempDirPath = null;
+      fs = null;
+    }
+    super.tearDown();
+  }
+
+  protected final Path getTestTempDirPath() throws IOException {
+    if (testTempDirPath == null) {
+      fs = FileSystem.get(new Configuration());
+      long simpleRandomLong = (long) (Long.MAX_VALUE * Math.random());
+      testTempDirPath =
+          fs.makeQualified(new Path("/tmp/mahout-" + getClass().getSimpleName() + '-' + simpleRandomLong));
+      if (!fs.mkdirs(testTempDirPath)) {
+        throw new IOException("Could not create " + testTempDirPath);
+      }
+      fs.deleteOnExit(testTempDirPath);
+    }
+    return testTempDirPath;
   }
 
+  protected final Path getTestTempFilePath(String name) throws IOException {
+    return getTestTempFileOrDirPath(name, false);
+  }
+
+  protected final Path getTestTempDirPath(String name) throws IOException {
+    return getTestTempFileOrDirPath(name, true);
+  }
+
+  private Path getTestTempFileOrDirPath(String name, boolean dir) throws IOException {
+    Path testTempDirPath = getTestTempDirPath();
+    Path tempFileOrDir = fs.makeQualified(new Path(testTempDirPath, name));
+    fs.deleteOnExit(tempFileOrDir);
+    if (dir) {
+      if (!fs.mkdirs(tempFileOrDir)) {
+        throw new IOException("Could not create " + tempFileOrDir);
+      }
+    }
+    return tempFileOrDir;
+  }
+
+
 }

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/df/mapred/partial/PartialBuilderTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/df/mapred/partial/PartialBuilderTest.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/df/mapred/partial/PartialBuilderTest.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/df/mapred/partial/PartialBuilderTest.java Thu May  6 11:43:01 2010
@@ -61,11 +61,8 @@ public class PartialBuilderTest extends 
     randomKeyValues(rng, keys, values, firstIds);
 
     // store the output in a sequence file
-    Path base = new Path("testdata");
+    Path base = getTestTempDirPath("testdata");
     FileSystem fs = base.getFileSystem(job);
-    if (fs.exists(base)) {
-      fs.delete(base, true);
-    }
 
     Path outputFile = new Path(base, "PartialBuilderTest.seq");
     Writer writer = SequenceFile.createWriter(fs, job, outputFile,

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/df/mapred/partial/PartitionBugTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/df/mapred/partial/PartitionBugTest.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/df/mapred/partial/PartitionBugTest.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/df/mapred/partial/PartitionBugTest.java Thu May  6 11:43:01 2010
@@ -26,6 +26,7 @@ import java.util.Random;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.mahout.common.HadoopUtil;
 import org.apache.mahout.common.MahoutTestCase;
 import org.apache.mahout.common.RandomUtils;
 import org.apache.mahout.df.builder.TreeBuilder;
@@ -89,9 +90,7 @@ public class PartitionBugTest extends Ma
     // remove the output path (its only used for testing)
     Path outputPath = builder.getOutputPath(jobConf);
     FileSystem fs = outputPath.getFileSystem(jobConf);
-    if (fs.exists(outputPath)) {
-      fs.delete(outputPath, true);
-    }
+    HadoopUtil.overwriteOutput(outputPath);
     
     builder.build(numTrees, new MockCallback(data));
   }

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/df/mapreduce/partial/PartialBuilderTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/df/mapreduce/partial/PartialBuilderTest.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/df/mapreduce/partial/PartialBuilderTest.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/df/mapreduce/partial/PartialBuilderTest.java Thu May  6 11:43:01 2010
@@ -61,10 +61,8 @@ public class PartialBuilderTest extends 
     randomKeyValues(rng, keys, values, firstIds);
 
     // store the output in a sequence file
-    Path base = new Path("testdata");
+    Path base = getTestTempDirPath("testdata");
     FileSystem fs = base.getFileSystem(conf);
-    if (fs.exists(base))
-      fs.delete(base, true);
 
     Path outputFile = new Path(base, "PartialBuilderTest.seq");
     Writer writer = SequenceFile.createWriter(fs, conf, outputFile,

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/df/mapreduce/partial/PartitionBugTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/df/mapreduce/partial/PartitionBugTest.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/df/mapreduce/partial/PartitionBugTest.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/df/mapreduce/partial/PartitionBugTest.java Thu May  6 11:43:01 2010
@@ -26,6 +26,7 @@ import java.util.Random;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.mahout.common.HadoopUtil;
 import org.apache.mahout.common.MahoutTestCase;
 import org.apache.mahout.common.RandomUtils;
 import org.apache.mahout.df.builder.TreeBuilder;
@@ -85,11 +86,8 @@ public class PartitionBugTest extends Ma
 
     // remove the output path (its only used for testing)
     Path outputPath = builder.getOutputPath(conf);
-    FileSystem fs = outputPath.getFileSystem(conf);
-    if (fs.exists(outputPath)) {
-      fs.delete(outputPath, true);
-    }
-    
+    HadoopUtil.overwriteOutput(outputPath);
+
     builder.build(numTrees, new MockCallback(data));
   }
 

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/fpm/pfpgrowth/FPGrowthTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/fpm/pfpgrowth/FPGrowthTest.java?rev=941676&r1=941675&r2=941676&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/fpm/pfpgrowth/FPGrowthTest.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/fpm/pfpgrowth/FPGrowthTest.java Thu May  6 11:43:01 2010
@@ -55,14 +55,10 @@ public class FPGrowthTest extends Mahout
     transactions.add(new Pair<List<String>,Long>(Arrays.asList("A", "D", "E"), 1L));
     transactions.add(new Pair<List<String>,Long>(Arrays.asList("B", "C"), 1L));
 
-    File tmpFile = File.createTempFile("fpgrowthTest", ".dat");
-    tmpFile.deleteOnExit();
-
-    Path path = new Path(tmpFile.getAbsolutePath());
+    Path path = getTestTempFilePath("fpgrowthTest.dat");
     Configuration conf = new Configuration();
     FileSystem fs = FileSystem.get(conf);
 
-
     SequenceFile.Writer writer =
         new SequenceFile.Writer(fs, conf, path, Text.class, TopKStringPatterns.class);
     fp.generateTopKFrequentPatterns(