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 22:16:57 UTC

svn commit: r939800 [2/2] - in /lucene/mahout/trunk: core/ 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...

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=939800&r1=939799&r2=939800&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 Fri Apr 30 20:16:56 2010
@@ -45,24 +45,23 @@ import org.apache.mahout.math.Vector;
 import org.apache.mahout.math.VectorWritable;
 
 public class TestCanopyCreation extends MahoutTestCase {
-  
-  private static final double[][] raw = { {1, 1}, {2, 1}, {1, 2}, {2, 2}, {3, 3}, {4, 4}, {5, 4}, {4, 5},
-                                         {5, 5}};
-  
+
+  private static final double[][] raw = { { 1, 1 }, { 2, 1 }, { 1, 2 }, { 2, 2 }, { 3, 3 }, { 4, 4 }, { 5, 4 }, { 4, 5 }, { 5, 5 } };
+
   private List<Canopy> referenceManhattan;
-  
+
   private final DistanceMeasure manhattanDistanceMeasure = new ManhattanDistanceMeasure();
-  
+
   private List<Vector> manhattanCentroids;
-  
+
   private List<Canopy> referenceEuclidean;
-  
+
   private final DistanceMeasure euclideanDistanceMeasure = new EuclideanDistanceMeasure();
-  
+
   private List<Vector> euclideanCentroids;
-  
+
   private FileSystem fs;
-  
+
   private static List<VectorWritable> getPointsWritable() {
     List<VectorWritable> points = new ArrayList<VectorWritable>();
     for (double[] fr : raw) {
@@ -72,7 +71,7 @@ public class TestCanopyCreation extends 
     }
     return points;
   }
-  
+
   private static List<Vector> getPoints() {
     List<Vector> points = new ArrayList<Vector>();
     for (double[] fr : raw) {
@@ -82,17 +81,17 @@ public class TestCanopyCreation extends 
     }
     return points;
   }
-  
+
   /** Verify that the given canopies are equivalent to the referenceManhattan */
   private void verifyManhattanCanopies(List<Canopy> canopies) {
     verifyCanopies(canopies, referenceManhattan);
   }
-  
+
   /** Verify that the given canopies are equivalent to the referenceEuclidean */
   private void verifyEuclideanCanopies(List<Canopy> canopies) {
     verifyCanopies(canopies, referenceEuclidean);
   }
-  
+
   /**
    * Verify that the given canopies are equivalent to the reference. This means the number of canopies is the
    * same, the number of points in each is the same and the centroids are the same.
@@ -106,12 +105,11 @@ public class TestCanopyCreation extends 
       Vector refCentroid = refCanopy.computeCentroid();
       Vector testCentroid = testCanopy.computeCentroid();
       for (int pointIx = 0; pointIx < refCentroid.size(); pointIx++) {
-        assertEquals("canopy centroid " + canopyIx + '[' + pointIx + ']', refCentroid.get(pointIx),
-          testCentroid.get(pointIx));
+        assertEquals("canopy centroid " + canopyIx + '[' + pointIx + ']', refCentroid.get(pointIx), testCentroid.get(pointIx));
       }
     }
   }
-  
+
   /**
    * Print the canopies to the transcript
    * 
@@ -123,7 +121,7 @@ public class TestCanopyCreation extends 
       System.out.println(canopy.toString());
     }
   }
-  
+
   public static void rmr(String path) {
     File f = new File(path);
     if (f.exists()) {
@@ -136,7 +134,7 @@ public class TestCanopyCreation extends 
       f.delete();
     }
   }
-  
+
   @Override
   protected void setUp() throws Exception {
     super.setUp();
@@ -149,7 +147,7 @@ public class TestCanopyCreation extends 
     referenceEuclidean = CanopyClusterer.createCanopies(getPoints(), euclideanDistanceMeasure, 3.1, 2.1);
     euclideanCentroids = CanopyClusterer.calculateCentroids(referenceEuclidean);
   }
-  
+
   /** Story: User can cluster points using a ManhattanDistanceMeasure and a reference implementation */
   public void testReferenceManhattan() throws Exception {
     System.out.println("testReferenceManhattan");
@@ -158,18 +156,17 @@ public class TestCanopyCreation extends 
     assertEquals("number of canopies", 3, referenceManhattan.size());
     for (int canopyIx = 0; canopyIx < referenceManhattan.size(); canopyIx++) {
       Canopy testCanopy = referenceManhattan.get(canopyIx);
-      int[] expectedNumPoints = {4, 4, 3};
-      double[][] expectedCentroids = { {1.5, 1.5}, {4.0, 4.0}, {4.666666666666667, 4.6666666666666667}};
+      int[] expectedNumPoints = { 4, 4, 3 };
+      double[][] expectedCentroids = { { 1.5, 1.5 }, { 4.0, 4.0 }, { 4.666666666666667, 4.6666666666666667 } };
       assertEquals("canopy points " + canopyIx, expectedNumPoints[canopyIx], testCanopy.getNumPoints());
       double[] refCentroid = expectedCentroids[canopyIx];
       Vector testCentroid = testCanopy.computeCentroid();
       for (int pointIx = 0; pointIx < refCentroid.length; pointIx++) {
-        assertEquals("canopy centroid " + canopyIx + '[' + pointIx + ']', refCentroid[pointIx], testCentroid
-            .get(pointIx));
+        assertEquals("canopy centroid " + canopyIx + '[' + pointIx + ']', refCentroid[pointIx], testCentroid.get(pointIx));
       }
     }
   }
-  
+
   /** Story: User can cluster points using a EuclideanDistanceMeasure and a reference implementation */
   public void testReferenceEuclidean() throws Exception {
     System.out.println("testReferenceEuclidean()");
@@ -178,18 +175,17 @@ public class TestCanopyCreation extends 
     assertEquals("number of canopies", 3, referenceManhattan.size());
     for (int canopyIx = 0; canopyIx < referenceManhattan.size(); canopyIx++) {
       Canopy testCanopy = referenceEuclidean.get(canopyIx);
-      int[] expectedNumPoints = {5, 5, 3};
-      double[][] expectedCentroids = { {1.8, 1.8}, {4.2, 4.2}, {4.666666666666667, 4.666666666666667}};
+      int[] expectedNumPoints = { 5, 5, 3 };
+      double[][] expectedCentroids = { { 1.8, 1.8 }, { 4.2, 4.2 }, { 4.666666666666667, 4.666666666666667 } };
       assertEquals("canopy points " + canopyIx, expectedNumPoints[canopyIx], testCanopy.getNumPoints());
       double[] refCentroid = expectedCentroids[canopyIx];
       Vector testCentroid = testCanopy.computeCentroid();
       for (int pointIx = 0; pointIx < refCentroid.length; pointIx++) {
-        assertEquals("canopy centroid " + canopyIx + '[' + pointIx + ']', refCentroid[pointIx], testCentroid
-            .get(pointIx));
+        assertEquals("canopy centroid " + canopyIx + '[' + pointIx + ']', refCentroid[pointIx], testCentroid.get(pointIx));
       }
     }
   }
-  
+
   /** Story: User can cluster points without instantiating them all in memory at once */
   public void testIterativeManhattan() throws Exception {
     List<Vector> points = getPoints();
@@ -198,17 +194,17 @@ public class TestCanopyCreation extends 
     printCanopies(canopies);
     verifyManhattanCanopies(canopies);
   }
-  
+
   /** Story: User can cluster points without instantiating them all in memory at once */
   public void testIterativeEuclidean() throws Exception {
     List<Vector> points = getPoints();
     List<Canopy> canopies = CanopyClusterer.createCanopies(points, new EuclideanDistanceMeasure(), 3.1, 2.1);
-    
+
     System.out.println("testIterativeEuclidean");
     printCanopies(canopies);
     verifyEuclideanCanopies(canopies);
   }
-  
+
   /**
    * Story: User can produce initial canopy centers using a ManhattanDistanceMeasure and a
    * CanopyMapper/Combiner which clusters input points to produce an output set of canopy centroid points.
@@ -216,13 +212,12 @@ public class TestCanopyCreation extends 
   public void testCanopyMapperManhattan() throws Exception {
     CanopyMapper mapper = new CanopyMapper();
     JobConf conf = new JobConf();
-    conf.set(CanopyConfigKeys.DISTANCE_MEASURE_KEY,
-      "org.apache.mahout.common.distance.ManhattanDistanceMeasure");
+    conf.set(CanopyConfigKeys.DISTANCE_MEASURE_KEY, "org.apache.mahout.common.distance.ManhattanDistanceMeasure");
     conf.set(CanopyConfigKeys.T1_KEY, String.valueOf(3.1));
     conf.set(CanopyConfigKeys.T2_KEY, String.valueOf(2.1));
     mapper.configure(conf);
-    
-    DummyOutputCollector<Text,VectorWritable> collector = new DummyOutputCollector<Text,VectorWritable>();
+
+    DummyOutputCollector<Text, VectorWritable> collector = new DummyOutputCollector<Text, VectorWritable>();
     List<VectorWritable> points = getPointsWritable();
     // map the data
     for (VectorWritable point : points) {
@@ -234,11 +229,10 @@ public class TestCanopyCreation extends 
     List<VectorWritable> data = collector.getValue(new Text("centroid"));
     assertEquals("Number of centroids", 3, data.size());
     for (int i = 0; i < data.size(); i++) {
-      assertEquals("Centroid error", manhattanCentroids.get(i).asFormatString(), data.get(i).get()
-          .asFormatString());
+      assertEquals("Centroid error", manhattanCentroids.get(i).asFormatString(), data.get(i).get().asFormatString());
     }
   }
-  
+
   /**
    * Story: User can produce initial canopy centers using a EuclideanDistanceMeasure and a
    * CanopyMapper/Combiner which clusters input points to produce an output set of canopy centroid points.
@@ -246,13 +240,12 @@ public class TestCanopyCreation extends 
   public void testCanopyMapperEuclidean() throws Exception {
     CanopyMapper mapper = new CanopyMapper();
     JobConf conf = new JobConf();
-    conf.set(CanopyConfigKeys.DISTANCE_MEASURE_KEY,
-      "org.apache.mahout.common.distance.EuclideanDistanceMeasure");
+    conf.set(CanopyConfigKeys.DISTANCE_MEASURE_KEY, "org.apache.mahout.common.distance.EuclideanDistanceMeasure");
     conf.set(CanopyConfigKeys.T1_KEY, String.valueOf(3.1));
     conf.set(CanopyConfigKeys.T2_KEY, String.valueOf(2.1));
     mapper.configure(conf);
-    
-    DummyOutputCollector<Text,VectorWritable> collector = new DummyOutputCollector<Text,VectorWritable>();
+
+    DummyOutputCollector<Text, VectorWritable> collector = new DummyOutputCollector<Text, VectorWritable>();
     List<VectorWritable> points = getPointsWritable();
     // map the data
     for (VectorWritable point : points) {
@@ -264,11 +257,10 @@ public class TestCanopyCreation extends 
     List<VectorWritable> data = collector.getValue(new Text("centroid"));
     assertEquals("Number of centroids", 3, data.size());
     for (int i = 0; i < data.size(); i++) {
-      assertEquals("Centroid error", euclideanCentroids.get(i).asFormatString(), data.get(i).get()
-          .asFormatString());
+      assertEquals("Centroid error", euclideanCentroids.get(i).asFormatString(), data.get(i).get().asFormatString());
     }
   }
-  
+
   /**
    * Story: User can produce final canopy centers using a ManhattanDistanceMeasure and a CanopyReducer which
    * clusters input centroid points to produce an output set of final canopy centroid points.
@@ -276,13 +268,12 @@ public class TestCanopyCreation extends 
   public void testCanopyReducerManhattan() throws Exception {
     CanopyReducer reducer = new CanopyReducer();
     JobConf conf = new JobConf();
-    conf.set(CanopyConfigKeys.DISTANCE_MEASURE_KEY,
-      "org.apache.mahout.common.distance.ManhattanDistanceMeasure");
+    conf.set(CanopyConfigKeys.DISTANCE_MEASURE_KEY, "org.apache.mahout.common.distance.ManhattanDistanceMeasure");
     conf.set(CanopyConfigKeys.T1_KEY, String.valueOf(3.1));
     conf.set(CanopyConfigKeys.T2_KEY, String.valueOf(2.1));
     reducer.configure(conf);
-    
-    DummyOutputCollector<Text,Canopy> collector = new DummyOutputCollector<Text,Canopy>();
+
+    DummyOutputCollector<Text, Canopy> collector = new DummyOutputCollector<Text, Canopy>();
     List<VectorWritable> points = getPointsWritable();
     reducer.reduce(new Text("centroid"), points.iterator(), collector, new DummyReporter());
     reducer.close();
@@ -292,12 +283,11 @@ public class TestCanopyCreation extends 
     for (Text key : keys) {
       List<Canopy> data = collector.getValue(key);
       assertEquals(manhattanCentroids.get(i).asFormatString() + " is not equal to "
-                   + data.get(0).computeCentroid().asFormatString(), manhattanCentroids.get(i), data.get(0)
-          .computeCentroid());
+          + data.get(0).computeCentroid().asFormatString(), manhattanCentroids.get(i), data.get(0).computeCentroid());
       i++;
     }
   }
-  
+
   /**
    * Story: User can produce final canopy centers using a EuclideanDistanceMeasure and a CanopyReducer which
    * clusters input centroid points to produce an output set of final canopy centroid points.
@@ -305,13 +295,12 @@ public class TestCanopyCreation extends 
   public void testCanopyReducerEuclidean() throws Exception {
     CanopyReducer reducer = new CanopyReducer();
     JobConf conf = new JobConf();
-    conf.set(CanopyConfigKeys.DISTANCE_MEASURE_KEY,
-      "org.apache.mahout.common.distance.EuclideanDistanceMeasure");
+    conf.set(CanopyConfigKeys.DISTANCE_MEASURE_KEY, "org.apache.mahout.common.distance.EuclideanDistanceMeasure");
     conf.set(CanopyConfigKeys.T1_KEY, String.valueOf(3.1));
     conf.set(CanopyConfigKeys.T2_KEY, String.valueOf(2.1));
     reducer.configure(conf);
-    
-    DummyOutputCollector<Text,Canopy> collector = new DummyOutputCollector<Text,Canopy>();
+
+    DummyOutputCollector<Text, Canopy> collector = new DummyOutputCollector<Text, Canopy>();
     List<VectorWritable> points = getPointsWritable();
     reducer.reduce(new Text("centroid"), points.iterator(), collector, new DummyReporter());
     reducer.close();
@@ -321,12 +310,11 @@ public class TestCanopyCreation extends 
     for (Text key : keys) {
       List<Canopy> data = collector.getValue(key);
       assertEquals(euclideanCentroids.get(i).asFormatString() + " is not equal to "
-                   + data.get(0).computeCentroid().asFormatString(), euclideanCentroids.get(i), data.get(0)
-          .computeCentroid());
+          + data.get(0).computeCentroid().asFormatString(), euclideanCentroids.get(i), data.get(0).computeCentroid());
       i++;
     }
   }
-  
+
   /**
    * Story: User can produce final canopy centers using a Hadoop map/reduce job and a
    * ManhattanDistanceMeasure.
@@ -342,10 +330,10 @@ public class TestCanopyCreation extends 
     ClusteringTestUtils.writePointsToFile(points, "testdata/file1", fs, job);
     ClusteringTestUtils.writePointsToFile(points, "testdata/file2", fs, job);
     // now run the Canopy Driver
-    CanopyDriver.runJob("testdata", "output/canopies", ManhattanDistanceMeasure.class.getName(), 3.1, 2.1);
-    
+    CanopyDriver.runJob("testdata", "output", ManhattanDistanceMeasure.class.getName(), 3.1, 2.1, false);
+
     // verify output from sequence file
-    Path path = new Path("output/canopies/part-00000");
+    Path path = new Path("output/clusters-0/part-00000");
     FileSystem fs = FileSystem.get(path.toUri(), job);
     SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, job);
     Text key = new Text();
@@ -363,7 +351,7 @@ public class TestCanopyCreation extends 
     assertFalse("more to come", reader.next(key, canopy));
     reader.close();
   }
-  
+
   /**
    * Story: User can produce final canopy centers using a Hadoop map/reduce job and a
    * EuclideanDistanceMeasure.
@@ -378,10 +366,10 @@ public class TestCanopyCreation extends 
     ClusteringTestUtils.writePointsToFile(points, "testdata/file1", fs, job);
     ClusteringTestUtils.writePointsToFile(points, "testdata/file2", fs, job);
     // now run the Canopy Driver
-    CanopyDriver.runJob("testdata", "output/canopies", EuclideanDistanceMeasure.class.getName(), 3.1, 2.1);
-    
+    CanopyDriver.runJob("testdata", "output", EuclideanDistanceMeasure.class.getName(), 3.1, 2.1, false);
+
     // verify output from sequence file
-    Path path = new Path("output/canopies/part-00000");
+    Path path = new Path("output/clusters-0/part-00000");
     FileSystem fs = FileSystem.get(path.toUri(), job);
     SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, job);
     Text key = new Text();
@@ -397,19 +385,18 @@ public class TestCanopyCreation extends 
     assertFalse("more to come", reader.next(key, value));
     reader.close();
   }
-  
+
   /** Story: User can cluster a subset of the points using a ClusterMapper and a ManhattanDistanceMeasure. */
   public void testClusterMapperManhattan() throws Exception {
     ClusterMapper mapper = new ClusterMapper();
     JobConf conf = new JobConf();
-    conf.set(CanopyConfigKeys.DISTANCE_MEASURE_KEY,
-      "org.apache.mahout.common.distance.ManhattanDistanceMeasure");
+    conf.set(CanopyConfigKeys.DISTANCE_MEASURE_KEY, "org.apache.mahout.common.distance.ManhattanDistanceMeasure");
     conf.set(CanopyConfigKeys.T1_KEY, String.valueOf(3.1));
     conf.set(CanopyConfigKeys.T2_KEY, String.valueOf(2.1));
     mapper.configure(conf);
-    
+
     List<Canopy> canopies = new ArrayList<Canopy>();
-    DummyOutputCollector<IntWritable,WeightedVectorWritable> collector = new DummyOutputCollector<IntWritable,WeightedVectorWritable>();
+    DummyOutputCollector<IntWritable, WeightedVectorWritable> collector = new DummyOutputCollector<IntWritable, WeightedVectorWritable>();
     int nextCanopyId = 0;
     for (Vector centroid : manhattanCentroids) {
       canopies.add(new Canopy(centroid, nextCanopyId++));
@@ -431,7 +418,7 @@ public class TestCanopyCreation extends 
       }
     }
   }
