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

svn commit: r939509 - in /lucene/mahout/trunk: core/src/main/java/org/apache/mahout/clustering/ core/src/main/java/org/apache/mahout/clustering/canopy/ core/src/main/java/org/apache/mahout/clustering/dirichlet/ core/src/main/java/org/apache/mahout/clus...

Author: jeastman
Date: Thu Apr 29 23:22:18 2010
New Revision: 939509

URL: http://svn.apache.org/viewvc?rev=939509&view=rev
Log:
MAHOUT-236: Reimplemented ClusterDumper printouts for clustered points and added tests thereof. May have broken ClusterLabels or changed it at least but no tests fail.

Modified:
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/ClusterBase.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyClusterer.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/ClusterMapper.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletClusterMapper.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletDriver.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopy.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyClusterMapper.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/canopy/TestCanopyCreation.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/meanshift/TestMeanShift.java
    lucene/mahout/trunk/examples/   (props changed)
    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/test/java/org/apache/mahout/clustering/TestClusterDumper.java

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/ClusterBase.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/ClusterBase.java?rev=939509&r1=939508&r2=939509&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/ClusterBase.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/ClusterBase.java Thu Apr 29 23:22:18 2010
@@ -26,6 +26,7 @@ import java.util.Locale;
 
 import org.apache.hadoop.io.Writable;
 import org.apache.mahout.math.JsonVectorAdapter;
+import org.apache.mahout.math.NamedVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.math.Vector.Element;
 
