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/07 14:25:12 UTC

svn commit: r942056 - in /lucene/mahout/trunk/utils/src: main/java/org/apache/mahout/clustering/cdbw/ main/java/org/apache/mahout/text/ main/java/org/apache/mahout/utils/clustering/ main/java/org/apache/mahout/utils/vectors/lucene/ main/java/org/apache...

Author: srowen
Date: Fri May  7 12:25:11 2010
New Revision: 942056

URL: http://svn.apache.org/viewvc?rev=942056&view=rev
Log:
MAHOUT-302: more attempt to fix

Modified:
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/clustering/cdbw/CDbwDriver.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/clustering/cdbw/CDbwEvaluator.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/text/SparseVectorsFromSequenceFiles.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/clustering/ClusterDumper.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/lucene/ClusterLabels.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizer.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/tfidf/TFIDFConverter.java
    lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/clustering/TestClusterDumper.java
    lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/clustering/cdbw/TestCDbwEvaluator.java
    lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizerTest.java

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/clustering/cdbw/CDbwDriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/clustering/cdbw/CDbwDriver.java?rev=942056&r1=942055&r2=942056&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/clustering/cdbw/CDbwDriver.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/clustering/cdbw/CDbwDriver.java Fri May  7 12:25:11 2010
@@ -30,6 +30,7 @@ import org.apache.commons.cli2.builder.D
 import org.apache.commons.cli2.builder.GroupBuilder;
 import org.apache.commons.cli2.commandline.Parser;
 import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