-  
+
   private static Canopy findCanopy(Integer key, List<Canopy> canopies) {
     for (Canopy c : canopies) {
       if (c.getId() == key) {
@@ -440,19 +427,18 @@ public class TestCanopyCreation extends 
     }
     return null;
   }
-  
+
   /** Story: User can cluster a subset of the points using a ClusterMapper and a EuclideanDistanceMeasure. */
   public void testClusterMapperEuclidean() throws Exception {
     ClusterMapper mapper = new ClusterMapper();
     JobConf conf = new JobConf();
-    conf.set(CanopyConfigKeys.DISTANCE_MEASURE_KEY,
-      "org.apache.mahout.common.distance.EuclideanDistanceMeasure");
+    conf.set(CanopyConfigKeys.DISTANCE_MEASURE_KEY, "org.apache.mahout.common.distance.EuclideanDistanceMeasure");
     conf.set(CanopyConfigKeys.T1_KEY, String.valueOf(3.1));
     conf.set(CanopyConfigKeys.T2_KEY, String.valueOf(2.1));
     mapper.configure(conf);
-    
+
     List<Canopy> canopies = new ArrayList<Canopy>();
-    DummyOutputCollector<IntWritable,WeightedVectorWritable> collector = new DummyOutputCollector<IntWritable,WeightedVectorWritable>();
+    DummyOutputCollector<IntWritable, WeightedVectorWritable> collector = new DummyOutputCollector<IntWritable, WeightedVectorWritable>();
     int nextCanopyId = 0;
     for (Vector centroid : euclideanCentroids) {
       canopies.add(new Canopy(centroid, nextCanopyId++));
@@ -474,7 +460,7 @@ public class TestCanopyCreation extends 
       }
     }
   }