@@ -39,7 +40,7 @@ import com.google.gson.reflect.TypeToken
  *
  */
 public abstract class ClusterBase implements Writable, Cluster {
-  
+
   // default directory for all clustered points
   public static final String CLUSTERED_POINTS_DIR = "/clusteredPoints";
 
@@ -49,7 +50,6 @@ public abstract class ClusterBase implem
   // default directory for output of clusters per iteration
   public static final String CLUSTERS_DIR = "/clusters-";
 
-
   // this cluster's clusterId
   private int id;
 
@@ -107,7 +107,7 @@ public abstract class ClusterBase implem
   @Override
   public String asFormatString(String[] bindings) {
     StringBuilder buf = new StringBuilder();
-    buf.append(getIdentifier()).append(": ").append(formatVector(computeCentroid(), bindings));
+    buf.append(getIdentifier()).append(": ").append(formatVector(getCenter(), bindings));
     return buf.toString();
   }
 
@@ -152,6 +152,9 @@ public abstract class ClusterBase implem
    */
   public static String formatVector(Vector v, String[] bindings) {
     StringBuilder buf = new StringBuilder();
+    if (v instanceof NamedVector) {
+      buf.append(((NamedVector) v).getName()).append(" = ");
+    }
     int nzero = 0;
     Iterator<Element> iterateNonZero = v.iterateNonZero();
     while (iterateNonZero.hasNext()) {

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyClusterer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyClusterer.java?rev=939509&r1=939508&r2=939509&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyClusterer.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyClusterer.java Thu Apr 29 23:22:18 2010
@@ -150,9 +150,7 @@ public class CanopyClusterer {
   }
 
   /**
-   * This method is used by the CanopyMapper to perform canopy inclusion tests and to emit the point keyed by
-   * its covering canopies to the output. if the point is not covered by any canopies (due to canopy centroid
-   * clustering), emit the point to the closest covering canopy.
+   * Emit the point to the closest covering canopy. Used by the ClusterMapper.
    * 
    * @param point
    *          the point to be added
@@ -163,29 +161,21 @@ public class CanopyClusterer {
    * @param reporter
    *          to report status of the job
    */
-  public void emitPointToExistingCanopies(Vector point, List<Canopy> canopies,
+  public void emitPointToClosestCanopy(Vector point, List<Canopy> canopies,
       OutputCollector<IntWritable, WeightedVectorWritable> collector, Reporter reporter) throws IOException {
     double minDist = Double.MAX_VALUE;
     Canopy closest = null;
-    boolean isCovered = false;
+    // find closest canopy
     for (Canopy canopy : canopies) {
       double dist = measure.distance(canopy.getCenter().getLengthSquared(), canopy.getCenter(), point);
-      if (dist < t1) {
-        isCovered = true;
-        VectorWritable vw = new VectorWritable(point);
-        collector.collect(new IntWritable(canopy.getId()), new WeightedVectorWritable(1, vw));
-        reporter.setStatus("Emit Canopy ID:" + canopy.getIdentifier());
-      } else if (dist < minDist) {
+      if (dist < minDist) {
         minDist = dist;
         closest = canopy;
       }
     }
-    // if the point is not contained in any canopies (due to canopy centroid
-    // clustering), emit the point to the closest covering canopy.
-    if (!isCovered) {
-      collector.collect(new IntWritable(closest.getId()), new WeightedVectorWritable(1, new VectorWritable(point)));
-      reporter.setStatus("Emit Closest Canopy ID:" + closest.getIdentifier());
-    }
+    // emit to closest canopy
+    collector.collect(new IntWritable(closest.getId()), new WeightedVectorWritable(1, new VectorWritable(point)));
+    reporter.setStatus("Emit Closest Canopy ID:" + closest.getIdentifier());
   }
 
   /**

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/ClusterMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/ClusterMapper.java?rev=939509&r1=939508&r2=939509&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/ClusterMapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/ClusterMapper.java Thu Apr 29 23:22:18 2010
@@ -37,19 +37,18 @@ import org.apache.mahout.math.Vector;
 import org.apache.mahout.math.VectorWritable;
 
 public class ClusterMapper extends MapReduceBase implements
-    Mapper<WritableComparable<?>,VectorWritable,IntWritable,WeightedVectorWritable> {
-  
+    Mapper<WritableComparable<?>, VectorWritable, IntWritable, WeightedVectorWritable> {
+
   private CanopyClusterer canopyClusterer;
+
   private final List<Canopy> canopies = new ArrayList<Canopy>();
-  
+
   @Override
-  public void map(WritableComparable<?> key,
-                  VectorWritable point,
-                  OutputCollector<IntWritable,WeightedVectorWritable> output,
-                  Reporter reporter) throws IOException {
-    canopyClusterer.emitPointToExistingCanopies(point.get(), canopies, output, reporter);
+  public void map(WritableComparable<?> key, VectorWritable point, OutputCollector<IntWritable, WeightedVectorWritable> output,
+      Reporter reporter) throws IOException {
+    canopyClusterer.emitPointToClosestCanopy(point.get(), canopies, output, reporter);
   }
-  
+
   /**
    * Configure the mapper by providing its canopies. Used by unit tests.
    * 
@@ -60,12 +59,12 @@ public class ClusterMapper extends MapRe
     this.canopies.clear();
     this.canopies.addAll(canopies);
   }
-  
+
   @Override
   public void configure(JobConf job) {
     super.configure(job);
     canopyClusterer = new CanopyClusterer(job);
-    
+
     String canopyPath = job.get(CanopyConfigKeys.CANOPY_PATH_KEY);
     if ((canopyPath != null) && (canopyPath.length() > 0)) {
       try {
@@ -85,13 +84,13 @@ public class ClusterMapper extends MapRe
       } catch (IOException e) {
         throw new IllegalStateException(e);
       }
-      
+
       if (canopies.isEmpty()) {
         throw new IllegalStateException("Canopies are empty!");
       }
     }
   }
-  
+
   public boolean canopyCovers(Canopy canopy, Vector point) {
     return canopyClusterer.canopyCovers(canopy, point);
   }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletClusterMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletClusterMapper.java?rev=939509&r1=939508&r2=939509&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletClusterMapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletClusterMapper.java Thu Apr 29 23:22:18 2010
@@ -59,7 +59,7 @@ public class DirichletClusterMapper exte
         clusterPdf = pdf;
       }
     }
-    System.out.println(clusterId + ": " + ClusterBase.formatVector(vector.get(), null));
+    //System.out.println(clusterId + ": " + ClusterBase.formatVector(vector.get(), null));
     output.collect(new IntWritable(clusterId), new WeightedVectorWritable(clusterPdf, vector));
   }
 

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletDriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletDriver.java?rev=939509&r1=939508&r2=939509&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletDriver.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletDriver.java Thu Apr 29 23:22:18 2010
@@ -311,9 +311,9 @@ public class DirichletDriver {
    * @param modelFactory
    *          the class name of the model factory class
    * @param modelPrototype
-   *          TODO
+   *          the class name of the model prototype (a Vector implementation)
    * @param prototypeSize
-   *          TODO
+   *          the size of the model prototype vector
    * @param numClusters
    *          the number of clusters
    * @param alpha_0

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopy.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopy.java?rev=939509&r1=939508&r2=939509&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopy.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopy.java Thu Apr 29 23:22:18 2010
@@ -131,7 +131,7 @@ public class MeanShiftCanopy extends Clu
    */
   @Override
   public Vector computeCentroid() {
-    if (getNumPoints() == 0) {
+    if (getNumPoints() <= 1) {
       return getCenter();
     } else {
       return getPointTotal().divide(getNumPoints());
@@ -204,7 +204,7 @@ public class MeanShiftCanopy extends Clu
   @Override
   public void write(DataOutput out) throws IOException {
     super.write(out);
-    VectorWritable.writeVector(out, new SequentialAccessSparseVector(computeCentroid()));
+    VectorWritable.writeVector(out, computeCentroid());
     out.writeInt(boundPoints.size());
     for (int v : boundPoints.elements()) {
       out.writeInt(v);

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyClusterMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyClusterMapper.java?rev=939509&r1=939508&r2=939509&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyClusterMapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyClusterMapper.java Thu Apr 29 23:22:18 2010
@@ -41,21 +41,18 @@ import org.apache.mahout.math.VectorWrit
 public class MeanShiftCanopyClusterMapper extends MapReduceBase implements
     Mapper<WritableComparable<?>, MeanShiftCanopy, IntWritable, WeightedVectorWritable> {
 
-  private MeanShiftCanopyClusterer clusterer;
-
-  private OutputCollector<IntWritable, WeightedVectorWritable> output;
-
   private List<MeanShiftCanopy> canopies;
 
   @Override
-  public void map(WritableComparable<?> key, MeanShiftCanopy vector, OutputCollector<IntWritable, WeightedVectorWritable> output,
+  public void map(WritableComparable<?> key, MeanShiftCanopy canopy, OutputCollector<IntWritable, WeightedVectorWritable> output,
       Reporter reporter) throws IOException {
-    int vectorId = vector.getId();
+    // canopies use canopyIds assigned when input vectors are processed as vectorIds too
+    int vectorId = canopy.getId();
     for (MeanShiftCanopy msc : canopies) {
       for (int containedId : msc.getBoundPoints().toList()) {
         if (vectorId == containedId) {
           // System.out.println(msc.getId() + ": v" + vectorId + "=" + ClusterBase.formatVector(vector.getCenter(), null));
-          output.collect(new IntWritable(msc.getId()), new WeightedVectorWritable(1, new VectorWritable(vector.getCenter())));
+          output.collect(new IntWritable(msc.getId()), new WeightedVectorWritable(1, new VectorWritable(canopy.getCenter())));
         }
       }
     }

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=939509&r1=939508&r2=939509&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 Apr 29 23:22:18 2010
@@ -504,8 +504,7 @@ public class TestCanopyCreation extends 
       count++;
       System.out.println("Txt: " + clusterId + " Vec: " + vector.getVector().get().asFormatString());
     }
-    // the point [3.0,3.0] is covered by both canopies
-    assertEquals("number of points", 1 + points.size(), count);
+    assertEquals("number of points", points.size(), count);
     reader.close();
   }
   
@@ -527,19 +526,12 @@ public class TestCanopyCreation extends 
     Path path = new Path("output/clusteredPoints/part-00000");
     SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, conf);
     int count = 0;
-    /*
-     * while (reader.ready()) { System.out.println(reader.readLine()); count++; }
-     */
     IntWritable canopyId = new IntWritable(0);
     WeightedVectorWritable can = new WeightedVectorWritable();
     while (reader.next(canopyId, can)) {
       count++;
     }
-    /*
-     * while (reader.ready()) { System.out.println(reader.readLine()); count++; }
-     */
-    // the point [3.0,3.0] is covered by both canopies
-    assertEquals("number of points", 1 + points.size(), count);
+    assertEquals("number of points", points.size(), count);
     reader.close();
   }
   

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=939509&r1=939508&r2=939509&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 Apr 29 23:22:18 2010
@@ -57,7 +57,7 @@ public class TestMeanShift extends Mahou
    */
   private static void printCanopies(List<MeanShiftCanopy> canopies) {
     for (MeanShiftCanopy canopy : canopies) {
-      System.out.println(canopy.toString());
+      System.out.println(canopy.asFormatString(null));
     }
   }
 
@@ -70,7 +70,7 @@ public class TestMeanShift extends Mahou
       }
     }
     for (MeanShiftCanopy canopy : canopies) {
-      int ch = 'A' + canopy.getCanopyId() - 100;
+      int ch = 'A' + canopy.getCanopyId();
       for (int pid : canopy.getBoundPoints().elements()) {
         Vector pt = raw[pid];
         out[(int) pt.getQuick(0)][(int) pt.getQuick(1)] = (char) ch;

Propchange: lucene/mahout/trunk/examples/
------------------------------------------------------------------------------
--- svn:ignore (original)
+++ svn:ignore Thu Apr 29 23:22:18 2010
@@ -12,3 +12,4 @@ dist
 .checkstyle
 .pmd
 .ruleset
+testdata

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=939509&r1=939508&r2=939509&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 Thu Apr 29 23:22:18 2010
@@ -44,6 +44,7 @@ import org.apache.commons.cli2.commandli
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
@@ -51,6 +52,7 @@ import org.apache.hadoop.mapred.JobClien
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.jobcontrol.Job;
 import org.apache.mahout.clustering.Cluster;
+import org.apache.mahout.clustering.WeightedVectorWritable;
 import org.apache.mahout.common.CommandLineUtil;
 import org.apache.mahout.common.Pair;
 import org.apache.mahout.math.Vector;
@@ -76,7 +78,7 @@ public final class ClusterDumper {
 
   private int numTopFeatures = 10;
 
-  private Map<String, List<String>> clusterIdToPoints = null;
+  private Map<Integer, List<WeightedVectorWritable>> clusterIdToPoints = null;
 
   private boolean useJSON = false;
 
@@ -154,14 +156,14 @@ public final class ClusterDumper {
           writer.write('\n');
         }
 
-        List<String> points = clusterIdToPoints.get(String.valueOf(cluster.getId()));
+        List<WeightedVectorWritable> points = clusterIdToPoints.get(cluster.getId());
         if (points != null) {
-          writer.write("\tPoints: ");
-          for (Iterator<String> iterator = points.iterator(); iterator.hasNext();) {
-            String point = iterator.next();
-            writer.append(point);
+          writer.write("\tWeight:  Point:\n\t");
+          for (Iterator<WeightedVectorWritable> iterator = points.iterator(); iterator.hasNext();) {
+            WeightedVectorWritable point = iterator.next();
+            writer.append(point.toString());
             if (iterator.hasNext()) {
-              writer.append(", ");
+              writer.append("\n\t");
             }
           }
           writer.write('\n');
@@ -192,7 +194,7 @@ public final class ClusterDumper {
     this.subString = subString;
   }
 
-  public Map<String, List<String>> getClusterIdToPoints() {
+  public Map<Integer, List<WeightedVectorWritable>> getClusterIdToPoints() {
     return clusterIdToPoints;
   }
 
@@ -228,26 +230,24 @@ public final class ClusterDumper {
         abuilder.withName("substring").withMinimum(1).withMaximum(1).create()).withDescription(
         "The number of chars of the asFormatString() to print").withShortName("b").create();
     Option numWordsOpt = obuilder.withLongName("numWords").withRequired(false).withArgument(
-        abuilder.withName("numWords").withMinimum(1).withMaximum(1).create()).withDescription(
-        "The number of top terms to print").withShortName("n").create();
+        abuilder.withName("numWords").withMinimum(1).withMaximum(1).create()).withDescription("The number of top terms to print")
+        .withShortName("n").create();
     Option centroidJSonOpt = obuilder.withLongName("json").withRequired(false).withDescription(
-        "Output the centroid as JSON.  Otherwise it substitues in the terms for vector cell entries")
-        .withShortName("j").create();
+        "Output the centroid as JSON.  Otherwise it substitues in the terms for vector cell entries").withShortName("j").create();
     Option pointsOpt = obuilder.withLongName("pointsDir").withRequired(false).withArgument(
         abuilder.withName("pointsDir").withMinimum(1).withMaximum(1).create()).withDescription(
         "The directory containing points sequence files mapping input vectors to their cluster.  "
-            + "If specified, then the program will output the points associated with a cluster").withShortName("p")
-        .create();
+            + "If specified, then the program will output the points associated with a cluster").withShortName("p").create();
     Option dictOpt = obuilder.withLongName("dictionary").withRequired(false).withArgument(
-        abuilder.withName("dictionary").withMinimum(1).withMaximum(1).create())
-        .withDescription("The dictionary file. ").withShortName("d").create();
+        abuilder.withName("dictionary").withMinimum(1).withMaximum(1).create()).withDescription("The dictionary file. ")
+        .withShortName("d").create();
     Option dictTypeOpt = obuilder.withLongName("dictionaryType").withRequired(false).withArgument(
         abuilder.withName("dictionaryType").withMinimum(1).withMaximum(1).create()).withDescription(
         "The dictionary file type (text|sequencefile)").withShortName("dt").create();
     Option helpOpt = obuilder.withLongName("help").withDescription("Print out help").withShortName("h").create();
 
-    Group group = gbuilder.withName("Options").withOption(helpOpt).withOption(seqOpt).withOption(outputOpt).withOption(
-        substringOpt).withOption(pointsOpt).withOption(centroidJSonOpt).withOption(dictOpt).withOption(dictTypeOpt)
+    Group group = gbuilder.withName("Options").withOption(helpOpt).withOption(seqOpt).withOption(outputOpt)
+        .withOption(substringOpt).withOption(pointsOpt).withOption(centroidJSonOpt).withOption(dictOpt).withOption(dictTypeOpt)
         .withOption(numWordsOpt).create();
 
     try {
@@ -318,8 +318,8 @@ public final class ClusterDumper {
     this.useJSON = json;
   }
 
-  private static Map<String, List<String>> readPoints(String pointsPathDir, JobConf conf) throws IOException {
-    SortedMap<String, List<String>> result = new TreeMap<String, List<String>>();
+  private static Map<Integer, List<WeightedVectorWritable>> readPoints(String pointsPathDir, JobConf conf) throws IOException {
+    Map<Integer, List<WeightedVectorWritable>> result = new TreeMap<Integer, List<WeightedVectorWritable>>();
 
     File[] children = new File(pointsPathDir).listFiles(new FilenameFilter() {
       @Override
@@ -337,19 +337,20 @@ public final class ClusterDumper {
       FileSystem fs = FileSystem.get(path.toUri(), conf);
       SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, conf);
       try {
-        Text key = (Text) reader.getKeyClass().newInstance();
-        Text value = (Text) reader.getValueClass().newInstance();
+        IntWritable key = (IntWritable) reader.getKeyClass().newInstance();
+        WeightedVectorWritable value = (WeightedVectorWritable) reader.getValueClass().newInstance();
         while (reader.next(key, value)) {
           // 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();
-          List<String> pointList = result.get(clusterId);
+          List<WeightedVectorWritable> pointList = result.get(key.get());
           if (pointList == null) {
-            pointList = new ArrayList<String>();
-            result.put(clusterId, pointList);
+            pointList = new ArrayList<WeightedVectorWritable>();
+            result.put(key.get(), pointList);
           }
-          pointList.add(key.toString());
+          pointList.add(value);
+          value = (WeightedVectorWritable) reader.getValueClass().newInstance();
         }
       } catch (InstantiationException e) {
         log.error("Exception", e);
@@ -363,7 +364,9 @@ public final class ClusterDumper {
 
   static class TermIndexWeight {
     private int index = -1;
+
     private double weight = 0.0;
+
     TermIndexWeight(int index, double weight) {
       this.index = index;
       this.weight = weight;

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=939509&r1=939508&r2=939509&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 Thu Apr 29 23:22:18 2010
@@ -29,6 +29,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.Map.Entry;
 
 import org.apache.commons.cli2.CommandLine;
 import org.apache.commons.cli2.Group;
@@ -48,7 +49,10 @@ import org.apache.lucene.index.TermEnum;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.util.OpenBitSet;
+import org.apache.mahout.clustering.WeightedVectorWritable;
 import org.apache.mahout.common.CommandLineUtil;
+import org.apache.mahout.math.NamedVector;
+import org.apache.mahout.math.Vector;
 import org.apache.mahout.math.stats.LogLikelihood;
 import org.apache.mahout.utils.clustering.ClusterDumper;
 import org.apache.mahout.utils.vectors.TermEntry;
@@ -65,19 +69,22 @@ import org.slf4j.LoggerFactory;
  * More about LLR can be found at : http://tdunning.blogspot.com/2008/03/surprise-and-coincidence.html
  */
 public class ClusterLabels {
-  
+
   class TermInfoClusterInOut implements Comparable<TermInfoClusterInOut> {
     private final String term;
+
     private final int inClusterDF;
+
     private final int outClusterDF;
+
     private double logLikelihoodRatio;
-    
+
     TermInfoClusterInOut(String term, int inClusterDF, int outClusterDF) {
       this.term = term;
       this.inClusterDF = inClusterDF;
       this.outClusterDF = outClusterDF;
     }
-    
+
     @Override
     public int compareTo(TermInfoClusterInOut that) {
       int res = -Double.compare(logLikelihoodRatio, that.logLikelihoodRatio);
@@ -86,32 +93,38 @@ public class ClusterLabels {
       }
       return res;
     }
-    
+
     public int getInClusterDiff() {
       return this.inClusterDF - this.outClusterDF;
     }
   }
-  
+
   private static final Logger log = LoggerFactory.getLogger(ClusterLabels.class);
+
   public static final int DEFAULT_MIN_IDS = 50;
+
   public static final int DEFAULT_MAX_LABELS = 25;
-  
+
   private final String seqFileDir;
+
   private final String pointsDir;
+
   private final String indexDir;
+
   private final String contentField;
+
   private String idField;
-  private Map<String,List<String>> clusterIdToPoints = null;
+
+  private Map<Integer, List<WeightedVectorWritable>> clusterIdToPoints = null;
+
   private String output;
+
   private int minNumIds = DEFAULT_MIN_IDS;
+
   private int maxLabels = DEFAULT_MAX_LABELS;
-  
-  public ClusterLabels(String seqFileDir,
-                       String pointsDir,
-                       String indexDir,
-                       String contentField,
-                       int minNumIds,
-                       int maxLabels) throws IOException {
+
+  public ClusterLabels(String seqFileDir, String pointsDir, String indexDir, String contentField, int minNumIds, int maxLabels)
+      throws IOException {
     this.seqFileDir = seqFileDir;
     this.pointsDir = pointsDir;
     this.indexDir = indexDir;
@@ -120,34 +133,33 @@ public class ClusterLabels {
     this.maxLabels = maxLabels;
     init();
   }
-  
+
   private void init() throws IOException {
     ClusterDumper clusterDumper = new ClusterDumper(seqFileDir, pointsDir);
     this.clusterIdToPoints = clusterDumper.getClusterIdToPoints();
   }
-  
+
   public void getLabels() throws IOException {
-    
+
     Writer writer;
     if (this.output != null) {
       writer = new FileWriter(this.output);
     } else {
       writer = new OutputStreamWriter(System.out);
     }
-    
-    for (Map.Entry<String,List<String>> stringListEntry : clusterIdToPoints.entrySet()) {
-      List<String> ids = stringListEntry.getValue();
-      List<TermInfoClusterInOut> termInfos = getClusterLabels(stringListEntry.getKey(), ids);
+
+    for (Entry<Integer, List<WeightedVectorWritable>> integerListEntry : clusterIdToPoints.entrySet()) {
+      List<WeightedVectorWritable> wvws = integerListEntry.getValue();
+      List<TermInfoClusterInOut> termInfos = getClusterLabels(integerListEntry.getKey(), wvws);
       if (termInfos != null) {
         writer.write('\n');
-        writer.write("Top labels for Cluster " + stringListEntry.getKey() + " containing " + ids.size()
-                     + " vectors");
+        writer.write("Top labels for Cluster " + integerListEntry.getKey() + " containing " + wvws.size() + " vectors");
         writer.write('\n');
         writer.write("Term \t\t LLR \t\t In-ClusterDF \t\t Out-ClusterDF ");
         writer.write('\n');
         for (TermInfoClusterInOut termInfo : termInfos) {
-          writer.write(termInfo.term + "\t\t" + termInfo.logLikelihoodRatio + "\t\t" + termInfo.inClusterDF
-                       + "\t\t" + termInfo.outClusterDF);
+          writer.write(termInfo.term + "\t\t" + termInfo.logLikelihoodRatio + "\t\t" + termInfo.inClusterDF + "\t\t"
+              + termInfo.outClusterDF);
           writer.write('\n');
         }
       }
@@ -157,38 +169,43 @@ public class ClusterLabels {
       writer.close();
     }
   }
-  
+
   /**
    * Get the list of labels, sorted by best score.
    * 
-   * @param clusterID
-   * @param ids
+   * @param integer
+   * @param wvws
    * @return
    * @throws CorruptIndexException
    * @throws IOException
    */
-  protected List<TermInfoClusterInOut> getClusterLabels(String clusterID, List<String> ids) throws IOException {
-    
-    if (ids.size() < minNumIds) {
-      log.info("Skipping small cluster {} with size: {}", clusterID, ids.size());
+  protected List<TermInfoClusterInOut> getClusterLabels(Integer integer, List<WeightedVectorWritable> wvws) throws IOException {
+
+    if (wvws.size() < minNumIds) {
+      log.info("Skipping small cluster {} with size: {}", integer, wvws.size());
       return null;
     }
-    
-    log.info("Processing Cluster {} with {} documents", clusterID, ids.size());
+
+    log.info("Processing Cluster {} with {} documents", integer, wvws.size());
     Directory dir = FSDirectory.open(new File(this.indexDir));
     IndexReader reader = IndexReader.open(dir, false);
-    
+
     log.info("# of documents in the index {}", reader.numDocs());
-    
+
     Set<String> idSet = new HashSet<String>();
-    idSet.addAll(ids);
-    
+    for (WeightedVectorWritable wvw : wvws) {
+      Vector vector = wvw.getVector().get();
+      if (vector instanceof NamedVector) {
+        idSet.add(((NamedVector) vector).getName());
+      }
+    }
+
     int numDocs = reader.numDocs();
-    
+
     OpenBitSet clusterDocBitset = getClusterDocBitset(reader, idSet, this.idField);
-    
+
     log.info("Populating term infos from the index");
-    
+
     /**
      * This code is as that of CachedTermInfo, with one major change, which is to get the document frequency.
      * 
@@ -198,38 +215,38 @@ public class ClusterLabels {
      * frequencies in each document. The number of results of this call will be the in-cluster document
      * frequency.
      */
-    
+
     TermEnum te = reader.terms(new Term(contentField, ""));
     int count = 0;
-    
-    Map<String,TermEntry> termEntryMap = new LinkedHashMap<String,TermEntry>();
+
+    Map<String, TermEntry> termEntryMap = new LinkedHashMap<String, TermEntry>();
     do {
       Term term = te.term();
       if (term == null || term.field().equals(contentField) == false) {
         break;
       }
       OpenBitSet termBitset = new OpenBitSet(reader.maxDoc());
-      
+
       // Generate bitset for the term
       TermDocs termDocs = reader.termDocs(term);
-      
+
       while (termDocs.next()) {
         termBitset.set(termDocs.doc());
       }
-      
+
       // AND the term's bitset with cluster doc bitset to get the term's in-cluster frequency.
       // This modifies the termBitset, but that's fine as we are not using it anywhere else.
       termBitset.and(clusterDocBitset);
       int inclusterDF = (int) termBitset.cardinality();
-      
+
       TermEntry entry = new TermEntry(term.text(), count++, inclusterDF);
       termEntryMap.put(entry.term, entry);
     } while (te.next());
     te.close();
-    
+
     List<TermInfoClusterInOut> clusteredTermInfo = new LinkedList<TermInfoClusterInOut>();
-    
-    int clusterSize = ids.size();
+
+    int clusterSize = wvws.size();
 
     for (TermEntry termEntry : termEntryMap.values()) {
       int corpusDF = reader.terms(new Term(this.contentField, termEntry.term)).docFreq();
@@ -239,25 +256,22 @@ public class ClusterLabels {
       termInfoCluster.logLikelihoodRatio = scoreDocumentFrequencies(inDF, outDF, clusterSize, numDocs);
       clusteredTermInfo.add(termInfoCluster);
     }
-    
+
     Collections.sort(clusteredTermInfo);
     // Cleanup
     reader.close();
     termEntryMap.clear();
-    
+
     return clusteredTermInfo.subList(0, Math.min(clusteredTermInfo.size(), maxLabels));
   }
-  
-  private static OpenBitSet getClusterDocBitset(IndexReader reader,
-                                                Set<String> idSet,
-                                                String idField) throws IOException {
+
+  private static OpenBitSet getClusterDocBitset(IndexReader reader, Set<String> idSet, String idField) throws IOException {
     int numDocs = reader.numDocs();
-    
+
     OpenBitSet bitset = new OpenBitSet(numDocs);
-    
-    FieldSelector idFieldSelector = new SetBasedFieldSelector(Collections.singleton(idField), Collections
-        .emptySet());
-    
+
+    FieldSelector idFieldSelector = new SetBasedFieldSelector(Collections.singleton(idField), Collections.emptySet());
+
     for (int i = 0; i < numDocs; i++) {
       String id = null;
       // Use Lucene's internal ID if idField is not specified. Else, get it from the document.
@@ -273,92 +287,89 @@ public class ClusterLabels {
     log.info("Created bitset for in-cluster documents : {}", bitset.cardinality());
     return bitset;
   }
-  
+
   private static double scoreDocumentFrequencies(int inDF, int outDF, int clusterSize, int corpusSize) {
     int k12 = clusterSize - inDF;
     int k22 = corpusSize - clusterSize - outDF;
-    
+
     return LogLikelihood.logLikelihoodRatio(inDF, k12, outDF, k22);
   }
-  
+
   public String getIdField() {
     return idField;
   }
-  
+
   public void setIdField(String idField) {
     this.idField = idField;
   }
-  
+
   public String getOutput() {
     return output;
   }
-  
+
   public void setOutput(String output) {
     this.output = output;
   }
-  
+
   public static void main(String[] args) {
-    
+
     DefaultOptionBuilder obuilder = new DefaultOptionBuilder();
     ArgumentBuilder abuilder = new ArgumentBuilder();
     GroupBuilder gbuilder = new GroupBuilder();
-    
+
     Option indexOpt = obuilder.withLongName("dir").withRequired(true).withArgument(
-      abuilder.withName("dir").withMinimum(1).withMaximum(1).create()).withDescription(
-      "The Lucene index directory").withShortName("d").create();
-    
+        abuilder.withName("dir").withMinimum(1).withMaximum(1).create()).withDescription("The Lucene index directory")
+        .withShortName("d").create();
+
     Option outputOpt = obuilder.withLongName("output").withRequired(false).withArgument(
-      abuilder.withName("output").withMinimum(1).withMaximum(1).create()).withDescription(
-      "The output file. If not specified, the result is printed on console.").withShortName("o").create();
-    
+        abuilder.withName("output").withMinimum(1).withMaximum(1).create()).withDescription(
+        "The output file. If not specified, the result is printed on console.").withShortName("o").create();
+
     Option fieldOpt = obuilder.withLongName("field").withRequired(true).withArgument(
-      abuilder.withName("field").withMinimum(1).withMaximum(1).create()).withDescription(
-      "The content field in the index").withShortName("f").create();
-    
+        abuilder.withName("field").withMinimum(1).withMaximum(1).create()).withDescription("The content field in the index")
+        .withShortName("f").create();
+
     Option idFieldOpt = obuilder.withLongName("idField").withRequired(false).withArgument(
-      abuilder.withName("idField").withMinimum(1).withMaximum(1).create()).withDescription(
-      "The field for the document ID in the index.  If null, then the Lucene internal doc "
-          + "id is used which is prone to error if the underlying index changes").withShortName("i").create();
-    
+        abuilder.withName("idField").withMinimum(1).withMaximum(1).create()).withDescription(
+        "The field for the document ID in the index.  If null, then the Lucene internal doc "
+            + "id is used which is prone to error if the underlying index changes").withShortName("i").create();
+
     Option seqOpt = obuilder.withLongName("seqFileDir").withRequired(true).withArgument(
-      abuilder.withName("seqFileDir").withMinimum(1).withMaximum(1).create()).withDescription(
-      "The directory containing Sequence Files for the Clusters").withShortName("s").create();
-    
+        abuilder.withName("seqFileDir").withMinimum(1).withMaximum(1).create()).withDescription(
+        "The directory containing Sequence Files for the Clusters").withShortName("s").create();
+
     Option pointsOpt = obuilder.withLongName("pointsDir").withRequired(true).withArgument(
-      abuilder.withName("pointsDir").withMinimum(1).withMaximum(1).create()).withDescription(
-      "The directory containing points sequence files mapping input vectors to their cluster.  ")
-        .withShortName("p").create();
+        abuilder.withName("pointsDir").withMinimum(1).withMaximum(1).create()).withDescription(
+        "The directory containing points sequence files mapping input vectors to their cluster.  ").withShortName("p").create();
     Option minClusterSizeOpt = obuilder.withLongName("minClusterSize").withRequired(false).withArgument(
-      abuilder.withName("minClusterSize").withMinimum(1).withMaximum(1).create()).withDescription(
-      "The minimum number of points required in a cluster to print the labels for").withShortName("m")
-        .create();
+        abuilder.withName("minClusterSize").withMinimum(1).withMaximum(1).create()).withDescription(
+        "The minimum number of points required in a cluster to print the labels for").withShortName("m").create();
     Option maxLabelsOpt = obuilder.withLongName("maxLabels").withRequired(false).withArgument(
-      abuilder.withName("maxLabels").withMinimum(1).withMaximum(1).create()).withDescription(
-      "The maximum number of labels to print per cluster").withShortName("x").create();
-    Option helpOpt = obuilder.withLongName("help").withDescription("Print out help").withShortName("h")
-        .create();
-    
-    Group group = gbuilder.withName("Options").withOption(indexOpt).withOption(idFieldOpt).withOption(
-      outputOpt).withOption(fieldOpt).withOption(seqOpt).withOption(pointsOpt).withOption(helpOpt)
-        .withOption(maxLabelsOpt).withOption(minClusterSizeOpt).create();
-    
+        abuilder.withName("maxLabels").withMinimum(1).withMaximum(1).create()).withDescription(
+        "The maximum number of labels to print per cluster").withShortName("x").create();
+    Option helpOpt = obuilder.withLongName("help").withDescription("Print out help").withShortName("h").create();
+
+    Group group = gbuilder.withName("Options").withOption(indexOpt).withOption(idFieldOpt).withOption(outputOpt).withOption(
+        fieldOpt).withOption(seqOpt).withOption(pointsOpt).withOption(helpOpt).withOption(maxLabelsOpt).withOption(
+        minClusterSizeOpt).create();
+
     try {
       Parser parser = new Parser();
       parser.setGroup(group);
       CommandLine cmdLine = parser.parse(args);
-      
+
       if (cmdLine.hasOption(helpOpt)) {
         CommandLineUtil.printHelp(group);
         return;
       }
-      
+
       String seqFileDir = cmdLine.getValue(seqOpt).toString();
       String pointsDir = cmdLine.getValue(pointsOpt).toString();
       String indexDir = cmdLine.getValue(indexOpt).toString();
       String contentField = cmdLine.getValue(fieldOpt).toString();
-      
+
       String idField = null;
-      
+
       if (cmdLine.hasOption(idFieldOpt)) {
         idField = cmdLine.getValue(idFieldOpt).toString();
       }
@@ -374,18 +385,17 @@ public class ClusterLabels {
       if (cmdLine.hasOption(minClusterSizeOpt)) {
         minSize = Integer.parseInt(cmdLine.getValue(minClusterSizeOpt).toString());
       }
-      ClusterLabels clusterLabel = new ClusterLabels(seqFileDir, pointsDir, indexDir, contentField, minSize,
-          maxLabels);
-      
+      ClusterLabels clusterLabel = new ClusterLabels(seqFileDir, pointsDir, indexDir, contentField, minSize, maxLabels);
+
       if (idField != null) {
         clusterLabel.setIdField(idField);
       }
       if (output != null) {
         clusterLabel.setOutput(output);
       }
-      
+
       clusterLabel.getLabels();
-      
+
     } catch (OptionException e) {
       log.error("Exception", e);
       CommandLineUtil.printHelp(group);
@@ -393,5 +403,5 @@ public class ClusterLabels {
       log.error("Exception", e);
     }
   }
-  
+
 }

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=939509&r1=939508&r2=939509&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 Thu Apr 29 23:22:18 2010
@@ -42,7 +42,9 @@ import org.apache.mahout.clustering.kmea
 import org.apache.mahout.clustering.meanshift.MeanShiftCanopyJob;
 import org.apache.mahout.common.MahoutTestCase;
 import org.apache.mahout.common.RandomUtils;
+import org.apache.mahout.common.distance.CosineDistanceMeasure;
 import org.apache.mahout.common.distance.EuclideanDistanceMeasure;
+import org.apache.mahout.math.NamedVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.math.VectorWritable;
 import org.apache.mahout.utils.clustering.ClusterDumper;
@@ -61,14 +63,11 @@ public class TestClusterDumper extends M
   private static final String[] DOCS = { "The quick red fox jumped over the lazy brown dogs.",
       "The quick brown fox jumped over the lazy red dogs.", "The quick red cat jumped over the lazy brown dogs.",
       "The quick brown cat jumped over the lazy red dogs.", "Mary had a little lamb whose fleece was white as snow.",
-      "Mary had a little goat whose fleece was white as snow.",
-      "Mary had a little lamb whose fleece was black as tar.",
+      "Mary had a little goat whose fleece was white as snow.", "Mary had a little lamb whose fleece was black as tar.",
       "Dick had a little goat whose fleece was white as snow.", "Moby Dick is a story of a whale and a man obsessed.",
       "Moby Bob is a story of a walrus and a man obsessed.", "Moby Dick is a story of a whale and a crazy man.",
-      "The robber wore a black fleece jacket and a baseball cap.",
-      "The robber wore a red fleece jacket and a baseball cap.",
-      "The robber wore a white fleece jacket and a baseball cap.",
-      "The English Springer Spaniel is the best of all dogs." };
+      "The robber wore a black fleece jacket and a baseball cap.", "The robber wore a red fleece jacket and a baseball cap.",
+      "The robber wore a white fleece jacket and a baseball cap.", "The English Springer Spaniel is the best of all dogs." };
 
   @Override
   protected void setUp() throws Exception {
@@ -129,15 +128,17 @@ public class TestClusterDumper extends M
     int i = 0;
     for (Vector vector : iterable) {
       Assert.assertNotNull(vector);
-      System.out.println("Vector[" + i++ + "]=" + ClusterBase.formatVector(vector, null));
-      sampleData.add(new VectorWritable(vector));
+      NamedVector vector2 = new NamedVector(vector, "P(" + i + ")");
+      System.out.println(ClusterBase.formatVector(vector2, null));
+      sampleData.add(new VectorWritable(vector2));
+      i++;
     }
   }
 
   public void testCanopy() throws Exception { // now run the Job
     CanopyClusteringJob.runJob("testdata/points", "output", EuclideanDistanceMeasure.class.getName(), 8, 4);
     // run ClusterDumper
-    ClusterDumper clusterDumper = new ClusterDumper("output/clusters-0", null);
+    ClusterDumper clusterDumper = new ClusterDumper("output/clusters-0", "output/clusteredPoints");
     clusterDumper.printClusters();
   }
 
@@ -145,10 +146,9 @@ public class TestClusterDumper extends M
     // now run the Canopy job to prime kMeans canopies
     CanopyDriver.runJob("testdata/points", "output/clusters-0", EuclideanDistanceMeasure.class.getName(), 8, 4);
     // now run the KMeans job
-    KMeansDriver.runJob("testdata/points", "output/clusters-0", "output", EuclideanDistanceMeasure.class.getName(),
-        0.001, 10, 1);
+    KMeansDriver.runJob("testdata/points", "output/clusters-0", "output", EuclideanDistanceMeasure.class.getName(), 0.001, 10, 1);
     // run ClusterDumper
-    ClusterDumper clusterDumper = new ClusterDumper("output/clusters-2", null);
+    ClusterDumper clusterDumper = new ClusterDumper("output/clusters-2", "output/clusteredPoints");
     clusterDumper.printClusters();
   }
 
@@ -156,26 +156,26 @@ public class TestClusterDumper extends M
     // now run the Canopy job to prime kMeans canopies
     CanopyDriver.runJob("testdata/points", "output/clusters-0", EuclideanDistanceMeasure.class.getName(), 8, 4);
     // now run the KMeans job
-    FuzzyKMeansDriver.runJob("testdata/points", "output/clusters-0", "output", EuclideanDistanceMeasure.class.getName(), 0.001, 10, 1, 1, 2);
+    FuzzyKMeansDriver.runJob("testdata/points", "output/clusters-0", "output", EuclideanDistanceMeasure.class.getName(), 0.001, 10,
+        1, 1, (float) 1.1);
     // run ClusterDumper
-    ClusterDumper clusterDumper = new ClusterDumper("output/clusters-3", null);
+    ClusterDumper clusterDumper = new ClusterDumper("output/clusters-3", "output/clusteredPoints");
     clusterDumper.printClusters();
   }
 
   public void testMeanShift() throws Exception {
-    MeanShiftCanopyJob.runJob("testdata/points", "output", EuclideanDistanceMeasure.class.getName(), 9, 1.0, 0.001, 10);
+    MeanShiftCanopyJob.runJob("testdata/points", "output", CosineDistanceMeasure.class.getName(), 0.5, 0.01, 0.05, 10);
     // run ClusterDumper
-    ClusterDumper clusterDumper = new ClusterDumper("output/clusters-1", null);
+    ClusterDumper clusterDumper = new ClusterDumper("output/clusters-1", "output/clusteredPoints");
     clusterDumper.printClusters();
   }
 
   public void testDirichlet() throws Exception {
-    Vector prototype = sampleData.get(0).get();
-    DirichletDriver.runJob("testdata/points", "output",
-        L1ModelDistribution.class.getName(), prototype.getClass().getName(), prototype
-            .size(), 15, 10, 1.0, 1);
+    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);
     // run ClusterDumper
-    ClusterDumper clusterDumper = new ClusterDumper("output/clusters-10", null);
+    ClusterDumper clusterDumper = new ClusterDumper("output/clusters-10", "output/clusteredPoints");
     clusterDumper.printClusters();
   }
 }