@@ -95,8 +96,8 @@ public class CDbwDriver {
         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());
       String modelFactory = "org.apache.mahout.clustering.dirichlet.models.NormalModelDistribution";
       if (cmdLine.hasOption(modelOpt)) {
         modelFactory = cmdLine.getValue(modelOpt).toString();
@@ -126,17 +127,17 @@ public class CDbwDriver {
    * @param numReducers
    *          the number of Reducers desired
    */
-  public static void runJob(String clustersIn, String clusteredPointsIn, String output, String distanceMeasureClass,
+  public static void runJob(Path clustersIn, Path clusteredPointsIn, Path output, String distanceMeasureClass,
       int numIterations, int numReducers) throws ClassNotFoundException, InstantiationException, IllegalAccessException,
       IOException, SecurityException, NoSuchMethodException, InvocationTargetException {
 
-    String stateIn = output + "/representativePoints-0";
+    Path stateIn = new Path(output, "representativePoints-0");
     writeInitialState(stateIn, clustersIn);
 
     for (int iteration = 0; iteration < numIterations; iteration++) {
       log.info("Iteration {}", iteration);
       // point the output to a new directory per iteration
-      String stateOut = output + "/representativePoints-" + (iteration + 1);
+      Path stateOut = new Path(output, "representativePoints-" + (iteration + 1));
       runIteration(clusteredPointsIn, stateIn, stateOut, distanceMeasureClass, numReducers);
       // now point the input to the old output directory
       stateIn = stateOut;
@@ -144,26 +145,24 @@ public class CDbwDriver {
 
     Configurable client = new JobClient();
     JobConf conf = new JobConf(CDbwDriver.class);
-    conf.set(STATE_IN_KEY, stateIn);
+    conf.set(STATE_IN_KEY, stateIn.toString());
     conf.set(DISTANCE_MEASURE_KEY, distanceMeasureClass);
     CDbwEvaluator evaluator = new CDbwEvaluator(conf, clustersIn);
     System.out.println("CDbw = " + evaluator.CDbw());
   }
 
-  private static void writeInitialState(String output, String clustersIn) throws ClassNotFoundException, InstantiationException,
-      IllegalAccessException, IOException, SecurityException, NoSuchMethodException, InvocationTargetException {
+  private static void writeInitialState(Path output, Path clustersIn)
+      throws InstantiationException, IllegalAccessException, IOException, SecurityException {
 
     JobConf job = new JobConf(KMeansDriver.class);
-    Path outPath = new Path(output);
-    FileSystem fs = FileSystem.get(outPath.toUri(), job);
-    File f = new File(clustersIn);
-    for (File part : f.listFiles()) {
-      if (!part.getName().startsWith(".")) {
-        Path inPart = new Path(clustersIn + "/" + part.getName());
+    FileSystem fs = FileSystem.get(output.toUri(), job);
+    for (FileStatus part : fs.listStatus(clustersIn)) {
+      if (!part.getPath().getName().startsWith(".")) {
+        Path inPart = part.getPath();
         SequenceFile.Reader reader = new SequenceFile.Reader(fs, inPart, job);
         Writable key = (Writable) reader.getKeyClass().newInstance();
         Writable value = (Writable) reader.getValueClass().newInstance();
-        Path path = new Path(output + "/" + part.getName());
+        Path path = new Path(output, inPart.getName());
         SequenceFile.Writer writer = new SequenceFile.Writer(fs, job, path, IntWritable.class, VectorWritable.class);
         while (reader.next(key, value)) {
           Cluster cluster = (Cluster) value;
@@ -191,7 +190,8 @@ public class CDbwDriver {
    * @param numReducers
    *          the number of Reducers desired
    */
-  public static void runIteration(String input, String stateIn, String stateOut, String distanceMeasureClass, int numReducers) {
+  public static void runIteration(Path input, Path stateIn, Path stateOut,
+                                  String distanceMeasureClass, int numReducers) {
     Configurable client = new JobClient();
     JobConf conf = new JobConf(CDbwDriver.class);
 
@@ -200,16 +200,15 @@ public class CDbwDriver {
     conf.setMapOutputKeyClass(IntWritable.class);
     conf.setMapOutputValueClass(WeightedVectorWritable.class);
 
-    FileInputFormat.setInputPaths(conf, new Path(input));
-    Path outPath = new Path(stateOut);
-    FileOutputFormat.setOutputPath(conf, outPath);
+    FileInputFormat.setInputPaths(conf, input);
+    FileOutputFormat.setOutputPath(conf, stateOut);
 
     conf.setMapperClass(CDbwMapper.class);
     conf.setReducerClass(CDbwReducer.class);
     conf.setNumReduceTasks(numReducers);
     conf.setInputFormat(SequenceFileInputFormat.class);
     conf.setOutputFormat(SequenceFileOutputFormat.class);
-    conf.set(STATE_IN_KEY, stateIn);
+    conf.set(STATE_IN_KEY, stateIn.toString());
     conf.set(DISTANCE_MEASURE_KEY, distanceMeasureClass);
 
     client.setConf(conf);

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/clustering/cdbw/CDbwEvaluator.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/clustering/cdbw/CDbwEvaluator.java?rev=942056&r1=942055&r2=942056&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/clustering/cdbw/CDbwEvaluator.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/clustering/cdbw/CDbwEvaluator.java Fri May  7 12:25:11 2010
@@ -24,6 +24,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.SequenceFile;
@@ -83,7 +84,8 @@ public class CDbwEvaluator {
    * @throws IllegalAccessException
    * @throws IOException
    */
-  public CDbwEvaluator(JobConf job, String clustersIn) throws SecurityException, IllegalArgumentException, NoSuchMethodException,
+  public CDbwEvaluator(JobConf job, Path clustersIn)
+      throws SecurityException, IllegalArgumentException, NoSuchMethodException,
       InvocationTargetException, ClassNotFoundException, InstantiationException, IllegalAccessException, IOException {
     super();
     ClassLoader ccl = Thread.currentThread().getContextClassLoader();
@@ -117,15 +119,13 @@ public class CDbwEvaluator {
    * @throws NoSuchMethodException
    * @throws InvocationTargetException
    */
-  private HashMap<Integer, Cluster> loadClusters(JobConf job, String clustersIn) throws ClassNotFoundException,
-      InstantiationException, IllegalAccessException, IOException, SecurityException, NoSuchMethodException,
-      InvocationTargetException {
+  private HashMap<Integer, Cluster> loadClusters(JobConf job, Path clustersIn)
+      throws InstantiationException, IllegalAccessException, IOException, SecurityException {
     HashMap<Integer, Cluster> clusters = new HashMap<Integer, Cluster>();
-    File f = new File(clustersIn);
-    for (File part : f.listFiles()) {
-      if (!part.getName().startsWith(".")) {
-        Path inPart = new Path(clustersIn + "/" + part.getName());
-        FileSystem fs = FileSystem.get(inPart.toUri(), job);
+    FileSystem fs = clustersIn.getFileSystem(job);
+    for (FileStatus part : fs.listStatus(clustersIn)) {
+      if (!part.getPath().getName().startsWith(".")) {
+        Path inPart = part.getPath();
         SequenceFile.Reader reader = new SequenceFile.Reader(fs, inPart, job);
         Writable key = (Writable) reader.getKeyClass().newInstance();
         Writable value = (Writable) reader.getValueClass().newInstance();

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/text/SparseVectorsFromSequenceFiles.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/text/SparseVectorsFromSequenceFiles.java?rev=942056&r1=942055&r2=942056&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/text/SparseVectorsFromSequenceFiles.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/text/SparseVectorsFromSequenceFiles.java Fri May  7 12:25:11 2010
@@ -222,8 +222,9 @@ public final class SparseVectorsFromSequ
       DictionaryVectorizer.createTermFrequencyVectors(tokenizedPath, outputDir, minSupport, maxNGramSize,
         minLLRValue, reduceTasks, chunkSize, sequentialAccessOutput);
       if (processIdf) {
-        TFIDFConverter.processTfIdf(outputDir + DictionaryVectorizer.DOCUMENT_VECTOR_OUTPUT_FOLDER,
-          outputDir + TFIDFConverter.TFIDF_OUTPUT_FOLDER, chunkSize, minDf, maxDFPercent, norm,
+        TFIDFConverter.processTfIdf(
+          new Path(outputDir, DictionaryVectorizer.DOCUMENT_VECTOR_OUTPUT_FOLDER),
+          new Path(outputDir, TFIDFConverter.TFIDF_OUTPUT_FOLDER), chunkSize, minDf, maxDFPercent, norm,
           sequentialAccessOutput);
       }
     } catch (OptionException e) {

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/clustering/ClusterDumper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/clustering/ClusterDumper.java?rev=942056&r1=942055&r2=942056&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/clustering/ClusterDumper.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/clustering/ClusterDumper.java Fri May  7 12:25:11 2010
@@ -42,8 +42,10 @@ import org.apache.commons.cli2.builder.D
 import org.apache.commons.cli2.builder.GroupBuilder;
 import org.apache.commons.cli2.commandline.Parser;
 import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
@@ -65,9 +67,9 @@ public final class ClusterDumper {
 
   private static final Logger log = LoggerFactory.getLogger(ClusterDumper.class);
 
-  private final String seqFileDir;
+  private final Path seqFileDir;
 
-  private final String pointsDir;
+  private final Path pointsDir;
 
   private String termDictionary;
 
@@ -83,7 +85,7 @@ public final class ClusterDumper {
 
   private boolean useJSON = false;
 
-  public ClusterDumper(String seqFileDir, String pointsDir) throws IOException {
+  public ClusterDumper(Path seqFileDir, Path pointsDir) throws IOException {
     this.seqFileDir = seqFileDir;
     this.pointsDir = pointsDir;
     init();
@@ -115,26 +117,18 @@ public final class ClusterDumper {
       }
     }
 
-    Writer writer = null;
-    if (this.outputFile != null) {
-      writer = new FileWriter(this.outputFile);
-    } else {
-      writer = new OutputStreamWriter(System.out);
-    }
+    Writer writer = this.outputFile == null ? new OutputStreamWriter(System.out) : new FileWriter(this.outputFile);
 
-    File[] seqFileList = new File(this.seqFileDir).listFiles(new FilenameFilter() {
+    FileSystem fs = seqFileDir.getFileSystem(conf);
+    FileStatus[] seqFileList = fs.listStatus(seqFileDir, new PathFilter() {
       @Override
-      public boolean accept(File file, String name) {
-        return name.endsWith(".crc") == false;
+      public boolean accept(Path path) {
+        return !path.getName().endsWith(".crc");
       }
     });
-    for (File seqFile : seqFileList) {
-      if (!seqFile.isFile()) {
-        continue;
-      }
-      Path path = new Path(seqFile.getAbsolutePath());
+    for (FileStatus seqFile : seqFileList) {
+      Path path = seqFile.getPath();
       System.out.println("Input Path: " + path);
-      FileSystem fs = FileSystem.get(path.toUri(), conf);
       SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, conf);
       Writable key = (Writable) reader.getKeyClass().newInstance();
       Writable value = (Writable) reader.getValueClass().newInstance();
@@ -262,15 +256,15 @@ public final class ClusterDumper {
       if (!cmdLine.hasOption(seqOpt)) {
         return;
       }
-      String seqFileDir = cmdLine.getValue(seqOpt).toString();
+      Path seqFileDir = new Path(cmdLine.getValue(seqOpt).toString());
       String termDictionary = null;
       if (cmdLine.hasOption(dictOpt)) {
         termDictionary = cmdLine.getValue(dictOpt).toString();
       }
 
-      String pointsDir = null;
+      Path pointsDir = null;
       if (cmdLine.hasOption(pointsOpt)) {
-        pointsDir = cmdLine.getValue(pointsOpt).toString();
+        pointsDir = new Path(cmdLine.getValue(pointsOpt).toString());
       }
       String outputFile = null;
       if (cmdLine.hasOption(outputOpt)) {
@@ -319,23 +313,20 @@ public final class ClusterDumper {
     this.useJSON = json;
   }
 
-  private static Map<Integer, List<WeightedVectorWritable>> readPoints(String pointsPathDir, JobConf conf) throws IOException {
+  private static Map<Integer, List<WeightedVectorWritable>> readPoints(Path pointsPathDir, JobConf conf)
+      throws IOException {
     Map<Integer, List<WeightedVectorWritable>> result = new TreeMap<Integer, List<WeightedVectorWritable>>();
 
-    File[] children = new File(pointsPathDir).listFiles(new FilenameFilter() {
+    FileSystem fs = pointsPathDir.getFileSystem(conf);
+    FileStatus[] children = fs.listStatus(pointsPathDir, new PathFilter() {
       @Override
-      public boolean accept(File file, String name) {
-        return name.endsWith(".crc") == false;
+      public boolean accept(Path path) {
+        return !path.getName().endsWith(".crc");
       }
     });
 
-    for (File file : children) {
-      if (!file.isFile()) {
-        continue;
-      }
-      String pointsPath = file.getAbsolutePath();
-      Path path = new Path(pointsPath);
-      FileSystem fs = FileSystem.get(path.toUri(), conf);
+    for (FileStatus file : children) {
+      Path path = file.getPath();
       SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, conf);
       try {
         IntWritable key = (IntWritable) reader.getKeyClass().newInstance();
@@ -344,7 +335,7 @@ public final class ClusterDumper {
           // value is the cluster id as an int, key is the name/id of the
           // vector, but that doesn't matter because we only care about printing
           // it
-          String clusterId = value.toString();
+          //String clusterId = value.toString();
           List<WeightedVectorWritable> pointList = result.get(key.get());
           if (pointList == null) {
             pointList = new ArrayList<WeightedVectorWritable>();

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/lucene/ClusterLabels.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/lucene/ClusterLabels.java?rev=942056&r1=942055&r2=942056&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/lucene/ClusterLabels.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/lucene/ClusterLabels.java Fri May  7 12:25:11 2010
@@ -39,6 +39,7 @@ import org.apache.commons.cli2.builder.A
 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.lucene.document.FieldSelector;
 import org.apache.lucene.document.SetBasedFieldSelector;
 import org.apache.lucene.index.CorruptIndexException;
@@ -105,9 +106,9 @@ public class ClusterLabels {
 
   public static final int DEFAULT_MAX_LABELS = 25;
 
-  private final String seqFileDir;
+  private final Path seqFileDir;
 
-  private final String pointsDir;
+  private final Path pointsDir;
 
   private final String indexDir;
 
@@ -123,7 +124,7 @@ public class ClusterLabels {
 
   private int maxLabels = DEFAULT_MAX_LABELS;
 
-  public ClusterLabels(String seqFileDir, String pointsDir, String indexDir, String contentField, int minNumIds, int maxLabels)
+  public ClusterLabels(Path seqFileDir, Path pointsDir, String indexDir, String contentField, int minNumIds, int maxLabels)
       throws IOException {
     this.seqFileDir = seqFileDir;
     this.pointsDir = pointsDir;
@@ -363,8 +364,8 @@ public class ClusterLabels {
         return;
       }
 
-      String seqFileDir = cmdLine.getValue(seqOpt).toString();
-      String pointsDir = cmdLine.getValue(pointsOpt).toString();
+      Path seqFileDir = new Path(cmdLine.getValue(seqOpt).toString());
+      Path pointsDir = new Path(cmdLine.getValue(pointsOpt).toString());
       String indexDir = cmdLine.getValue(indexOpt).toString();
       String contentField = cmdLine.getValue(fieldOpt).toString();
 

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizer.java?rev=942056&r1=942055&r2=942056&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizer.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizer.java Fri May  7 12:25:11 2010
@@ -73,7 +73,7 @@ public final class DictionaryVectorizer 
   
   private static final int MIN_CHUNKSIZE = 100;
   
-  private static final String OUTPUT_FILES_PATTERN = "/part-*";
+  private static final String OUTPUT_FILES_PATTERN = "part-*";
   
   // 4 byte overhead for each entry in the OpenObjectIntHashMap
   private static final int DICTIONARY_BYTE_OVERHEAD = 4;
@@ -192,8 +192,7 @@ public final class DictionaryVectorizer 
     Configuration conf = new Configuration();
     
     FileSystem fs = FileSystem.get(wordCountPath.toUri(), conf);
-    FileStatus[] outputFiles = fs.globStatus(new Path(wordCountPath.toString()
-                                                      + OUTPUT_FILES_PATTERN));
+    FileStatus[] outputFiles = fs.globStatus(new Path(wordCountPath, OUTPUT_FILES_PATTERN));
     
     long chunkSizeLimit = chunkSizeInMegabytes * 1024L * 1024L;
     int chunkIndex = 0;

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/tfidf/TFIDFConverter.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/tfidf/TFIDFConverter.java?rev=942056&r1=942055&r2=942056&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/tfidf/TFIDFConverter.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/tfidf/TFIDFConverter.java Fri May  7 12:25:11 2010
@@ -64,23 +64,23 @@ public final class TFIDFConverter {
   
   public static final String MAX_DF_PERCENTAGE = "max.df.percentage";
   
-  public static final String TFIDF_OUTPUT_FOLDER = "/tfidf";
+  public static final String TFIDF_OUTPUT_FOLDER = "tfidf";
   
-  private static final String DOCUMENT_VECTOR_OUTPUT_FOLDER = "/vectors";
+  private static final String DOCUMENT_VECTOR_OUTPUT_FOLDER = "vectors";
   
-  private static final String FREQUENCY_FILE = "/frequency.file-";
+  private static final String FREQUENCY_FILE = "frequency.file-";
   
   private static final int MAX_CHUNKSIZE = 10000;
   
   private static final int MIN_CHUNKSIZE = 100;
   
-  private static final String OUTPUT_FILES_PATTERN = "/part-*";
+  private static final String OUTPUT_FILES_PATTERN = "part-*";
   
   private static final int SEQUENCEFILE_BYTE_OVERHEAD = 45;
   
-  private static final String VECTOR_OUTPUT_FOLDER = "/partial-vectors-";
+  private static final String VECTOR_OUTPUT_FOLDER = "partial-vectors-";
   
-  private static final String WORDCOUNT_OUTPUT_FOLDER = "/df-count";
+  private static final String WORDCOUNT_OUTPUT_FOLDER = "df-count";
   
   /**
    * Cannot be initialized. Use the static functions
@@ -112,8 +112,8 @@ public final class TFIDFConverter {
    *          partial vectors without thrashing the system due to increased swapping
    * @throws IOException
    */
-  public static void processTfIdf(String input,
-                                  String output,
+  public static void processTfIdf(Path input,
+                                  Path output,
                                   int chunkSizeInMegabytes,
                                   int minDf,
                                   int maxDFPercent,
@@ -136,10 +136,9 @@ public final class TFIDFConverter {
       maxDFPercent = 99;
     }
     
-    Path inputPath = new Path(input);
-    Path wordCountPath = new Path(output + WORDCOUNT_OUTPUT_FOLDER);
+    Path wordCountPath = new Path(output, WORDCOUNT_OUTPUT_FOLDER);
     
-    startDFCounting(inputPath, wordCountPath);
+    startDFCounting(input, wordCountPath);
     Pair<Long[],List<Path>> datasetFeatures = createDictionaryChunks(wordCountPath, output,
       chunkSizeInMegabytes);
     
@@ -147,8 +146,7 @@ public final class TFIDFConverter {
     List<Path> partialVectorPaths = new ArrayList<Path>();
     List<Path> dictionaryChunks = datasetFeatures.getSecond();
     for (Path dictionaryChunk : dictionaryChunks) {
-      Path partialVectorOutputPath = getPath(output + VECTOR_OUTPUT_FOLDER,
-        partialVectorIndex++);
+      Path partialVectorOutputPath = new Path(output, VECTOR_OUTPUT_FOLDER + partialVectorIndex++);
       partialVectorPaths.add(partialVectorOutputPath);
       makePartialVectors(input, datasetFeatures.getFirst()[0], datasetFeatures.getFirst()[1],
         minDf, maxDFPercent, dictionaryChunk, partialVectorOutputPath, sequentialAccessOutput);
@@ -178,7 +176,7 @@ public final class TFIDFConverter {
    * @throws IOException
    */
   private static Pair<Long[],List<Path>> createDictionaryChunks(Path featureCountPath,
-                                                                String dictionaryPathBase,
+                                                                Path dictionaryPathBase,
                                                                 int chunkSizeInMegabytes) throws IOException {
     List<Path> chunkPaths = new ArrayList<Path>();
     
@@ -187,12 +185,11 @@ public final class TFIDFConverter {
     Configuration conf = new Configuration();
     
     FileSystem fs = FileSystem.get(featureCountPath.toUri(), conf);
-    FileStatus[] outputFiles = fs.globStatus(new Path(featureCountPath.toString()
-                                                      + OUTPUT_FILES_PATTERN));
+    FileStatus[] outputFiles = fs.globStatus(new Path(featureCountPath, OUTPUT_FILES_PATTERN));
     
     long chunkSizeLimit = chunkSizeInMegabytes * 1024L * 1024L;
     int chunkIndex = 0;
-    Path chunkPath = getPath(dictionaryPathBase + FREQUENCY_FILE, chunkIndex);
+    Path chunkPath = new Path(dictionaryPathBase, FREQUENCY_FILE + chunkIndex);
     chunkPaths.add(chunkPath);
     SequenceFile.Writer freqWriter = new SequenceFile.Writer(fs, conf, chunkPath, IntWritable.class,
         LongWritable.class);
@@ -209,7 +206,7 @@ public final class TFIDFConverter {
           freqWriter.close();
           chunkIndex++;
           
-          chunkPath = getPath(dictionaryPathBase + FREQUENCY_FILE, chunkIndex);
+          chunkPath = new Path(dictionaryPathBase, FREQUENCY_FILE + chunkIndex);
           chunkPaths.add(chunkPath);
           
           freqWriter = new SequenceFile.Writer(fs, conf, chunkPath, IntWritable.class, LongWritable.class);
@@ -233,10 +230,6 @@ public final class TFIDFConverter {
     return new Pair<Long[],List<Path>>(counts, chunkPaths);
   }
   
-  public static Path getPath(String basePath, int index) {
-    return new Path(basePath + index);
-  }
-  
   /**
    * Create a partial tfidf vector using a chunk of features from the input vectors. The input vectors has to
    * be in the {@link SequenceFile} format
@@ -258,7 +251,7 @@ public final class TFIDFConverter {
    *          output directory were the partial vectors have to be created
    * @throws IOException
    */
-  private static void makePartialVectors(String input,
+  private static void makePartialVectors(Path input,
                                          Long featureCount,
                                          Long vectorCount,
                                          int minDf,
@@ -283,7 +276,7 @@ public final class TFIDFConverter {
     conf.setOutputKeyClass(Text.class);
     conf.setOutputValueClass(VectorWritable.class);
     DistributedCache.setCacheFiles(new URI[] {dictionaryFilePath.toUri()}, conf);
-    FileInputFormat.setInputPaths(conf, new Path(input));
+    FileInputFormat.setInputPaths(conf, input);
     
     FileOutputFormat.setOutputPath(conf, output);
     

Modified: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/clustering/TestClusterDumper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/clustering/TestClusterDumper.java?rev=942056&r1=942055&r2=942056&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/clustering/TestClusterDumper.java (original)
+++ lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/clustering/TestClusterDumper.java Fri May  7 12:25:11 2010
@@ -27,6 +27,7 @@ import junit.framework.Assert;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -75,25 +76,11 @@ public class TestClusterDumper extends M
   @Override
   protected void setUp() throws Exception {
     super.setUp();
-    RandomUtils.useTestSeed();
     Configuration conf = new Configuration();
     FileSystem fs = FileSystem.get(conf);
-    // Create testdata directory
-    File f = new File("testdata");
-    if (!f.exists()) {
-      f.mkdir();
-    }
-    f = new File("testdata/points");
-    if (!f.exists()) {
-      f.mkdir();
-    }
-    f = new File("output");
-    rmDir(f);
     // Create test data
     getSampleData(DOCS);
-    ClusteringTestUtils.writePointsToFile(sampleData, "testdata/points/file1", fs, conf);
-    // Run clustering job
-    // Run ClusterDumper test
+    ClusteringTestUtils.writePointsToFile(sampleData, getTestTempFilePath("testdata/file1"), fs, conf);
   }
 
   private static void rmDir(File f) {
@@ -145,7 +132,7 @@ public class TestClusterDumper extends M
     i = 0;
     for (Vector vector : iterable) {
       Assert.assertNotNull(vector);
-      NamedVector namedVector = new NamedVector(vector, "P(" + i + ")");
+      NamedVector namedVector = new NamedVector(vector, "P(" + i + ')');
       System.out.println(ClusterBase.formatVector(namedVector, termDictionary));
       sampleData.add(new VectorWritable(namedVector));
       i++;
@@ -153,46 +140,63 @@ public class TestClusterDumper extends M
   }
 
   public void testCanopy() throws Exception { // now run the Job
-    CanopyDriver.runJob("testdata/points", "output", EuclideanDistanceMeasure.class.getName(), 8, 4, true);
+    Path output = getTestTempDirPath("output");
+    CanopyDriver.runJob(getTestTempDirPath("testdata"), output,
+                        EuclideanDistanceMeasure.class.getName(), 8, 4, true);
     // run ClusterDumper
-    ClusterDumper clusterDumper = new ClusterDumper("output/clusters-0", "output/clusteredPoints");
+    ClusterDumper clusterDumper = new ClusterDumper(new Path(output, "clusters-0"),
+                                                    new Path(output, "clusteredPoints"));
     clusterDumper.printClusters(termDictionary);
   }
 
   public void testKmeans() throws Exception {
     // now run the Canopy job to prime kMeans canopies
-    CanopyDriver.runJob("testdata/points", "output", EuclideanDistanceMeasure.class.getName(), 8, 4, false);
+    Path output = getTestTempDirPath("output");
+    CanopyDriver.runJob(getTestTempDirPath("testdata"), output,
+                        EuclideanDistanceMeasure.class.getName(), 8, 4, false);
     // now run the KMeans job
-    KMeansDriver.runJob("testdata/points", "output/clusters-0", "output", EuclideanDistanceMeasure.class.getName(), 0.001, 10, 1, true);
+    KMeansDriver.runJob(getTestTempDirPath("testdata"), new Path(output, "clusters-0"), output,
+                        EuclideanDistanceMeasure.class.getName(), 0.001, 10, 1, true);
     // run ClusterDumper
-    ClusterDumper clusterDumper = new ClusterDumper("output/clusters-2", "output/clusteredPoints");
+    ClusterDumper clusterDumper = new ClusterDumper(new Path(output, "clusters-2"),
+                                                    new Path(output, "clusteredPoints"));
     clusterDumper.printClusters(termDictionary);
   }
 
   public void testFuzzyKmeans() throws Exception {
     // now run the Canopy job to prime kMeans canopies
-    CanopyDriver.runJob("testdata/points", "output", EuclideanDistanceMeasure.class.getName(), 8, 4, false);
+    Path output = getTestTempDirPath("output");
+    CanopyDriver.runJob(getTestTempDirPath("testdata"), output,
+                        EuclideanDistanceMeasure.class.getName(), 8, 4, false);
     // now run the KMeans job
-    FuzzyKMeansDriver.runJob("testdata/points", "output/clusters-0", "output", EuclideanDistanceMeasure.class.getName(), 0.001, 10,
+    FuzzyKMeansDriver.runJob(getTestTempDirPath("testdata"), new Path(output, "clusters-0"), output,
+                             EuclideanDistanceMeasure.class.getName(), 0.001, 10,
         1, 1, (float) 1.1, true, true, 0);
     // run ClusterDumper
-    ClusterDumper clusterDumper = new ClusterDumper("output/clusters-3", "output/clusteredPoints");
+    ClusterDumper clusterDumper = new ClusterDumper(new Path(output, "clusters-3"),
+                                                    new Path(output, "clusteredPoints"));
     clusterDumper.printClusters(termDictionary);
   }
 
   public void testMeanShift() throws Exception {
-    MeanShiftCanopyDriver.runJob("testdata/points", "output", CosineDistanceMeasure.class.getName(), 0.5, 0.01, 0.05, 10, false, true);
+    Path output = getTestTempDirPath("output");
+    MeanShiftCanopyDriver.runJob(getTestTempDirPath("testdata"), output,
+                                 CosineDistanceMeasure.class.getName(), 0.5, 0.01, 0.05, 10, false, true);
     // run ClusterDumper
-    ClusterDumper clusterDumper = new ClusterDumper("output/clusters-1", "output/clusteredPoints");
+    ClusterDumper clusterDumper = new ClusterDumper(new Path(output, "clusters-1"),
+                                                    new Path(output, "clusteredPoints"));
     clusterDumper.printClusters(termDictionary);
   }
 
   public void testDirichlet() throws Exception {
+    Path output = getTestTempDirPath("output");
     NamedVector prototype = (NamedVector) sampleData.get(0).get();
-    DirichletDriver.runJob("testdata/points", "output", L1ModelDistribution.class.getName(), prototype.getDelegate().getClass()
-        .getName(), prototype.size(), 15, 10, 1.0, 1, true, true, 0);
+    DirichletDriver.runJob(getTestTempDirPath("testdata"), output,
+                           L1ModelDistribution.class.getName(), prototype.getDelegate().getClass().getName(),
+                           prototype.size(), 15, 10, 1.0, 1, true, true, 0);
     // run ClusterDumper
-    ClusterDumper clusterDumper = new ClusterDumper("output/clusters-10", "output/clusteredPoints");
+    ClusterDumper clusterDumper = new ClusterDumper(new Path(output, "clusters-10"),
+                                                    new Path(output, "clusteredPoints"));
     clusterDumper.printClusters(termDictionary);
   }
 }

Modified: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/clustering/cdbw/TestCDbwEvaluator.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/clustering/cdbw/TestCDbwEvaluator.java?rev=942056&r1=942055&r2=942056&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/clustering/cdbw/TestCDbwEvaluator.java (original)
+++ lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/clustering/cdbw/TestCDbwEvaluator.java Fri May  7 12:25:11 2010
@@ -17,7 +17,6 @@
 
 package org.apache.mahout.clustering.cdbw;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -25,6 +24,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
@@ -41,7 +41,6 @@ import org.apache.mahout.clustering.kmea
 import org.apache.mahout.clustering.kmeans.TestKmeansClustering;
 import org.apache.mahout.clustering.meanshift.MeanShiftCanopyDriver;
 import org.apache.mahout.common.MahoutTestCase;
-import org.apache.mahout.common.RandomUtils;
 import org.apache.mahout.common.distance.EuclideanDistanceMeasure;
 import org.apache.mahout.math.DenseVector;
 import org.apache.mahout.math.Vector;
@@ -49,10 +48,8 @@ import org.apache.mahout.math.VectorWrit
 
 public class TestCDbwEvaluator 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 List<VectorWritable> sampleData;
+  public static final double[][] reference = { { 1, 1 }, { 2, 1 }, { 1, 2 }, { 2, 2 }, { 3, 3 },
+      { 4, 4 }, { 5, 4 }, { 4, 5 }, { 5, 5 } };
 
   private Map<Integer, List<VectorWritable>> representativePoints;
 
@@ -61,32 +58,21 @@ public class TestCDbwEvaluator extends M
   @Override
   protected void setUp() throws Exception {
     super.setUp();
-    RandomUtils.useTestSeed();
     Configuration conf = new Configuration();
     FileSystem fs = FileSystem.get(conf);
-    // Create testdata directory
-    ClusteringTestUtils.rmr("testdata");
-    File f = new File("testdata");
-    f.mkdir();
-    ClusteringTestUtils.rmr("output");
     // Create test data
-    sampleData = TestKmeansClustering.getPointsWritable(reference);
-    ClusteringTestUtils.writePointsToFile(sampleData, "testdata/file1", fs, conf);
+    List<VectorWritable> sampleData = TestKmeansClustering.getPointsWritable(reference);
+    ClusteringTestUtils.writePointsToFile(sampleData, getTestTempFilePath("testdata/file1"), fs, conf);
   }
 
   private void checkRefPoints(int numIterations) throws IOException {
-    File out = new File("output");
-    assertTrue("output is not Dir", out.isDirectory());
     for (int i = 0; i <= numIterations; i++) {
-      out = new File("output/representativePoints-" + i);
-      assertTrue("rep-i is not a Dir", out.isDirectory());
-      System.out.println(out.getName() + ":");
-      File[] files = out.listFiles();
+      Path out = new Path(getTestTempDirPath("output"), "representativePoints-" + i);
       Configuration conf = new Configuration();
       FileSystem fs = FileSystem.get(conf);
-      for (File file : files) {
-        if (!file.getName().startsWith(".")) {
-          SequenceFile.Reader reader = new SequenceFile.Reader(fs, new Path(file.getAbsolutePath()), conf);
+      for (FileStatus file : fs.listStatus(out)) {
+        if (!file.getPath().getName().startsWith(".")) {
+          SequenceFile.Reader reader = new SequenceFile.Reader(fs, file.getPath(), conf);
           try {
             IntWritable clusterId = new IntWritable(0);
             VectorWritable point = new VectorWritable();
@@ -152,51 +138,62 @@ public class TestCDbwEvaluator extends M
   }
 
   public void testCanopy() throws Exception { // now run the Job
-    CanopyDriver.runJob("testdata", "output", EuclideanDistanceMeasure.class.getName(), 3.1, 2.1, true);
+    CanopyDriver.runJob(getTestTempDirPath("testdata"), getTestTempDirPath("output"),
+                        EuclideanDistanceMeasure.class.getName(), 3.1, 2.1, true);
     int numIterations = 2;
-    CDbwDriver.runJob("output/clusters-0", "output/clusteredPoints", "output", EuclideanDistanceMeasure.class.getName(),
-        numIterations, 1);
+    Path output = getTestTempDirPath("output");
+    CDbwDriver.runJob(new Path(output, "clusters-0"), new Path(output, "clusteredPoints"), output,
+                      EuclideanDistanceMeasure.class.getName(), numIterations, 1);
     checkRefPoints(numIterations);
   }
 
   public void testKmeans() throws Exception {
     // now run the Canopy job to prime kMeans canopies
-    CanopyDriver.runJob("testdata", "output", EuclideanDistanceMeasure.class.getName(), 3.1, 2.1, false);
+    CanopyDriver.runJob(getTestTempDirPath("testdata"), getTestTempDirPath("output"),
+                        EuclideanDistanceMeasure.class.getName(), 3.1, 2.1, false);
     // now run the KMeans job
-    KMeansDriver.runJob("testdata", "output/clusters-0", "output", EuclideanDistanceMeasure.class.getName(), 0.001, 10, 1, true);
+    Path output = getTestTempDirPath("output");
+    KMeansDriver.runJob(getTestTempDirPath("testdata"), new Path(output, "clusters-0"), output,
+                        EuclideanDistanceMeasure.class.getName(), 0.001, 10, 1, true);
     int numIterations = 2;
-    CDbwDriver.runJob("output/clusters-2", "output/clusteredPoints", "output", EuclideanDistanceMeasure.class.getName(),
-        numIterations, 1);
+    CDbwDriver.runJob(new Path(output, "clusters-2"), new Path(output, "clusteredPoints"), output,
+                      EuclideanDistanceMeasure.class.getName(), numIterations, 1);
     checkRefPoints(numIterations);
   }
 
   public void testFuzzyKmeans() throws Exception {
     // now run the Canopy job to prime kMeans canopies
-    CanopyDriver.runJob("testdata", "output", EuclideanDistanceMeasure.class.getName(), 3.1, 2.1, false);
+    CanopyDriver.runJob(getTestTempDirPath("testdata"), getTestTempDirPath("output"),
+                        EuclideanDistanceMeasure.class.getName(), 3.1, 2.1, false);
     // now run the KMeans job
-    FuzzyKMeansDriver.runJob("testdata", "output/clusters-0", "output", EuclideanDistanceMeasure.class.getName(), 0.001, 10, 1, 1,
-        2, false, true, 0);
+    Path output = getTestTempDirPath("output");
+    FuzzyKMeansDriver.runJob(getTestTempDirPath("testdata"), new Path(output, "clusters-0"), output,
+                             EuclideanDistanceMeasure.class.getName(), 0.001, 10, 1, 1, 2, false, true, 0);
     int numIterations = 2;
-    CDbwDriver.runJob("output/clusters-4", "output/clusteredPoints", "output", EuclideanDistanceMeasure.class.getName(),
-        numIterations, 1);
+    CDbwDriver.runJob(new Path(output, "clusters-4"), new Path(output, "clusteredPoints"), output,
+                      EuclideanDistanceMeasure.class.getName(), numIterations, 1);
     checkRefPoints(numIterations);
   }
 
   public void testMeanShift() throws Exception {
-    MeanShiftCanopyDriver.runJob("testdata", "output", EuclideanDistanceMeasure.class.getName(), 2.1, 1.0, 0.001, 10, false, true);
+    MeanShiftCanopyDriver.runJob(getTestTempDirPath("testdata"), getTestTempDirPath("output"),
+                                 EuclideanDistanceMeasure.class.getName(), 2.1, 1.0, 0.001, 10, false, true);
     int numIterations = 2;
-    CDbwDriver.runJob("output/clusters-2", "output/clusteredPoints", "output", EuclideanDistanceMeasure.class.getName(),
-        numIterations, 1);
+    Path output = getTestTempDirPath("output");
+    CDbwDriver.runJob(new Path(output, "clusters-2"), new Path(output, "clusteredPoints"), output,
+                      EuclideanDistanceMeasure.class.getName(), numIterations, 1);
     checkRefPoints(numIterations);
   }
 
   public void testDirichlet() throws Exception {
     Vector prototype = new DenseVector(2);
-    DirichletDriver.runJob("testdata", "output", L1ModelDistribution.class.getName(), prototype.getClass().getName(), prototype
-        .size(), 15, 5, 1.0, 1, true, true, 0);
+    DirichletDriver.runJob(getTestTempDirPath("testdata"), getTestTempDirPath("output"),
+                           L1ModelDistribution.class.getName(), prototype.getClass().getName(),
+                           prototype.size(), 15, 5, 1.0, 1, true, true, 0);
     int numIterations = 2;
-    CDbwDriver.runJob("output/clusters-5", "output/clusteredPoints", "output", EuclideanDistanceMeasure.class.getName(),
-        numIterations, 1);
+    Path output = getTestTempDirPath("output");
+    CDbwDriver.runJob(new Path(output, "clusters-5"), new Path(output, "clusteredPoints"), output,
+                      EuclideanDistanceMeasure.class.getName(), numIterations, 1);
     checkRefPoints(numIterations);
   }
 

Modified: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizerTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizerTest.java?rev=942056&r1=942055&r2=942056&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizerTest.java (original)
+++ lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizerTest.java Fri May  7 12:25:11 2010
@@ -17,7 +17,6 @@
 
 package org.apache.mahout.utils.vectors.text;
 
-import java.io.File;
 import java.io.IOException;
 import java.net.URISyntaxException;
 import java.util.Random;
@@ -58,7 +57,7 @@ public class DictionaryVectorizerTest ex
   private FileSystem fs;
   
   private static char getRandomDelimiter() {
-    return DictionaryVectorizerTest.DELIM.charAt(DictionaryVectorizerTest.random.nextInt(DictionaryVectorizerTest.DELIM.length()));
+    return DELIM.charAt(random.nextInt(DictionaryVectorizerTest.DELIM.length()));
   }
   
   public static String getRandomDocument() {
@@ -96,35 +95,16 @@ public class DictionaryVectorizerTest ex
     return sb.toString();
   }
   
-  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();
-    }
-  }
-  
   @Override
   public void setUp() throws Exception {
     super.setUp();
-    rmr("target/output");
-    rmr("target/testdata");
     Configuration conf = new Configuration();
     fs = FileSystem.get(conf);
   }
   
-  public void testCreateTermFrequencyVectors() throws IOException,
-  InterruptedException,
-  ClassNotFoundException,
-  URISyntaxException {
+  public void testCreateTermFrequencyVectors() throws IOException {
     Configuration conf = new Configuration();
-    String pathString = "target/testdata/documents/docs.file";
-    Path path = new Path(pathString);
+    Path path = getTestTempFilePath("documents/docs.file");
     SequenceFile.Writer writer = new SequenceFile.Writer(fs, conf, path,
       Text.class, Text.class);
     
@@ -134,11 +114,12 @@ public class DictionaryVectorizerTest ex
     }
     writer.close();
     Class<? extends Analyzer> analyzer = StandardAnalyzer.class;
-    DocumentProcessor.tokenizeDocuments(pathString, analyzer,
-    "target/output/tokenized-documents");
-    DictionaryVectorizer.createTermFrequencyVectors("target/output/tokenized-documents",
-      "target/output/wordcount", 2, 1, 0.0f, 1, 100, false);
-    TFIDFConverter.processTfIdf("target/output/wordcount/vectors", "target/output/tfidf/", 100, 1, 99, 1.0f, false);
+    DocumentProcessor.tokenizeDocuments(path, analyzer,
+    getTestTempDirPath("output/tokenized-documents"));
+    DictionaryVectorizer.createTermFrequencyVectors(getTestTempDirPath("output/tokenized-documents"),
+      getTestTempDirPath("output/wordcount"), 2, 1, 0.0f, 1, 100, false);
+    TFIDFConverter.processTfIdf(getTestTempDirPath("output/wordcount/vectors"),
+                                getTestTempDirPath("output/tfidf"), 100, 1, 99, 1.0f, false);
     
   }
 }