-  
+
   /**
    * Story: User can produce final point clustering using a Hadoop map/reduce job and a
    * ManhattanDistanceMeasure.
@@ -490,7 +476,7 @@ public class TestCanopyCreation extends 
     ClusteringTestUtils.writePointsToFile(points, "testdata/file1", fs, conf);
     ClusteringTestUtils.writePointsToFile(points, "testdata/file2", fs, conf);
     // now run the Job
-    CanopyClusteringJob.runJob("testdata", "output", ManhattanDistanceMeasure.class.getName(), 3.1, 2.1);
+    CanopyDriver.runJob("testdata", "output", ManhattanDistanceMeasure.class.getName(), 3.1, 2.1, true);
     // TODO: change
     Path path = new Path("output/clusteredPoints/part-00000");
     SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, conf);
@@ -507,7 +493,7 @@ public class TestCanopyCreation extends 
     assertEquals("number of points", points.size(), count);
     reader.close();
   }
-  
+
   /**
    * Story: User can produce final point clustering using a Hadoop map/reduce job and a
    * EuclideanDistanceMeasure.
@@ -522,7 +508,7 @@ public class TestCanopyCreation extends 
     ClusteringTestUtils.writePointsToFile(points, "testdata/file1", fs, conf);
     ClusteringTestUtils.writePointsToFile(points, "testdata/file2", fs, conf);
     // now run the Job
-    CanopyClusteringJob.runJob("testdata", "output", EuclideanDistanceMeasure.class.getName(), 3.1, 2.1);
+    CanopyDriver.runJob("testdata", "output", EuclideanDistanceMeasure.class.getName(), 3.1, 2.1, true);
     Path path = new Path("output/clusteredPoints/part-00000");
     SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, conf);
     int count = 0;
@@ -534,7 +520,7 @@ public class TestCanopyCreation extends 
     assertEquals("number of points", points.size(), count);
     reader.close();
   }
-  
+
   /** Story: Clustering algorithm must support arbitrary user defined distance measure */
   public void testUserDefinedDistanceMeasure() throws Exception {
     List<VectorWritable> points = getPointsWritable();
@@ -547,23 +533,23 @@ public class TestCanopyCreation extends 
     ClusteringTestUtils.writePointsToFile(points, "testdata/file2", fs, conf);
     // now run the Canopy Driver. User defined measure happens to be a Manhattan
     // subclass so results are same.
-    CanopyDriver.runJob("testdata", "output/canopies", UserDefinedDistanceMeasure.class.getName(), 3.1, 2.1);
-    
+    CanopyDriver.runJob("testdata", "output", UserDefinedDistanceMeasure.class.getName(), 3.1, 2.1, false);
+
     // verify output from sequence file
     JobConf job = new JobConf(CanopyDriver.class);
-    Path path = new Path("output/canopies/part-00000");
+    Path path = new Path("output/clusters-0/part-00000");
     FileSystem fs = FileSystem.get(path.toUri(), job);
     SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, job);
     Text key = new Text();
     Canopy value = new Canopy();
     assertTrue("more to come", reader.next(key, value));
     assertEquals("1st key", "C-0", key.toString());
-    
+
     assertEquals("1st x value", 1.5, value.getCenter().get(0));
     assertEquals("1st y value", 1.5, value.getCenter().get(1));
     assertTrue("more to come", reader.next(key, value));
     assertEquals("2nd key", "C-1", key.toString());
-    
+
     assertEquals("1st x value", 4.333333333333334, value.getCenter().get(0));
     assertEquals("1st y value", 4.333333333333334, value.getCenter().get(1));
     assertFalse("more to come", reader.next(key, value));

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestMapReduce.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestMapReduce.java?rev=939800&r1=939799&r2=939800&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestMapReduce.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestMapReduce.java Fri Apr 30 20:16:56 2010
@@ -220,7 +220,7 @@ public class TestMapReduce extends Mahou
     ClusteringTestUtils.writePointsToFile(sampleData, "input/data.txt", fs, conf);
     // Now run the driver
     DirichletDriver.runJob("input", "output",
-      "org.apache.mahout.clustering.dirichlet.models.SampledNormalDistribution", 20, 10, 1.0, 1);
+      "org.apache.mahout.clustering.dirichlet.models.SampledNormalDistribution", 20, 5, 1.0, 1);
     // and inspect results
     List<List<DirichletCluster<VectorWritable>>> clusters = new ArrayList<List<DirichletCluster<VectorWritable>>>();
     JobConf conf = new JobConf(KMeansDriver.class);
@@ -262,7 +262,7 @@ public class TestMapReduce extends Mahou
     generate4Datasets();
     // Now run the driver
     DirichletDriver.runJob("input", "output",
-      "org.apache.mahout.clustering.dirichlet.models.SampledNormalDistribution", 20, 15, 1.0, 1);
+      "org.apache.mahout.clustering.dirichlet.models.SampledNormalDistribution", 20, 3, 1.0, 1);
     // and inspect results
     List<List<DirichletCluster<VectorWritable>>> clusters = new ArrayList<List<DirichletCluster<VectorWritable>>>();
     JobConf conf = new JobConf(KMeansDriver.class);
@@ -302,7 +302,7 @@ public class TestMapReduce extends Mahou
     generate4Datasets();
     // Now run the driver
     DirichletDriver.runJob("input", "output",
-      "org.apache.mahout.clustering.dirichlet.models.SampledNormalDistribution", 20, 15, 1.0, 2);
+      "org.apache.mahout.clustering.dirichlet.models.SampledNormalDistribution", 20, 3, 1.0, 2);
     // and inspect results
     List<List<DirichletCluster<VectorWritable>>> clusters = new ArrayList<List<DirichletCluster<VectorWritable>>>();
     JobConf conf = new JobConf(KMeansDriver.class);
@@ -338,7 +338,7 @@ public class TestMapReduce extends Mahou
     ClusteringTestUtils.writePointsToFile(sampleData, "input/data4.txt", fs, conf);
     // Now run the driver
     DirichletDriver.runJob("input", "output",
-      "org.apache.mahout.clustering.dirichlet.models.AsymmetricSampledNormalDistribution", 20, 15, 1.0, 2);
+      "org.apache.mahout.clustering.dirichlet.models.AsymmetricSampledNormalDistribution", 20, 3, 1.0, 2);
     // and inspect results
     List<List<DirichletCluster<VectorWritable>>> clusters = new ArrayList<List<DirichletCluster<VectorWritable>>>();
     JobConf conf = new JobConf(KMeansDriver.class);

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/fuzzykmeans/TestFuzzyKmeansClustering.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/fuzzykmeans/TestFuzzyKmeansClustering.java?rev=939800&r1=939799&r2=939800&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/fuzzykmeans/TestFuzzyKmeansClustering.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/fuzzykmeans/TestFuzzyKmeansClustering.java Fri Apr 30 20:16:56 2010
@@ -225,7 +225,7 @@ public class TestFuzzyKmeansClustering e
       fs.mkdirs(outPath);
       // now run the Job
       FuzzyKMeansDriver.runJob("testdata/points", "testdata/clusters", "output", EuclideanDistanceMeasure.class.getName(), 0.001,
-          2, 1, k + 1, 2);
+          2, 1, k + 1, 2, false, true, 0);
 
       // now compare the expected clusters with actual
       File outDir = new File("output/clusteredPoints");
@@ -268,6 +268,8 @@ public class TestFuzzyKmeansClustering e
       conf.set(FuzzyKMeansConfigKeys.DISTANCE_MEASURE_KEY, "org.apache.mahout.common.distance.EuclideanDistanceMeasure");
       conf.set(FuzzyKMeansConfigKeys.CLUSTER_CONVERGENCE_KEY, "0.001");
       conf.set(FuzzyKMeansConfigKeys.M_KEY, "2");
+      conf.set(FuzzyKMeansConfigKeys.EMIT_MOST_LIKELY_KEY, "true");
+      conf.set(FuzzyKMeansConfigKeys.THRESHOLD_KEY, "0");
       mapper.configure(conf);
 
       DummyOutputCollector<Text, FuzzyKMeansInfo> mapCollector = new DummyOutputCollector<Text, FuzzyKMeansInfo>();
@@ -329,6 +331,8 @@ public class TestFuzzyKmeansClustering e
       conf.set(FuzzyKMeansConfigKeys.DISTANCE_MEASURE_KEY, "org.apache.mahout.common.distance.EuclideanDistanceMeasure");
       conf.set(FuzzyKMeansConfigKeys.CLUSTER_CONVERGENCE_KEY, "0.001");
       conf.set(FuzzyKMeansConfigKeys.M_KEY, "2");
+      conf.set(FuzzyKMeansConfigKeys.EMIT_MOST_LIKELY_KEY, "true");
+      conf.set(FuzzyKMeansConfigKeys.THRESHOLD_KEY, "0");
       mapper.configure(conf);
 
       DummyOutputCollector<Text, FuzzyKMeansInfo> mapCollector = new DummyOutputCollector<Text, FuzzyKMeansInfo>();
@@ -381,6 +385,8 @@ public class TestFuzzyKmeansClustering e
       conf.set(FuzzyKMeansConfigKeys.DISTANCE_MEASURE_KEY, "org.apache.mahout.common.distance.EuclideanDistanceMeasure");
       conf.set(FuzzyKMeansConfigKeys.CLUSTER_CONVERGENCE_KEY, "0.001");
       conf.set(FuzzyKMeansConfigKeys.M_KEY, "2");
+      conf.set(FuzzyKMeansConfigKeys.EMIT_MOST_LIKELY_KEY, "true");
+      conf.set(FuzzyKMeansConfigKeys.THRESHOLD_KEY, "0");
       mapper.configure(conf);
 
       DummyOutputCollector<Text, FuzzyKMeansInfo> mapCollector = new DummyOutputCollector<Text, FuzzyKMeansInfo>();
@@ -463,6 +469,8 @@ public class TestFuzzyKmeansClustering e
       conf.set(FuzzyKMeansConfigKeys.DISTANCE_MEASURE_KEY, "org.apache.mahout.common.distance.EuclideanDistanceMeasure");
       conf.set(FuzzyKMeansConfigKeys.CLUSTER_CONVERGENCE_KEY, "0.001");
       conf.set(FuzzyKMeansConfigKeys.M_KEY, "2");
+      conf.set(FuzzyKMeansConfigKeys.EMIT_MOST_LIKELY_KEY, "true");
+      conf.set(FuzzyKMeansConfigKeys.THRESHOLD_KEY, "0");
       mapper.configure(conf);
 
       DummyOutputCollector<Text, FuzzyKMeansInfo> mapCollector = new DummyOutputCollector<Text, FuzzyKMeansInfo>();

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java?rev=939800&r1=939799&r2=939800&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java Fri Apr 30 20:16:56 2010
@@ -409,10 +409,10 @@ public class TestKmeansClustering extend
     ClusteringTestUtils.writePointsToFile(points, "testdata/points/file2", fs, conf);
 
     // now run the Canopy job
-    CanopyDriver.runJob("testdata/points", "testdata/canopies", ManhattanDistanceMeasure.class.getName(), 3.1, 2.1);
+    CanopyDriver.runJob("testdata/points", "output", ManhattanDistanceMeasure.class.getName(), 3.1, 2.1, false);
 
     // now run the KMeans job
-    KMeansDriver.runJob("testdata/points", "testdata/canopies", "output", EuclideanDistanceMeasure.class.getName(), 0.001, 10, 1);
+    KMeansDriver.runJob("testdata/points", "output/clusters-0", "output", EuclideanDistanceMeasure.class.getName(), 0.001, 10, 1);
 
     // now compare the expected clusters with actual
     File outDir = new File("output/clusteredPoints");

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/Job.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/Job.java?rev=939800&r1=939799&r2=939800&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/Job.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/Job.java Fri Apr 30 20:16:56 2010
@@ -31,7 +31,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.mahout.clustering.canopy.CanopyClusteringJob;
+import org.apache.mahout.clustering.canopy.CanopyDriver;
 import org.apache.mahout.clustering.syntheticcontrol.Constants;
 import org.apache.mahout.common.CommandLineUtil;
 import org.slf4j.Logger;
@@ -138,7 +138,7 @@ public final class Job {
     String directoryContainingConvertedInput = output + Constants.DIRECTORY_CONTAINING_CONVERTED_INPUT;
     InputDriver.runJob(input, directoryContainingConvertedInput,
       "org.apache.mahout.math.RandomAccessSparseVector");
-    CanopyClusteringJob.runJob(directoryContainingConvertedInput, output, measureClassName, t1, t2);
+    CanopyDriver.runJob(directoryContainingConvertedInput, output, measureClassName, t1, t2, true);
   }
   
 }

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/dirichlet/Job.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/dirichlet/Job.java?rev=939800&r1=939799&r2=939800&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/dirichlet/Job.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/dirichlet/Job.java Fri Apr 30 20:16:56 2010
@@ -35,7 +35,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.mahout.clustering.dirichlet.DirichletCluster;
 import org.apache.mahout.clustering.dirichlet.DirichletDriver;
-import org.apache.mahout.clustering.dirichlet.DirichletJob;
 import org.apache.mahout.clustering.dirichlet.DirichletMapper;
 import org.apache.mahout.clustering.dirichlet.models.Model;
 import org.apache.mahout.clustering.kmeans.KMeansDriver;
@@ -153,7 +152,7 @@ public class Job {
                                                    NoSuchMethodException,
                                                    InvocationTargetException {
     // delete the output directory
-    JobConf conf = new JobConf(DirichletJob.class);
+    JobConf conf = new JobConf(DirichletDriver.class);
     Path outPath = new Path(output);
     FileSystem fs = FileSystem.get(outPath.toUri(), conf);
     if (fs.exists(outPath)) {
@@ -163,7 +162,7 @@ public class Job {
     String directoryContainingConvertedInput = output + Constants.DIRECTORY_CONTAINING_CONVERTED_INPUT;
     InputDriver.runJob(input, directoryContainingConvertedInput, vectorClassName);
     DirichletDriver.runJob(directoryContainingConvertedInput, output + "/state", modelFactory,
-      vectorClassName, 60, numModels, maxIterations, alpha_0, numReducers);
+      vectorClassName, 60, numModels, maxIterations, alpha_0, numReducers, true, true, 0);
     printResults(output + "/state", modelFactory, vectorClassName, 60, maxIterations, numModels, alpha_0);
   }
   

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/kmeans/Job.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/kmeans/Job.java?rev=939800&r1=939799&r2=939800&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/kmeans/Job.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/kmeans/Job.java Fri Apr 30 20:16:56 2010
@@ -31,8 +31,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.mahout.clustering.ClusterBase;
-import org.apache.mahout.clustering.canopy.CanopyClusteringJob;
+import org.apache.mahout.clustering.Cluster;
 import org.apache.mahout.clustering.canopy.CanopyDriver;
 import org.apache.mahout.clustering.kmeans.KMeansDriver;
 import org.apache.mahout.clustering.syntheticcontrol.Constants;
@@ -151,10 +150,10 @@ public final class Job {
       "org.apache.mahout.math.RandomAccessSparseVector");
     log.info("Running Canopy to get initial clusters");
     CanopyDriver.runJob(directoryContainingConvertedInput,
-      output + ClusterBase.INITIAL_CLUSTERS_DIR, measureClass, t1, t2);
+      output + Cluster.INITIAL_CLUSTERS_DIR, measureClass, t1, t2, false);
     log.info("Running KMeans");
     KMeansDriver.runJob(directoryContainingConvertedInput,
-      output + ClusterBase.INITIAL_CLUSTERS_DIR, output, measureClass, convergenceDelta,
+      output + Cluster.INITIAL_CLUSTERS_DIR, output, measureClass, convergenceDelta,
       maxIterations, 1);
   }
 }

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=939800&r1=939799&r2=939800&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 Apr 30 20:16:56 2010
@@ -52,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.ClusterBase;
 import org.apache.mahout.clustering.WeightedVectorWritable;
 import org.apache.mahout.common.CommandLineUtil;
 import org.apache.mahout.common.Pair;
@@ -98,12 +99,11 @@ public final class ClusterDumper {
     }
   }
 
-  public void printClusters() throws IOException, InstantiationException, IllegalAccessException {
+  public void printClusters(String[] dictionary) throws IOException, InstantiationException, IllegalAccessException {
     JobClient client = new JobClient();
     JobConf conf = new JobConf(Job.class);
     client.setConf(conf);
 
-    String[] dictionary = null;
     if (this.termDictionary != null) {
       if (dictionaryFormat.equals("text")) {
         dictionary = VectorHelper.loadTermDictionary(new File(this.termDictionary));
@@ -161,7 +161,8 @@ public final class ClusterDumper {
           writer.write("\tWeight:  Point:\n\t");
           for (Iterator<WeightedVectorWritable> iterator = points.iterator(); iterator.hasNext();) {
             WeightedVectorWritable point = iterator.next();
-            writer.append(point.toString());
+            writer.append(Double.toString(point.getWeight())).append(": ");
+            writer.append(ClusterBase.formatVector(point.getVector().get(), dictionary));
             if (iterator.hasNext()) {
               writer.append("\n\t");
             }
@@ -307,7 +308,7 @@ public final class ClusterDumper {
       if (sub >= 0) {
         clusterDumper.setSubString(sub);
       }
-      clusterDumper.printClusters();
+      clusterDumper.printClusters(null);
     } catch (OptionException e) {
       log.error("Exception", e);
       CommandLineUtil.printHelp(group);

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=939800&r1=939799&r2=939800&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 Apr 30 20:16:56 2010
@@ -20,6 +20,7 @@ package org.apache.mahout.clustering;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 
 import junit.framework.Assert;
@@ -33,7 +34,6 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.Version;
-import org.apache.mahout.clustering.canopy.CanopyClusteringJob;
 import org.apache.mahout.clustering.canopy.CanopyDriver;
 import org.apache.mahout.clustering.dirichlet.DirichletDriver;
 import org.apache.mahout.clustering.dirichlet.models.L1ModelDistribution;
@@ -49,6 +49,7 @@ import org.apache.mahout.math.Vector;
 import org.apache.mahout.math.VectorWritable;
 import org.apache.mahout.utils.clustering.ClusterDumper;
 import org.apache.mahout.utils.vectors.TFIDF;
+import org.apache.mahout.utils.vectors.TermEntry;
 import org.apache.mahout.utils.vectors.TermInfo;
 import org.apache.mahout.utils.vectors.Weight;
 import org.apache.mahout.utils.vectors.lucene.CachedTermInfo;
@@ -69,6 +70,8 @@ public class TestClusterDumper extends M
       "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." };
 
+  private String[] termDictionary = null;
+
   @Override
   protected void setUp() throws Exception {
     super.setUp();
@@ -122,60 +125,74 @@ public class TestClusterDumper extends M
     IndexReader reader = IndexReader.open(directory, true);
     Weight weight = new TFIDF();
     TermInfo termInfo = new CachedTermInfo(reader, "content", 1, 100);
+
+    int numTerms = 0;
+    for (Iterator<TermEntry> it = termInfo.getAllEntries(); it.hasNext();) {
+      it.next();
+      numTerms++;
+    }
+    termDictionary = new String[numTerms];
+    int i = 0;
+    for (Iterator<TermEntry> it = termInfo.getAllEntries(); it.hasNext();) {
+      String term = it.next().term;
+      termDictionary[i] = term;
+      System.out.println(i + " " + term);
+      i++;
+    }
     VectorMapper mapper = new TFDFMapper(reader, weight, termInfo);
     LuceneIterable iterable = new LuceneIterable(reader, "id", "content", mapper);
 
-    int i = 0;
+    i = 0;
     for (Vector vector : iterable) {
       Assert.assertNotNull(vector);
-      NamedVector vector2 = new NamedVector(vector, "P(" + i + ")");
-      System.out.println(ClusterBase.formatVector(vector2, null));
-      sampleData.add(new VectorWritable(vector2));
+      NamedVector namedVector = new NamedVector(vector, "P(" + i + ")");
+      System.out.println(ClusterBase.formatVector(namedVector, termDictionary));
+      sampleData.add(new VectorWritable(namedVector));
       i++;
     }
   }
 
   public void testCanopy() throws Exception { // now run the Job
-    CanopyClusteringJob.runJob("testdata/points", "output", EuclideanDistanceMeasure.class.getName(), 8, 4);
+    CanopyDriver.runJob("testdata/points", "output", EuclideanDistanceMeasure.class.getName(), 8, 4, true);
     // run ClusterDumper
     ClusterDumper clusterDumper = new ClusterDumper("output/clusters-0", "output/clusteredPoints");
-    clusterDumper.printClusters();
+    clusterDumper.printClusters(termDictionary);
   }
 
   public void testKmeans() throws Exception {
     // now run the Canopy job to prime kMeans canopies
-    CanopyDriver.runJob("testdata/points", "output/clusters-0", EuclideanDistanceMeasure.class.getName(), 8, 4);
+    CanopyDriver.runJob("testdata/points", "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);
     // run ClusterDumper
     ClusterDumper clusterDumper = new ClusterDumper("output/clusters-2", "output/clusteredPoints");
-    clusterDumper.printClusters();
+    clusterDumper.printClusters(termDictionary);
   }
 
   public void testFuzzyKmeans() throws Exception {
     // now run the Canopy job to prime kMeans canopies
-    CanopyDriver.runJob("testdata/points", "output/clusters-0", EuclideanDistanceMeasure.class.getName(), 8, 4);
+    CanopyDriver.runJob("testdata/points", "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,
-        1, 1, (float) 1.1);
+        1, 1, (float) 1.1, true, true, 0);
     // run ClusterDumper
     ClusterDumper clusterDumper = new ClusterDumper("output/clusters-3", "output/clusteredPoints");
-    clusterDumper.printClusters();
+    clusterDumper.printClusters(termDictionary);
   }
 
   public void testMeanShift() throws Exception {
     MeanShiftCanopyJob.runJob("testdata/points", "output", CosineDistanceMeasure.class.getName(), 0.5, 0.01, 0.05, 10);
     // run ClusterDumper
     ClusterDumper clusterDumper = new ClusterDumper("output/clusters-1", "output/clusteredPoints");
-    clusterDumper.printClusters();
+    clusterDumper.printClusters(termDictionary);
   }
 
   public void testDirichlet() throws Exception {
     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);
+    DirichletDriver.runJob("testdata/points", "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.printClusters();
+    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=939800&r1=939799&r2=939800&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 Apr 30 20:16:56 2010
@@ -33,7 +33,6 @@ import org.apache.mahout.clustering.Clus
 import org.apache.mahout.clustering.ClusterBase;
 import org.apache.mahout.clustering.ClusteringTestUtils;
 import org.apache.mahout.clustering.canopy.Canopy;
-import org.apache.mahout.clustering.canopy.CanopyClusteringJob;
 import org.apache.mahout.clustering.canopy.CanopyDriver;
 import org.apache.mahout.clustering.dirichlet.DirichletDriver;
 import org.apache.mahout.clustering.dirichlet.models.L1ModelDistribution;
@@ -153,7 +152,7 @@ public class TestCDbwEvaluator extends M
   }
 
   public void testCanopy() throws Exception { // now run the Job
-    CanopyClusteringJob.runJob("testdata", "output", EuclideanDistanceMeasure.class.getName(), 3.1, 2.1);
+    CanopyDriver.runJob("testdata", "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);
@@ -162,7 +161,7 @@ public class TestCDbwEvaluator extends M
 
   public void testKmeans() throws Exception {
     // now run the Canopy job to prime kMeans canopies
-    CanopyDriver.runJob("testdata", "output/clusters-0", EuclideanDistanceMeasure.class.getName(), 3.1, 2.1);
+    CanopyDriver.runJob("testdata", "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);
     int numIterations = 2;
@@ -173,10 +172,10 @@ public class TestCDbwEvaluator extends M
 
   public void testFuzzyKmeans() throws Exception {
     // now run the Canopy job to prime kMeans canopies
-    CanopyDriver.runJob("testdata", "output/clusters-0", EuclideanDistanceMeasure.class.getName(), 3.1, 2.1);
+    CanopyDriver.runJob("testdata", "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);
+        2, false, true, 0);
     int numIterations = 2;
     CDbwDriver.runJob("output/clusters-4", "output/clusteredPoints", "output", EuclideanDistanceMeasure.class.getName(),
         numIterations, 1);
@@ -194,7 +193,7 @@ public class TestCDbwEvaluator extends M
   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);
+        .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);