You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by ro...@apache.org on 2010/03/01 06:42:36 UTC

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

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=917396&r1=917395&r2=917396&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 Mon Mar  1 05:42:35 2010
@@ -41,21 +41,21 @@
 import org.apache.mahout.math.AbstractVector;
 import org.apache.mahout.math.DenseVector;
 import org.apache.mahout.math.RandomAccessSparseVector;
+import org.apache.mahout.math.SequentialAccessSparseVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.math.VectorWritable;
 
 public class TestKmeansClustering extends MahoutTestCase {
-
-  public static final double[][] reference = {{1, 1}, {2, 1}, {1, 2},
-      {2, 2}, {3, 3}, {4, 4}, {5, 4}, {4, 5}, {5, 5}};
-
-  private static final int[][] expectedNumPoints = {{9}, {4, 5},
-      {4, 4, 1}, {1, 2, 1, 5}, {1, 1, 1, 2, 4}, {1, 1, 1, 1, 1, 4},
-      {1, 1, 1, 1, 1, 2, 2}, {1, 1, 1, 1, 1, 1, 2, 1},
-      {1, 1, 1, 1, 1, 1, 1, 1, 1}};
-
+  
+  public static final double[][] reference = { {1, 1}, {2, 1}, {1, 2}, {2, 2}, {3, 3}, {4, 4}, {5, 4},
+                                              {4, 5}, {5, 5}};
+  
+  private static final int[][] expectedNumPoints = { {9}, {4, 5}, {4, 4, 1}, {1, 2, 1, 5}, {1, 1, 1, 2, 4},
+                                                    {1, 1, 1, 1, 1, 4}, {1, 1, 1, 1, 1, 2, 2},
+                                                    {1, 1, 1, 1, 1, 1, 2, 1}, {1, 1, 1, 1, 1, 1, 1, 1, 1}};
+  
   private FileSystem fs;
-
+  
   private static void rmr(String path) {
     File f = new File(path);
     if (f.exists()) {
@@ -68,7 +68,7 @@
       f.delete();
     }
   }
-
+  
   @Override
   protected void setUp() throws Exception {
     super.setUp();
@@ -77,81 +77,32 @@
     Configuration conf = new Configuration();
     fs = FileSystem.get(conf);
   }
-
-  /**
-   * This is the reference k-means implementation. Given its inputs it iterates over the points and clusters until their
-   * centers converge or until the maximum number of iterations is exceeded.
-   *
-   * @param points   the input List<Vector> of points
-   * @param clusters the initial List<Cluster> of clusters
-   * @param measure  the DistanceMeasure to use
-   * @param maxIter  the maximum number of iterations
-   * @param convergenceDelta threshold until cluster is considered stable
-   */
-  private static void referenceKmeans(List<VectorWritable> points, List<Cluster> clusters,
-                               DistanceMeasure measure, int maxIter, double convergenceDelta) {
-    boolean converged = false;
-    int iteration = 0;
-    while (!converged && iteration++ < maxIter) {
-      converged = iterateReference(points, clusters, measure, convergenceDelta);
-    }
-  }
-
-  /**
-   * Perform a single iteration over the points and clusters, assigning points to clusters and returning if the
-   * iterations are completed.
-   *
-   * @param points   the List<Vector> having the input points
-   * @param clusters the List<Cluster> clusters
-   * @param measure  a DistanceMeasure to use
-   * @param convergenceDelta threshold until cluster is considered stable
-   */
-  private static boolean iterateReference(List<VectorWritable> points, List<Cluster> clusters,
-                                   DistanceMeasure measure, double convergenceDelta) {
-    // iterate through all points, assigning each to the nearest cluster
-    for (VectorWritable pointWritable : points) {
-      Vector point = pointWritable.get();
-      Cluster closestCluster = null;
-      double closestDistance = Double.MAX_VALUE;
-      for (Cluster cluster : clusters) {
-        double distance = measure.distance(cluster.getCenter(), point);
-        if (closestCluster == null || closestDistance > distance) {
-          closestCluster = cluster;
-          closestDistance = distance;
-        }
-      }
-      closestCluster.addPoint(point);
-    }
-    // test for convergence
-    boolean converged = true;
-    for (Cluster cluster : clusters) {
-      if (!cluster.computeConvergence(measure, convergenceDelta)) {
-        converged = false;
-      }
-    }
-    // update the cluster centers
-    if (!converged) {
-      for (Cluster cluster : clusters) {
-        cluster.recomputeCenter();
-      }
+  
+  public static List<VectorWritable> getPointsWritable(double[][] raw) {
+    List<VectorWritable> points = new ArrayList<VectorWritable>();
+    int i = 0;
+    for (double[] fr : raw) {
+      Vector vec = new RandomAccessSparseVector(String.valueOf(i++), fr.length);
+      vec.assign(fr);
+      points.add(new VectorWritable(vec));
     }
-    return converged;
+    return points;
   }
-
-  public static List<VectorWritable> getPoints(double[][] raw) {
-    List<VectorWritable> points = new ArrayList<VectorWritable>();
+  
+  public static List<Vector> getPoints(double[][] raw) {
+    List<Vector> points = new ArrayList<Vector>();
     for (int i = 0; i < raw.length; i++) {
       double[] fr = raw[i];
-      Vector vec = new RandomAccessSparseVector(String.valueOf(i), fr.length);
+      Vector vec = new SequentialAccessSparseVector(String.valueOf(i), fr.length);
       vec.assign(fr);
-      points.add(new VectorWritable(vec));
+      points.add(vec);
     }
     return points;
   }
-
+  
   /** Story: Test the reference implementation */
   public void testReferenceImplementation() throws Exception {
-    List<VectorWritable> points = getPoints(reference);
+    List<Vector> points = getPoints(reference);
     DistanceMeasure measure = new EuclideanDistanceMeasure();
     // try all possible values of k
     for (int k = 0; k < points.size(); k++) {
@@ -159,44 +110,57 @@
       // pick k initial cluster centers at random
       List<Cluster> clusters = new ArrayList<Cluster>();
       for (int i = 0; i < k + 1; i++) {
-        Vector vec = points.get(i).get();
+        Vector vec = points.get(i);
         clusters.add(new VisibleCluster(vec));
       }
       // iterate clusters until they converge
       int maxIter = 10;
-      referenceKmeans(points, clusters, measure, maxIter, 0.001);
+      List<List<Cluster>> clustersList = KMeansClusterer.clusterPoints(points, clusters, measure, maxIter,
+        0.001);
+      clusters = clustersList.get(clustersList.size() - 1);
       for (int c = 0; c < clusters.size(); c++) {
         Cluster cluster = clusters.get(c);
         System.out.println(cluster.toString());
-        assertEquals("Cluster " + c + " test " + (k + 1), expectedNumPoints[k][c],
-            cluster.getNumPoints());
+        assertEquals("Cluster " + c + " test " + (k + 1), expectedNumPoints[k][c], cluster.getNumPoints());
+      }
+    }
+  }
+  
+  public void testStd() {
+    List<Vector> points = getPoints(reference);
+    Cluster c = new Cluster(points.get(0));
+    for (Vector p : points) {
+      c.addPoint(p);
+      if (c.getNumPoints() > 1) {
+        assertEquals(c.getStd() > 0.0, true);
       }
     }
   }
 
-  private static Map<String, Cluster> loadClusterMap(List<Cluster> clusters) {
-    Map<String, Cluster> clusterMap = new HashMap<String, Cluster>();
-
+  private static Map<String,Cluster> loadClusterMap(List<Cluster> clusters) {
+    Map<String,Cluster> clusterMap = new HashMap<String,Cluster>();
+    
     for (Cluster cluster : clusters) {
       clusterMap.put(cluster.getIdentifier(), cluster);
     }
     return clusterMap;
   }
-
+  
   /** Story: test that the mapper will map input points to the nearest cluster */
   public void testKMeansMapper() throws Exception {
     KMeansMapper mapper = new KMeansMapper();
     JobConf conf = new JobConf();
-    conf.set(KMeansConfigKeys.DISTANCE_MEASURE_KEY, "org.apache.mahout.common.distance.EuclideanDistanceMeasure");
+    conf.set(KMeansConfigKeys.DISTANCE_MEASURE_KEY,
+      "org.apache.mahout.common.distance.EuclideanDistanceMeasure");
     conf.set(KMeansConfigKeys.CLUSTER_CONVERGENCE_KEY, "0.001");
     conf.set(KMeansConfigKeys.CLUSTER_PATH_KEY, "");
     mapper.configure(conf);
-    List<VectorWritable> points = getPoints(reference);
+    List<VectorWritable> points = getPointsWritable(reference);
     for (int k = 0; k < points.size(); k++) {
       // pick k initial cluster centers at random
-      DummyOutputCollector<Text, KMeansInfo> collector = new DummyOutputCollector<Text, KMeansInfo>();
+      DummyOutputCollector<Text,KMeansInfo> collector = new DummyOutputCollector<Text,KMeansInfo>();
       List<Cluster> clusters = new ArrayList<Cluster>();
-
+      
       for (int i = 0; i < k + 1; i++) {
         Cluster cluster = new Cluster(points.get(i).get(), i);
         // add the center so the centroid will be correct upon output
@@ -204,7 +168,7 @@
         clusters.add(cluster);
       }
       mapper.config(clusters);
-
+      
       // map the data
       for (VectorWritable point : points) {
         mapper.map(new Text(), point, collector, null);
@@ -212,38 +176,41 @@
       assertEquals("Number of map results", k + 1, collector.getData().size());
       // now verify that all points are correctly allocated
       EuclideanDistanceMeasure euclideanDistanceMeasure = new EuclideanDistanceMeasure();
-      Map<String, Cluster> clusterMap = loadClusterMap(clusters);
+      Map<String,Cluster> clusterMap = loadClusterMap(clusters);
       for (String key : collector.getKeys()) {
         Cluster cluster = clusterMap.get(key);
         List<KMeansInfo> values = collector.getValue(key);
         for (KMeansInfo value : values) {
-          double distance = euclideanDistanceMeasure.distance(cluster
-              .getCenter(), value.getPointTotal());
+          double distance = euclideanDistanceMeasure.distance(cluster.getCenter(), value.getPointTotal());
           for (Cluster c : clusters) {
-            assertTrue("distance error", distance <= euclideanDistanceMeasure
-                .distance(value.getPointTotal(), c.getCenter()));
+            assertTrue("distance error", distance <= euclideanDistanceMeasure.distance(value.getPointTotal(),
+              c.getCenter()));
           }
         }
       }
     }
   }
-
-  /** Story: test that the combiner will produce partial cluster totals for all of the clusters and points that it sees */
+  
+  /**
+   * Story: test that the combiner will produce partial cluster totals for all of the clusters and points that
+   * it sees
+   */
   public void testKMeansCombiner() throws Exception {
     KMeansMapper mapper = new KMeansMapper();
     JobConf conf = new JobConf();
-    conf.set(KMeansConfigKeys.DISTANCE_MEASURE_KEY, "org.apache.mahout.common.distance.EuclideanDistanceMeasure");
+    conf.set(KMeansConfigKeys.DISTANCE_MEASURE_KEY,
+      "org.apache.mahout.common.distance.EuclideanDistanceMeasure");
     conf.set(KMeansConfigKeys.CLUSTER_CONVERGENCE_KEY, "0.001");
     conf.set(KMeansConfigKeys.CLUSTER_PATH_KEY, "");
     mapper.configure(conf);
-    List<VectorWritable> points = getPoints(reference);
+    List<VectorWritable> points = getPointsWritable(reference);
     for (int k = 0; k < points.size(); k++) {
       // pick k initial cluster centers at random
-      DummyOutputCollector<Text, KMeansInfo> collector = new DummyOutputCollector<Text, KMeansInfo>();
+      DummyOutputCollector<Text,KMeansInfo> collector = new DummyOutputCollector<Text,KMeansInfo>();
       List<Cluster> clusters = new ArrayList<Cluster>();
       for (int i = 0; i < k + 1; i++) {
         Vector vec = points.get(i).get();
-
+        
         Cluster cluster = new Cluster(vec, i);
         // add the center so the centroid will be correct upon output
         cluster.addPoint(cluster.getCenter());
@@ -252,17 +219,15 @@
       mapper.config(clusters);
       // map the data
       for (VectorWritable point : points) {
-        mapper.map(new Text(), point, collector,
-            null);
+        mapper.map(new Text(), point, collector, null);
       }
       // now combine the data
       KMeansCombiner combiner = new KMeansCombiner();
-      DummyOutputCollector<Text, KMeansInfo> collector2 = new DummyOutputCollector<Text, KMeansInfo>();
+      DummyOutputCollector<Text,KMeansInfo> collector2 = new DummyOutputCollector<Text,KMeansInfo>();
       for (String key : collector.getKeys()) {
-        combiner.reduce(new Text(key), collector.getValue(key).iterator(),
-            collector2, null);
+        combiner.reduce(new Text(key), collector.getValue(key).iterator(), collector2, null);
       }
-
+      
       assertEquals("Number of map results", k + 1, collector2.getData().size());
       // now verify that all points are accounted for
       int count = 0;
@@ -270,9 +235,9 @@
       for (String key : collector2.getKeys()) {
         List<KMeansInfo> values = collector2.getValue(key);
         assertEquals("too many values", 1, values.size());
-        //String value = values.get(0).toString();
+        // String value = values.get(0).toString();
         KMeansInfo info = values.get(0);
-
+        
         count += info.getPoints();
         total = total.plus(info.getPointTotal());
       }
@@ -281,21 +246,25 @@
       assertEquals("point total[1]", 27, (int) total.get(1));
     }
   }
-
-  /** Story: test that the reducer will sum the partial cluster totals for all of the clusters and points that it sees */
+  
+  /**
+   * Story: test that the reducer will sum the partial cluster totals for all of the clusters and points that
+   * it sees
+   */
   public void testKMeansReducer() throws Exception {
     KMeansMapper mapper = new KMeansMapper();
     EuclideanDistanceMeasure euclideanDistanceMeasure = new EuclideanDistanceMeasure();
     JobConf conf = new JobConf();
-    conf.set(KMeansConfigKeys.DISTANCE_MEASURE_KEY, "org.apache.mahout.common.distance.EuclideanDistanceMeasure");
+    conf.set(KMeansConfigKeys.DISTANCE_MEASURE_KEY,
+      "org.apache.mahout.common.distance.EuclideanDistanceMeasure");
     conf.set(KMeansConfigKeys.CLUSTER_CONVERGENCE_KEY, "0.001");
     conf.set(KMeansConfigKeys.CLUSTER_PATH_KEY, "");
     mapper.configure(conf);
-    List<VectorWritable> points = getPoints(reference);
+    List<VectorWritable> points = getPointsWritable(reference);
     for (int k = 0; k < points.size(); k++) {
       System.out.println("K = " + k);
       // pick k initial cluster centers at random
-      DummyOutputCollector<Text, KMeansInfo> collector = new DummyOutputCollector<Text, KMeansInfo>();
+      DummyOutputCollector<Text,KMeansInfo> collector = new DummyOutputCollector<Text,KMeansInfo>();
       List<Cluster> clusters = new ArrayList<Cluster>();
       for (int i = 0; i < k + 1; i++) {
         Vector vec = points.get(i).get();
@@ -307,43 +276,43 @@
       mapper.config(clusters);
       // map the data
       for (VectorWritable point : points) {
-        mapper.map(new Text(), point, collector,
-            null);
+        mapper.map(new Text(), point, collector, null);
       }
       // now combine the data
       KMeansCombiner combiner = new KMeansCombiner();
-      DummyOutputCollector<Text, KMeansInfo> collector2 = new DummyOutputCollector<Text, KMeansInfo>();
+      DummyOutputCollector<Text,KMeansInfo> collector2 = new DummyOutputCollector<Text,KMeansInfo>();
       for (String key : collector.getKeys()) {
-        combiner.reduce(new Text(key), collector.getValue(key).iterator(),
-            collector2, null);
+        combiner.reduce(new Text(key), collector.getValue(key).iterator(), collector2, null);
       }
-
+      
       // now reduce the data
       KMeansReducer reducer = new KMeansReducer();
       reducer.configure(conf);
       reducer.config(clusters);
-      DummyOutputCollector<Text, Cluster> collector3 = new DummyOutputCollector<Text, Cluster>();
+      DummyOutputCollector<Text,Cluster> collector3 = new DummyOutputCollector<Text,Cluster>();
       for (String key : collector2.getKeys()) {
-        reducer.reduce(new Text(key), collector2.getValue(key).iterator(),
-            collector3, new DummyReporter());
+        reducer.reduce(new Text(key), collector2.getValue(key).iterator(), collector3, new DummyReporter());
       }
-
+      
       assertEquals("Number of map results", k + 1, collector3.getData().size());
-
+      
       // compute the reference result after one iteration and compare
       List<Cluster> reference = new ArrayList<Cluster>();
       for (int i = 0; i < k + 1; i++) {
         Vector vec = points.get(i).get();
         reference.add(new Cluster(vec, i));
       }
-      boolean converged = iterateReference(points, reference,
-          euclideanDistanceMeasure, 0.001);
+      List<Vector> pointsVectors = new ArrayList<Vector>();
+      for(VectorWritable point : points)
+        pointsVectors.add(point.get());
+      boolean converged = KMeansClusterer.runKMeansIteration(pointsVectors, reference,
+        euclideanDistanceMeasure, 0.001);
       if (k == 8) {
         assertTrue("not converged? " + k, converged);
       } else {
         assertFalse("converged? " + k, converged);
       }
-
+      
       // now verify that all clusters have correct centers
       converged = true;
       for (int i = 0; i < reference.size(); i++) {
@@ -352,13 +321,16 @@
         List<Cluster> values = collector3.getValue(key);
         Cluster cluster = values.get(0);
         converged = converged && cluster.isConverged();
-        //Since we aren't roundtripping through Writable, we need to compare the reference center with the cluster centroid
+        // Since we aren't roundtripping through Writable, we need to compare the reference center with the
+        // cluster centroid
         cluster.recomputeCenter();
         assertTrue(i + " reference center: " + ref.getCenter().asFormatString() + " and cluster center:  "
-            + cluster.getCenter().asFormatString() + " are not equal", AbstractVector.equivalent(ref.getCenter(), cluster.getCenter()));
-
-        /*assertEquals(k + " center[" + key + "][1]", ref.getCenter().get(1),
-            cluster.getCenter().get(1));*/
+                   + cluster.getCenter().asFormatString() + " are not equal", AbstractVector.equivalent(ref
+            .getCenter(), cluster.getCenter()));
+        
+        /*
+         * assertEquals(k + " center[" + key + "][1]", ref.getCenter().get(1), cluster.getCenter().get(1));
+         */
       }
       if (k == 8) {
         assertTrue("not converged? " + k, converged);
@@ -367,10 +339,10 @@
       }
     }
   }
-
+  
   /** Story: User wishes to run kmeans job on reference data */
   public void testKMeansMRJob() throws Exception {
-    List<VectorWritable> points = getPoints(reference);
+    List<VectorWritable> points = getPointsWritable(reference);
     File testData = new File("testdata");
     if (!testData.exists()) {
       testData.mkdir();
@@ -379,7 +351,7 @@
     if (!testData.exists()) {
       testData.mkdir();
     }
-
+    
     Configuration conf = new Configuration();
     ClusteringTestUtils.writePointsToFile(points, "testdata/points/file1", fs, conf);
     ClusteringTestUtils.writePointsToFile(points, "testdata/points/file2", fs, conf);
@@ -389,12 +361,11 @@
       JobConf job = new JobConf(KMeansDriver.class);
       Path path = new Path("testdata/clusters/part-00000");
       FileSystem fs = FileSystem.get(path.toUri(), job);
-      SequenceFile.Writer writer = new SequenceFile.Writer(fs, job, path,
-          Text.class, Cluster.class);
-
+      SequenceFile.Writer writer = new SequenceFile.Writer(fs, job, path, Text.class, Cluster.class);
+      
       for (int i = 0; i < k + 1; i++) {
         Vector vec = points.get(i).get();
-
+        
         Cluster cluster = new Cluster(vec, i);
         // add the center so the centroid will be correct upon output
         cluster.addPoint(cluster.getCenter());
@@ -403,51 +374,45 @@
       writer.close();
       // now run the Job
       HadoopUtil.overwriteOutput("output");
-      KMeansDriver.runJob("testdata/points",
-          "testdata/clusters",
-          "output",
-          EuclideanDistanceMeasure.class.getName(),
-          0.001,
-          10,
-          k + 1);
+      KMeansDriver.runJob("testdata/points", "testdata/clusters", "output", EuclideanDistanceMeasure.class
+          .getName(), 0.001, 10, k + 1);
       // now compare the expected clusters with actual
       File outDir = new File("output/points");
       assertTrue("output dir exists?", outDir.exists());
       // assertEquals("output dir files?", 4, outFiles.length);
       SequenceFile.Reader reader = new SequenceFile.Reader(fs, new Path("output/points/part-00000"), conf);
       int[] expect = expectedNumPoints[k];
-      DummyOutputCollector<Text, Text> collector = new DummyOutputCollector<Text, Text>();
-      //The key is the name of the vector, or the vector itself
+      DummyOutputCollector<Text,Text> collector = new DummyOutputCollector<Text,Text>();
+      // The key is the name of the vector, or the vector itself
       Text key = new Text();
-      //The value is the cluster id
+      // The value is the cluster id
       Text value = new Text();
       while (reader.next(key, value)) {
-        /*String line = reader.readLine();
-        String[] lineParts = line.split("\t");
-        assertEquals("line parts", 2, lineParts.length);*/
+        /*
+         * String line = reader.readLine(); String[] lineParts = line.split("\t"); assertEquals("line parts",
+         * 2, lineParts.length);
+         */
         // String cl = line.substring(0, line.indexOf(':'));
-        //collector.collect(new Text(lineParts[1]), new Text(lineParts[0]));
+        // collector.collect(new Text(lineParts[1]), new Text(lineParts[0]));
         collector.collect(value, key);
         key = new Text();
         value = new Text();
-
+        
       }
       reader.close();
       if (k == 2)
       // cluster 3 is empty so won't appear in output
       {
-        assertEquals("clusters[" + k + ']', expect.length - 1, collector
-            .getKeys().size());
+        assertEquals("clusters[" + k + ']', expect.length - 1, collector.getKeys().size());
       } else {
-        assertEquals("clusters[" + k + ']', expect.length, collector.getKeys()
-            .size());
+        assertEquals("clusters[" + k + ']', expect.length, collector.getKeys().size());
       }
     }
   }
-
+  
   /** Story: User wants to use canopy clustering to input the initial clusters for kmeans job. */
   public void testKMeansWithCanopyClusterInput() throws Exception {
-    List<VectorWritable> points = getPoints(reference);
+    List<VectorWritable> points = getPointsWritable(reference);
     File testData = new File("testdata");
     if (!testData.exists()) {
       testData.mkdir();
@@ -459,36 +424,32 @@
     Configuration conf = new Configuration();
     ClusteringTestUtils.writePointsToFile(points, "testdata/points/file1", fs, conf);
     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", "testdata/canopies", ManhattanDistanceMeasure.class.getName(),
+      3.1, 2.1);
+    
     // now run the KMeans job
-    KMeansDriver.runJob("testdata/points", 
-        "testdata/canopies",
-        "output",
-        EuclideanDistanceMeasure.class.getName(),
-        0.001,
-        10,
-        1);
-
+    KMeansDriver.runJob("testdata/points", "testdata/canopies", "output", EuclideanDistanceMeasure.class
+        .getName(), 0.001, 10, 1);
+    
     // now compare the expected clusters with actual
     File outDir = new File("output/points");
     assertTrue("output dir exists?", outDir.exists());
     String[] outFiles = outDir.list();
     assertEquals("output dir files?", 4, outFiles.length);
-    DummyOutputCollector<Text, Text> collector = new DummyOutputCollector<Text, Text>();
+    DummyOutputCollector<Text,Text> collector = new DummyOutputCollector<Text,Text>();
     SequenceFile.Reader reader = new SequenceFile.Reader(fs, new Path("output/points/part-00000"), conf);
-   
-    //The key is the name of the vector, or the vector itself
+    
+    // The key is the name of the vector, or the vector itself
     Text key = new Text();
-    //The value is the cluster id
+    // The value is the cluster id
     Text value = new Text();
     while (reader.next(key, value)) {
       collector.collect(value, key);
       key = new Text();
       value = new Text();
-
+      
     }
     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=917396&r1=917395&r2=917396&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 Mon Mar  1 05:42:35 2010
@@ -17,53 +17,54 @@
 
 package org.apache.mahout.clustering.meanshift;
 
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.mahout.clustering.ClusteringTestUtils;
+import org.apache.mahout.common.DummyOutputCollector;
 import org.apache.mahout.common.DummyReporter;
 import org.apache.mahout.common.MahoutTestCase;
-import org.apache.mahout.math.DenseVector;
-import org.apache.mahout.math.Vector;
 import org.apache.mahout.common.distance.DistanceMeasure;
-import org.apache.mahout.common.DummyOutputCollector;
 import org.apache.mahout.common.distance.EuclideanDistanceMeasure;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import org.apache.mahout.math.DenseVector;
+import org.apache.mahout.math.Vector;
+import org.apache.mahout.math.VectorWritable;
 
 public class TestMeanShift extends MahoutTestCase {
-
+  
   private Vector[] raw = null;
-
+  
   private FileSystem fs;
-
+  
   private Configuration conf;
-
+  
   // DistanceMeasure manhattanDistanceMeasure = new ManhattanDistanceMeasure();
-
+  
   private final DistanceMeasure euclideanDistanceMeasure = new EuclideanDistanceMeasure();
-
+  
   /**
    * Print the canopies to the transcript
-   *
-   * @param canopies a List<Canopy>
+   * 
+   * @param canopies
+   *          a List<Canopy>
    */
   private static void printCanopies(List<MeanShiftCanopy> canopies) {
     for (MeanShiftCanopy canopy : canopies) {
       System.out.println(canopy.toString());
     }
   }
-
+  
   /** Print a graphical representation of the clustered image points as a 10x10 character mask */
-  private static void printImage(List<MeanShiftCanopy> canopies) {
+  private void printImage(List<MeanShiftCanopy> canopies) {
     char[][] out = new char[10][10];
     for (int i = 0; i < out.length; i++) {
       for (int j = 0; j < out[0].length; j++) {
@@ -72,7 +73,8 @@
     }
     for (MeanShiftCanopy canopy : canopies) {
       int ch = 'A' + canopy.getCanopyId() - 100;
-      for (Vector pt : canopy.getBoundPoints()) {
+      for (int pid : canopy.getBoundPoints().elements()) {
+        Vector pt = raw[pid];
         out[(int) pt.getQuick(0)][(int) pt.getQuick(1)] = (char) ch;
       }
     }
@@ -80,7 +82,7 @@
       System.out.println(anOut);
     }
   }
-
+  
   private static void rmr(String path) throws Exception {
     File f = new File(path);
     if (f.exists()) {
@@ -93,21 +95,7 @@
       f.delete();
     }
   }
-
-  private void writePointsToFile(Vector[] points, String fileName)
-      throws IOException {
-    Path path = new Path(fileName);
-    SequenceFile.Writer writer = new SequenceFile.Writer(fs, conf, path,
-        LongWritable.class, MeanShiftCanopy.class);
-    long recNum = 0;
-    int nextCanopyId = 0;
-    for (Vector point : points) {
-      MeanShiftCanopy canopy = new MeanShiftCanopy(point, nextCanopyId++);
-      writer.append(new LongWritable(recNum++), canopy);
-    }
-    writer.close();
-  }
-
+  
   private List<MeanShiftCanopy> getInitialCanopies() {
     int nextCanopyId = 0;
     List<MeanShiftCanopy> canopies = new ArrayList<MeanShiftCanopy>();
@@ -116,7 +104,7 @@
     }
     return canopies;
   }
-
+  
   @Override
   protected void setUp() throws Exception {
     super.setUp();
@@ -140,13 +128,14 @@
       }
     }
   }
-
+  
   /**
-   * Story: User can exercise the reference implementation to verify that the test datapoints are clustered in a
-   * reasonable manner.
+   * Story: User can exercise the reference implementation to verify that the test datapoints are clustered in
+   * a reasonable manner.
    */
   public void testReferenceImplementation() {
-    MeanShiftCanopyClusterer clusterer = new MeanShiftCanopyClusterer(new EuclideanDistanceMeasure(), 4.0, 1.0, 0.5);
+    MeanShiftCanopyClusterer clusterer = new MeanShiftCanopyClusterer(new EuclideanDistanceMeasure(), 4.0,
+        1.0, 0.5);
     List<MeanShiftCanopy> canopies = new ArrayList<MeanShiftCanopy>();
     // add all points to the canopies
     int nextCanopyId = 0;
@@ -168,14 +157,14 @@
       System.out.println(iter++);
     }
   }
-
+  
   /**
-   * Story: User can produce initial canopy centers using a EuclideanDistanceMeasure and a CanopyMapper/Combiner which
-   * clusters input points to produce an output set of canopies.
+   * Story: User can produce initial canopy centers using a EuclideanDistanceMeasure and a
+   * CanopyMapper/Combiner which clusters input points to produce an output set of canopies.
    */
   public void testCanopyMapperEuclidean() throws Exception {
     MeanShiftCanopyMapper mapper = new MeanShiftCanopyMapper();
-    DummyOutputCollector<Text, MeanShiftCanopy> collector = new DummyOutputCollector<Text, MeanShiftCanopy>();
+    DummyOutputCollector<Text,MeanShiftCanopy> collector = new DummyOutputCollector<Text,MeanShiftCanopy>();
     MeanShiftCanopyClusterer clusterer = new MeanShiftCanopyClusterer(euclideanDistanceMeasure, 4, 1, 0.5);
     // get the initial canopies
     List<MeanShiftCanopy> canopies = getInitialCanopies();
@@ -185,9 +174,10 @@
     for (Vector aRaw : raw) {
       clusterer.mergeCanopy(new MeanShiftCanopy(aRaw, nextCanopyId++), refCanopies);
     }
-
+    
     JobConf conf = new JobConf();
-    conf.set(MeanShiftCanopyConfigKeys.DISTANCE_MEASURE_KEY, "org.apache.mahout.common.distance.EuclideanDistanceMeasure");
+    conf.set(MeanShiftCanopyConfigKeys.DISTANCE_MEASURE_KEY,
+      "org.apache.mahout.common.distance.EuclideanDistanceMeasure");
     conf.set(MeanShiftCanopyConfigKeys.T1_KEY, "4");
     conf.set(MeanShiftCanopyConfigKeys.T2_KEY, "1");
     conf.set(MeanShiftCanopyConfigKeys.CLUSTER_CONVERGENCE_KEY, "0.5");
@@ -198,46 +188,43 @@
       mapper.map(new Text(), canopy, collector, null);
     }
     mapper.close();
-
+    
     // now verify the output
     assertEquals("Number of map results", 1, collector.getData().size());
     List<MeanShiftCanopy> data = collector.getValue("canopy");
     assertEquals("Number of canopies", refCanopies.size(), data.size());
-
+    
     // add all points to the reference canopies
-    Map<String, MeanShiftCanopy> refCanopyMap = new HashMap<String, MeanShiftCanopy>();
+    Map<String,MeanShiftCanopy> refCanopyMap = new HashMap<String,MeanShiftCanopy>();
     for (MeanShiftCanopy canopy : refCanopies) {
       clusterer.shiftToMean(canopy);
       refCanopyMap.put(canopy.getIdentifier(), canopy);
     }
     // build a map of the combiner output
-    Map<String, MeanShiftCanopy> canopyMap = new HashMap<String, MeanShiftCanopy>();
+    Map<String,MeanShiftCanopy> canopyMap = new HashMap<String,MeanShiftCanopy>();
     for (MeanShiftCanopy d : data) {
       canopyMap.put(d.getIdentifier(), d);
     }
     // compare the maps
-    for (Map.Entry<String, MeanShiftCanopy> stringMeanShiftCanopyEntry : refCanopyMap
-        .entrySet()) {
+    for (Map.Entry<String,MeanShiftCanopy> stringMeanShiftCanopyEntry : refCanopyMap.entrySet()) {
       MeanShiftCanopy ref = stringMeanShiftCanopyEntry.getValue();
-
-      MeanShiftCanopy canopy = canopyMap.get((ref.isConverged() ? "V" : "C")
-          + ref.getCanopyId());
+      
+      MeanShiftCanopy canopy = canopyMap.get((ref.isConverged() ? "V" : "C") + ref.getCanopyId());
       assertEquals("ids", ref.getCanopyId(), canopy.getCanopyId());
-      assertEquals("centers(" + ref.getIdentifier() + ')', ref.getCenter()
-          .asFormatString(), canopy.getCenter().asFormatString());
-      assertEquals("bound points", ref.getBoundPoints().size(), canopy
-          .getBoundPoints().size());
+      assertEquals("centers(" + ref.getIdentifier() + ')', ref.getCenter().asFormatString(), canopy
+          .getCenter().asFormatString());
+      assertEquals("bound points", ref.getBoundPoints().size(), canopy.getBoundPoints().size());
     }
   }
-
+  
   /**
-   * 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.
+   * 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.
    */
   public void testCanopyReducerEuclidean() throws Exception {
     MeanShiftCanopyMapper mapper = new MeanShiftCanopyMapper();
     MeanShiftCanopyReducer reducer = new MeanShiftCanopyReducer();
-    DummyOutputCollector<Text, MeanShiftCanopy> mapCollector = new DummyOutputCollector<Text, MeanShiftCanopy>();
+    DummyOutputCollector<Text,MeanShiftCanopy> mapCollector = new DummyOutputCollector<Text,MeanShiftCanopy>();
     MeanShiftCanopyClusterer clusterer = new MeanShiftCanopyClusterer(euclideanDistanceMeasure, 4, 1, 0.5);
     // get the initial canopies
     List<MeanShiftCanopy> canopies = getInitialCanopies();
@@ -260,7 +247,8 @@
     }
     
     JobConf conf = new JobConf();
-    conf.set(MeanShiftCanopyConfigKeys.DISTANCE_MEASURE_KEY, "org.apache.mahout.common.distance.EuclideanDistanceMeasure");
+    conf.set(MeanShiftCanopyConfigKeys.DISTANCE_MEASURE_KEY,
+      "org.apache.mahout.common.distance.EuclideanDistanceMeasure");
     conf.set(MeanShiftCanopyConfigKeys.T1_KEY, "4");
     conf.set(MeanShiftCanopyConfigKeys.T2_KEY, "1");
     conf.set(MeanShiftCanopyConfigKeys.CLUSTER_CONVERGENCE_KEY, "0.5");
@@ -271,31 +259,29 @@
       mapper.map(new Text(), canopy, mapCollector, null);
     }
     mapper.close();
-
+    
     assertEquals("Number of map results", 1, mapCollector.getData().size());
     // now reduce the mapper output
-    DummyOutputCollector<Text, MeanShiftCanopy> reduceCollector = new DummyOutputCollector<Text, MeanShiftCanopy>();
+    DummyOutputCollector<Text,MeanShiftCanopy> reduceCollector = new DummyOutputCollector<Text,MeanShiftCanopy>();
     reducer.configure(conf);
-    reducer.reduce(new Text("canopy"), mapCollector.getValue("canopy")
-        .iterator(), reduceCollector, new DummyReporter());
+    reducer.reduce(new Text("canopy"), mapCollector.getValue("canopy").iterator(), reduceCollector,
+      new DummyReporter());
     reducer.close();
-
+    
     // now verify the output
-    assertEquals("Number of canopies", reducerReference.size(), reduceCollector
-        .getKeys().size());
-
+    assertEquals("Number of canopies", reducerReference.size(), reduceCollector.getKeys().size());
+    
     // add all points to the reference canopy maps
-    Map<String, MeanShiftCanopy> reducerReferenceMap = new HashMap<String, MeanShiftCanopy>();
+    Map<String,MeanShiftCanopy> reducerReferenceMap = new HashMap<String,MeanShiftCanopy>();
     for (MeanShiftCanopy canopy : reducerReference) {
       reducerReferenceMap.put(canopy.getIdentifier(), canopy);
     }
     // compare the maps
-    for (Map.Entry<String, MeanShiftCanopy> mapEntry : reducerReferenceMap
-        .entrySet()) {
+    for (Map.Entry<String,MeanShiftCanopy> mapEntry : reducerReferenceMap.entrySet()) {
       MeanShiftCanopy refCanopy = mapEntry.getValue();
-
-      List<MeanShiftCanopy> values = reduceCollector.getValue((refCanopy
-          .isConverged() ? "V" : "C") + refCanopy.getCanopyId());
+      
+      List<MeanShiftCanopy> values = reduceCollector.getValue((refCanopy.isConverged() ? "V" : "C")
+                                                              + refCanopy.getCanopyId());
       assertEquals("values", 1, values.size());
       MeanShiftCanopy reducerCanopy = values.get(0);
       assertEquals("ids", refCanopy.getCanopyId(), reducerCanopy.getCanopyId());
@@ -304,27 +290,30 @@
       assertEquals("numPoints", refNumPoints, reducerNumPoints);
       String refCenter = refCanopy.getCenter().asFormatString();
       String reducerCenter = reducerCanopy.getCenter().asFormatString();
-      assertEquals("centers(" + mapEntry.getKey() + ')', refCenter,
-          reducerCenter);
-      assertEquals("bound points", refCanopy.getBoundPoints().size(),
-          reducerCanopy.getBoundPoints().size());
+      assertEquals("centers(" + mapEntry.getKey() + ')', refCenter, reducerCenter);
+      assertEquals("bound points", refCanopy.getBoundPoints().size(), reducerCanopy.getBoundPoints().size());
     }
   }
-
-  /** Story: User can produce final point clustering using a Hadoop map/reduce job and a EuclideanDistanceMeasure. */
+  
+  /**
+   * Story: User can produce final point clustering using a Hadoop map/reduce job and a
+   * EuclideanDistanceMeasure.
+   */
   public void testCanopyEuclideanMRJob() throws Exception {
     File testData = new File("testdata");
     if (!testData.exists()) {
       testData.mkdir();
     }
-    writePointsToFile(raw, "testdata/file1");
-    writePointsToFile(raw, "testdata/file2");
+    FileSystem fs = FileSystem.get(new Path("testdata").toUri(), conf);
+    List<VectorWritable> points = new ArrayList<VectorWritable>();
+    for (Vector v : raw)
+      points.add(new VectorWritable(v));
+    ClusteringTestUtils.writePointsToFile(points, "testdata/file1", fs, conf);
+    ClusteringTestUtils.writePointsToFile(points, "testdata/file2", fs, conf);
     // now run the Job
-    MeanShiftCanopyJob.runJob("testdata", "output",
-        EuclideanDistanceMeasure.class.getName(), 4, 1, 0.5, 10);
+    MeanShiftCanopyJob.runJob("testdata", "output", EuclideanDistanceMeasure.class.getName(), 4, 1, 0.5, 10);
     JobConf conf = new JobConf(MeanShiftCanopyDriver.class);
     Path outPart = new Path("output/canopies-2/part-00000");
-    FileSystem fs = FileSystem.get(outPart.toUri(), conf);
     SequenceFile.Reader reader = new SequenceFile.Reader(fs, outPart, conf);
     Text key = new Text();
     MeanShiftCanopy value = new MeanShiftCanopy();

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/canopy/DisplayCanopy.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/canopy/DisplayCanopy.java?rev=917396&r1=917395&r2=917396&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/canopy/DisplayCanopy.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/canopy/DisplayCanopy.java Mon Mar  1 05:42:35 2010
@@ -17,16 +17,15 @@
 
 package org.apache.mahout.clustering.canopy;
 
+import java.awt.BasicStroke;
 import java.awt.Graphics;
 import java.awt.Graphics2D;
 import java.util.ArrayList;
-import java.util.Iterator;
 import java.util.List;
 
 import org.apache.mahout.clustering.dirichlet.DisplayDirichlet;
 import org.apache.mahout.clustering.dirichlet.models.NormalModelDistribution;
 import org.apache.mahout.common.RandomUtils;
-import org.apache.mahout.common.distance.DistanceMeasure;
 import org.apache.mahout.common.distance.ManhattanDistanceMeasure;
 import org.apache.mahout.math.DenseVector;
 import org.apache.mahout.math.Vector;
@@ -42,7 +41,7 @@
   
   private static final double t1 = 3.0;
   
-  private static final double t2 = 1.5;
+  private static final double t2 = 1.6;
   
   @Override
   public void paint(Graphics g) {
@@ -51,71 +50,27 @@
     Vector dv = new DenseVector(2);
     for (Canopy canopy : canopies) {
       if (canopy.getNumPoints() > DisplayDirichlet.sampleData.size() * 0.05) {
+        g2.setStroke(new BasicStroke(2));
+        g2.setColor(colors[1]);
         dv.assign(t1);
+        Vector center = canopy.computeCentroid();
+        DisplayDirichlet.plotEllipse(g2, center, dv);
+        g2.setStroke(new BasicStroke(3));
         g2.setColor(colors[0]);
-        DisplayDirichlet.plotEllipse(g2, canopy.getCenter(), dv);
         dv.assign(t2);
-        DisplayDirichlet.plotEllipse(g2, canopy.getCenter(), dv);
+        DisplayDirichlet.plotEllipse(g2, center, dv);
       }
     }
   }
   
-  /**
-   * Iterate through the points, adding new canopies. Return the canopies.
-   * 
-   * @param measure
-   *          a DistanceMeasure to use
-   * @param points
-   *          a list<Vector> defining the points to be clustered
-   * @param t1
-   *          the T1 distance threshold
-   * @param t2
-   *          the T2 distance threshold
-   * @return the List<Canopy> created
-   */
-  static List<Canopy> populateCanopies(DistanceMeasure measure,
-                                       List<VectorWritable> points,
-                                       double t1,
-                                       double t2) {
-    List<Canopy> canopies = new ArrayList<Canopy>();
-    /**
-     * Reference Implementation: Given a distance metric, one can create canopies as follows: Start with a
-     * list of the data points in any order, and with two distance thresholds, T1 and T2, where T1 > T2.
-     * (These thresholds can be set by the user, or selected by cross-validation.) Pick a point on the list
-     * and measure its distance to all other points. Put all points that are within distance threshold T1 into
-     * a canopy. Remove from the list all points that are within distance threshold T2. Repeat until the list
-     * is empty.
-     */
-    int nextCanopyId = 0;
-    while (!points.isEmpty()) {
-      Iterator<VectorWritable> ptIter = points.iterator();
-      Vector p1 = ptIter.next().get();
-      ptIter.remove();
-      Canopy canopy = new Canopy(p1, nextCanopyId++);
-      canopies.add(canopy);
-      while (ptIter.hasNext()) {
-        Vector p2 = ptIter.next().get();
-        double dist = measure.distance(p1, p2);
-        // Put all points that are within distance threshold T1 into the canopy
-        if (dist < t1) {
-          canopy.addPoint(p2);
-        }
-        // Remove from the list all points that are within distance threshold T2
-        if (dist < t2) {
-          ptIter.remove();
-        }
-      }
-    }
-    return canopies;
-  }
-  
   public static void main(String[] args) {
     RandomUtils.useTestSeed();
     DisplayDirichlet.generateSamples();
-    List<VectorWritable> points = new ArrayList<VectorWritable>();
-    points.addAll(sampleData);
-    canopies = populateCanopies(new ManhattanDistanceMeasure(), points,
-      t1, t2);
+    List<Vector> points = new ArrayList<Vector>();
+    for (VectorWritable sample : sampleData)
+      points.add(sample.get());
+    canopies = CanopyClusterer.createCanopies(points, new ManhattanDistanceMeasure(), t1, t2);
+    CanopyClusterer.updateCentroids(canopies);
     new DisplayCanopy();
   }
   

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/Display2dASNDirichlet.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/Display2dASNDirichlet.java?rev=917396&r1=917395&r2=917396&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/Display2dASNDirichlet.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/Display2dASNDirichlet.java Mon Mar  1 05:42:35 2010
@@ -33,8 +33,7 @@
   Display2dASNDirichlet() {
     initialize();
     this.setTitle("Dirichlet Process Clusters - 2-d Asymmetric Sampled Normal Distribution (>"
-      + (int) (significance * 100)
-      + "% of population)");
+                  + (int) (significance * 100) + "% of population)");
   }
   
   @Override
@@ -59,12 +58,14 @@
   
   public static void main(String[] args) {
     RandomUtils.useTestSeed();
+    k = 10;
     DisplayDirichlet.generate2dSamples();
     generateResults();
     new Display2dASNDirichlet();
   }
   
   private static void generateResults() {
-    DisplayDirichlet.generateResults(new AsymmetricSampledNormalDistribution(new VectorWritable(new DenseVector(2))));
+    DisplayDirichlet.generateResults(new AsymmetricSampledNormalDistribution(new VectorWritable(
+        new DenseVector(2))));
   }
 }

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayDirichlet.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayDirichlet.java?rev=917396&r1=917395&r2=917396&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayDirichlet.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayDirichlet.java Mon Mar  1 05:42:35 2010
@@ -36,22 +36,21 @@
 import org.apache.mahout.math.DenseVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.math.VectorWritable;
-import org.apache.mahout.math.function.TimesFunction;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class DisplayDirichlet extends Frame {
-
+  
   private static final Logger log = LoggerFactory.getLogger(DisplayDirichlet.class);
-
+  
   private static final List<Vector> sampleParams = new ArrayList<Vector>();
-   
+  
   protected static final int ds = 72; // default scale = 72 pixels per inch
   
   protected static final int size = 8; // screen size in inches
   
   protected static final List<VectorWritable> sampleData = new ArrayList<VectorWritable>();
- 
+  
   protected static final double significance = 0.05;
   
   protected static final Color[] colors = {Color.red, Color.orange, Color.yellow, Color.green, Color.blue,
@@ -61,6 +60,8 @@
   
   protected int res; // screen resolution
   
+  protected static int k = 12;
+  
   public DisplayDirichlet() {
     initialize();
   }
@@ -97,7 +98,9 @@
     Vector v = new DenseVector(2);
     Vector dv = new DenseVector(2);
     g2.setColor(Color.RED);
+    int i = 0;
     for (Vector param : sampleParams) {
+      i++;
       v.set(0, param.get(0));
       v.set(1, param.get(1));
       dv.set(0, param.get(2) * 3);
@@ -137,14 +140,12 @@
    */
   public static void plotRectangle(Graphics2D g2, Vector v, Vector dv) {
     double[] flip = {1, -1};
-    Vector v2 = v.clone().assign(new DenseVector(flip), new TimesFunction());
+    Vector v2 = v.times(new DenseVector(flip));
     v2 = v2.minus(dv.divide(2));
     int h = size / 2;
     double x = v2.get(0) + h;
     double y = v2.get(1) + h;
-    g2.draw(new Rectangle2D.Double(x * ds, y * ds, dv.get(0)
-                                                                                     * ds,
-        dv.get(1) * ds));
+    g2.draw(new Rectangle2D.Double(x * ds, y * ds, dv.get(0) * ds, dv.get(1) * ds));
   }
   
   /**
@@ -159,7 +160,7 @@
    */
   public static void plotEllipse(Graphics2D g2, Vector v, Vector dv) {
     double[] flip = {1, -1};
-    Vector v2 = v.clone().assign(new DenseVector(flip), new TimesFunction());
+    Vector v2 = v.times(new DenseVector(flip));
     v2 = v2.minus(dv.divide(2));
     int h = size / 2;
     double x = v2.get(0) + h;
@@ -185,13 +186,13 @@
   }
   
   public static void generateSamples() {
-    generateSamples(400, 1, 1, 3);
+    generateSamples(500, 1, 1, 3);
     generateSamples(300, 1, 0, 0.5);
     generateSamples(300, 0, 2, 0.1);
   }
   
   public static void generate2dSamples() {
-    generate2dSamples(400, 1, 1, 3, 1);
+    generate2dSamples(500, 1, 1, 3, 1);
     generate2dSamples(300, 1, 0, 0.5, 1);
     generate2dSamples(300, 0, 2, 0.1, 0.5);
   }
@@ -213,10 +214,8 @@
     sampleParams.add(new DenseVector(params));
     log.info("Generating {} samples m=[{}, {}] sd={}", new Object[] {num, mx, my, sd});
     for (int i = 0; i < num; i++) {
-      sampleData.add(new VectorWritable(new DenseVector(new double[] {
-          UncommonDistributions.rNorm(mx, sd),
-          UncommonDistributions.rNorm(my, sd)}
-      )));
+      sampleData.add(new VectorWritable(new DenseVector(new double[] {UncommonDistributions.rNorm(mx, sd),
+                                                                      UncommonDistributions.rNorm(my, sd)})));
     }
   }
   
@@ -246,8 +245,8 @@
   }
   
   public static void generateResults(ModelDistribution<VectorWritable> modelDist) {
-    DirichletClusterer<VectorWritable> dc = new DirichletClusterer<VectorWritable>(
-        sampleData, modelDist, 1.0, 10, 2, 2);
+    DirichletClusterer<VectorWritable> dc = new DirichletClusterer<VectorWritable>(sampleData, modelDist,
+        1.0, k, 2, 2);
     result = dc.cluster(20);
     printModels(result, 5);
   }

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/fuzzykmeans/DisplayFuzzyKMeans.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/fuzzykmeans/DisplayFuzzyKMeans.java?rev=917396&r1=917395&r2=917396&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/fuzzykmeans/DisplayFuzzyKMeans.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/fuzzykmeans/DisplayFuzzyKMeans.java Mon Mar  1 05:42:35 2010
@@ -21,10 +21,8 @@
 import java.awt.Graphics;
 import java.awt.Graphics2D;
 import java.util.ArrayList;
-import java.util.Iterator;
 import java.util.List;
 
-import org.apache.mahout.clustering.canopy.Canopy;
 import org.apache.mahout.clustering.dirichlet.DisplayDirichlet;
 import org.apache.mahout.common.RandomUtils;
 import org.apache.mahout.common.distance.DistanceMeasure;
@@ -56,139 +54,30 @@
       g2.setColor(colors[Math.min(DisplayDirichlet.colors.length - 1, i--)]);
       for (SoftCluster cluster : cls) {
         // if (true || cluster.getWeightedPointTotal().zSum() > sampleData.size() * 0.05) {
-        dv.assign(cluster.std() * 3);
+        dv.assign(Math.max(cluster.std(), 0.3) * 3);
         DisplayDirichlet.plotEllipse(g2, cluster.getCenter(), dv);
         // }
       }
     }
   }
   
-  public static void referenceFuzzyKMeans(List<VectorWritable> points,
-                                          DistanceMeasure measure,
-                                          double threshold,
-                                          double m,
-                                          int numIter) {
-    FuzzyKMeansClusterer clusterer = new FuzzyKMeansClusterer(measure, threshold, m);
-    boolean converged = false;
-    int iteration = 0;
-    for (int iter = 0; !converged && iter < numIter; iter++) {
-      List<SoftCluster> next = new ArrayList<SoftCluster>();
-      List<SoftCluster> cs = DisplayFuzzyKMeans.clusters.get(iteration++);
-      for (SoftCluster c : cs) {
-        next.add(new SoftCluster(c.getCenter()));
-      }
-      DisplayFuzzyKMeans.clusters.add(next);
-      converged = iterateReference(points, DisplayFuzzyKMeans.clusters.get(iteration),
-        clusterer);
-    }
-  }
-  
-  /**
-   * Perform a single iteration over the points and clusters, assigning points to clusters and returning if
-   * the iterations are completed.
-   * 
-   * @param points
-   *          the List<Vector> having the input points
-   * @param clusterList
-   *          the List<Cluster> clusters
-   * @return
-   */
-  public static boolean iterateReference(List<VectorWritable> points,
-                                         List<SoftCluster> clusterList,
-                                         FuzzyKMeansClusterer clusterer) {
-    // for each
-    for (VectorWritable point : points) {
-      List<Double> clusterDistanceList = new ArrayList<Double>();
-      for (SoftCluster cluster : clusterList) {
-        clusterDistanceList.add(clusterer.getMeasure().distance(point.get(), cluster.getCenter()));
-      }
-      
-      for (int i = 0; i < clusterList.size(); i++) {
-        double probWeight = clusterer.computeProbWeight(clusterDistanceList.get(i), clusterDistanceList);
-        clusterList.get(i).addPoint(point.get(), Math.pow(probWeight, clusterer.getM()));
-      }
-    }
-    boolean converged = true;
-    for (SoftCluster cluster : clusterList) {
-      if (!clusterer.computeConvergence(cluster)) {
-        converged = false;
-      }
-    }
-    // update the cluster centers
-    if (!converged) {
-      for (SoftCluster cluster : clusterList) {
-        cluster.recomputeCenter();
-      }
-    }
-    return converged;
-    
-  }
-  
-  /**
-   * Iterate through the points, adding new canopies. Return the canopies.
-   * 
-   * @param measure
-   *          a DistanceMeasure to use
-   * @param points
-   *          a list<Vector> defining the points to be clustered
-   * @param t1
-   *          the T1 distance threshold
-   * @param t2
-   *          the T2 distance threshold
-   * @return the List<Canopy> created
-   */
-  static List<Canopy> populateCanopies(DistanceMeasure measure,
-                                       List<VectorWritable> points,
-                                       double t1,
-                                       double t2) {
-    List<Canopy> canopies = new ArrayList<Canopy>();
-    /**
-     * Reference Implementation: Given a distance metric, one can create canopies as follows: Start with a
-     * list of the data points in any order, and with two distance thresholds, T1 and T2, where T1 > T2.
-     * (These thresholds can be set by the user, or selected by cross-validation.) Pick a point on the list
-     * and measure its distance to all other points. Put all points that are within distance threshold T1 into
-     * a canopy. Remove from the list all points that are within distance threshold T2. Repeat until the list
-     * is empty.
-     */
-    int nextCanopyId = 0;
-    while (!points.isEmpty()) {
-      Iterator<VectorWritable> ptIter = points.iterator();
-      Vector p1 = ptIter.next().get();
-      ptIter.remove();
-      Canopy canopy = new Canopy(p1, nextCanopyId++);
-      canopies.add(canopy);
-      while (ptIter.hasNext()) {
-        Vector p2 = ptIter.next().get();
-        double dist = measure.distance(p1, p2);
-        // Put all points that are within distance threshold T1 into the canopy
-        if (dist < t1) {
-          canopy.addPoint(p2);
-        }
-        // Remove from the list all points that are within distance threshold T2
-        if (dist < t2) {
-          ptIter.remove();
-        }
-      }
-    }
-    return canopies;
-  }
-  
   public static void main(String[] args) {
     RandomUtils.useTestSeed();
     DisplayDirichlet.generateSamples();
-    List<VectorWritable> points = new ArrayList<VectorWritable>();
-    points.addAll(sampleData);
-    List<Canopy> canopies = populateCanopies(new ManhattanDistanceMeasure(), points,
-      t1, t2);
+    List<Vector> points = new ArrayList<Vector>();
+    for (VectorWritable sample : sampleData)
+      points.add(sample.get());
     DistanceMeasure measure = new ManhattanDistanceMeasure();
-    clusters = new ArrayList<List<SoftCluster>>();
-    DisplayFuzzyKMeans.clusters.add(new ArrayList<SoftCluster>());
-    for (Canopy canopy : canopies) {
-      if (canopy.getNumPoints() > 0.05 * DisplayDirichlet.sampleData.size()) {
-        DisplayFuzzyKMeans.clusters.get(0).add(new SoftCluster(canopy.getCenter()));
-      }
+    List<SoftCluster> initialClusters = new ArrayList<SoftCluster>();
+    
+    k = 3;
+    int i = 0;
+    for (Vector point : points) {
+      if (initialClusters.size() < Math.min(k, points.size())) {
+        initialClusters.add(new SoftCluster(point, i++));
+      } else break;
     }
-    referenceFuzzyKMeans(sampleData, measure, 0.001, 2, 10);
+    clusters = FuzzyKMeansClusterer.clusterPoints(points, initialClusters, measure, 0.001, 3, 10);
     new DisplayFuzzyKMeans();
   }
 }

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/kmeans/DisplayKMeans.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/kmeans/DisplayKMeans.java?rev=917396&r1=917395&r2=917396&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/kmeans/DisplayKMeans.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/kmeans/DisplayKMeans.java Mon Mar  1 05:42:35 2010
@@ -21,10 +21,8 @@
 import java.awt.Graphics;
 import java.awt.Graphics2D;
 import java.util.ArrayList;
-import java.util.Iterator;
 import java.util.List;
 
-import org.apache.mahout.clustering.canopy.Canopy;
 import org.apache.mahout.clustering.dirichlet.DisplayDirichlet;
 import org.apache.mahout.common.RandomUtils;
 import org.apache.mahout.common.distance.DistanceMeasure;
@@ -57,151 +55,30 @@
       for (Cluster cluster : cls) {
         // if (true || cluster.getNumPoints() > sampleData.size() * 0.05) {
         dv.assign(cluster.getStd() * 3);
+        System.out.println(cluster.getCenter().asFormatString() + " " + dv.asFormatString());
         DisplayDirichlet.plotEllipse(g2, cluster.getCenter(), dv);
         // }
       }
     }
   }
   
-  /**
-   * This is the reference k-means implementation. Given its inputs it iterates over the points and clusters
-   * until their centers converge or until the maximum number of iterations is exceeded.
-   * 
-   * @param points
-   *          the input List<Vector> of points
-   * @param clusters
-   *          the initial List<Cluster> of clusters
-   * @param measure
-   *          the DistanceMeasure to use
-   * @param maxIter
-   *          the maximum number of iterations
-   */
-  private static void referenceKmeans(List<VectorWritable> points,
-                                      List<List<Cluster>> clusters,
-                                      DistanceMeasure measure,
-                                      int maxIter) {
-    boolean converged = false;
-    int iteration = 0;
-    while (!converged && iteration < maxIter) {
-      List<Cluster> next = new ArrayList<Cluster>();
-      List<Cluster> cs = clusters.get(iteration++);
-      for (Cluster c : cs) {
-        next.add(new Cluster(c.getCenter()));
-      }
-      clusters.add(next);
-      converged = iterateReference(points, clusters.get(iteration), measure);
-    }
-  }
-  
-  /**
-   * Perform a single iteration over the points and clusters, assigning points to clusters and returning if
-   * the iterations are completed.
-   * 
-   * @param points
-   *          the List<Vector> having the input points
-   * @param clusters
-   *          the List<Cluster> clusters
-   * @param measure
-   *          a DistanceMeasure to use
-   * @return
-   */
-  private static boolean iterateReference(List<VectorWritable> points,
-                                          List<Cluster> clusters,
-                                          DistanceMeasure measure) {
-    // iterate through all points, assigning each to the nearest cluster
-    for (VectorWritable point : points) {
-      Cluster closestCluster = null;
-      double closestDistance = Double.MAX_VALUE;
-      for (Cluster cluster : clusters) {
-        double distance = measure.distance(cluster.getCenter(), point.get());
-        if (closestCluster == null || closestDistance > distance) {
-          closestCluster = cluster;
-          closestDistance = distance;
-        }
-      }
-      closestCluster.addPoint(point.get());
-    }
-    // test for convergence
-    boolean converged = true;
-    for (Cluster cluster : clusters) {
-      if (!cluster.computeConvergence(measure, 0.001)) {
-        converged = false;
-      }
-    }
-    // update the cluster centers
-    if (!converged) {
-      for (Cluster cluster : clusters) {
-        cluster.recomputeCenter();
-      }
-    }
-    return converged;
-  }
-  
-  /**
-   * Iterate through the points, adding new canopies. Return the canopies.
-   * 
-   * @param measure
-   *          a DistanceMeasure to use
-   * @param points
-   *          a list<Vector> defining the points to be clustered
-   * @param t1
-   *          the T1 distance threshold
-   * @param t2
-   *          the T2 distance threshold
-   * @return the List<Canopy> created
-   */
-  static List<Canopy> populateCanopies(DistanceMeasure measure,
-                                       List<VectorWritable> points,
-                                       double t1,
-                                       double t2) {
-    List<Canopy> canopies = new ArrayList<Canopy>();
-    /**
-     * Reference Implementation: Given a distance metric, one can create canopies as follows: Start with a
-     * list of the data points in any order, and with two distance thresholds, T1 and T2, where T1 > T2.
-     * (These thresholds can be set by the user, or selected by cross-validation.) Pick a point on the list
-     * and measure its distance to all other points. Put all points that are within distance threshold T1 into
-     * a canopy. Remove from the list all points that are within distance threshold T2. Repeat until the list
-     * is empty.
-     */
-    int nextCanopyId = 0;
-    while (!points.isEmpty()) {
-      Iterator<VectorWritable> ptIter = points.iterator();
-      Vector p1 = ptIter.next().get();
-      ptIter.remove();
-      Canopy canopy = new Canopy(p1, nextCanopyId++);
-      canopies.add(canopy);
-      while (ptIter.hasNext()) {
-        Vector p2 = ptIter.next().get();
-        double dist = measure.distance(p1, p2);
-        // Put all points that are within distance threshold T1 into the canopy
-        if (dist < t1) {
-          canopy.addPoint(p2);
-        }
-        // Remove from the list all points that are within distance threshold T2
-        if (dist < t2) {
-          ptIter.remove();
-        }
-      }
-    }
-    return canopies;
-  }
-  
   public static void main(String[] args) {
     RandomUtils.useTestSeed();
     DisplayDirichlet.generateSamples();
-    List<VectorWritable> points = new ArrayList<VectorWritable>();
-    points.addAll(sampleData);
-    List<Canopy> canopies = populateCanopies(new ManhattanDistanceMeasure(), points,
-      t1, t2);
+    List<Vector> points = new ArrayList<Vector>();
+    for (VectorWritable sample : sampleData)
+      points.add(sample.get());
     DistanceMeasure measure = new ManhattanDistanceMeasure();
-    clusters = new ArrayList<List<Cluster>>();
-    DisplayKMeans.clusters.add(new ArrayList<Cluster>());
-    for (Canopy canopy : canopies) {
-      if (canopy.getNumPoints() > 0.05 * DisplayDirichlet.sampleData.size()) {
-        DisplayKMeans.clusters.get(0).add(new Cluster(canopy.getCenter()));
-      }
+    List<Cluster> initialClusters = new ArrayList<Cluster>();
+    k = 3;
+    int i = 0;
+    for (Vector point : points) {
+      if (initialClusters.size() < Math.min(k, points.size())) {
+        initialClusters.add(new Cluster(point, i++));
+      } else break;
     }
-    referenceKmeans(sampleData, clusters, measure, 10);
+    clusters = KMeansClusterer.clusterPoints(points, initialClusters, measure, 10, 0.001);
+    System.out.println(clusters.size());
     new DisplayKMeans();
   }
 }

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/meanshift/DisplayMeanShift.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/meanshift/DisplayMeanShift.java?rev=917396&r1=917395&r2=917396&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/meanshift/DisplayMeanShift.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/meanshift/DisplayMeanShift.java Mon Mar  1 05:42:35 2010
@@ -35,19 +35,20 @@
 import org.slf4j.LoggerFactory;
 
 class DisplayMeanShift extends DisplayDirichlet {
-
-  private static final Logger log = LoggerFactory.getLogger(DisplayMeanShift.class);  
   
-  private static final MeanShiftCanopyClusterer clusterer =
-    new MeanShiftCanopyClusterer(new EuclideanDistanceMeasure(), 1.0, 0.05, 0.5);
+  private static final Logger log = LoggerFactory.getLogger(DisplayMeanShift.class);
+  
+  private static final MeanShiftCanopyClusterer clusterer = new MeanShiftCanopyClusterer(
+      new EuclideanDistanceMeasure(), 1.0, 0.05, 0.5);
   private static List<MeanShiftCanopy> canopies = new ArrayList<MeanShiftCanopy>();
   
   private DisplayMeanShift() {
     initialize();
     this.setTitle("Canopy Clusters (> 1.5% of population)");
   }
+  
   // TODO this is never queried?
-  //private static final List<List<Vector>> iterationCenters = new ArrayList<List<Vector>>();
+  // private static final List<List<Vector>> iterationCenters = new ArrayList<List<Vector>>();
   
   @Override
   public void paint(Graphics g) {
@@ -73,8 +74,8 @@
     for (MeanShiftCanopy canopy : canopies) {
       if (canopy.getBoundPoints().size() > 0.015 * DisplayDirichlet.sampleData.size()) {
         g2.setColor(colors[Math.min(i++, DisplayDirichlet.colors.length - 1)]);
-        for (Vector v : canopy.getBoundPoints()) {
-          DisplayDirichlet.plotRectangle(g2, v, dv);
+        for (int v : canopy.getBoundPoints().elements()) {
+          DisplayDirichlet.plotRectangle(g2, sampleData.get(v).get(), dv);
         }
         DisplayDirichlet.plotEllipse(g2, canopy.getCenter(), dv1);
         DisplayDirichlet.plotEllipse(g2, canopy.getCenter(), dv2);
@@ -82,32 +83,14 @@
     }
   }
   
-  private static void testReferenceImplementation() {
-    // add all points to the canopies
-    int nextCanopyId = 0;
-    for (VectorWritable aRaw : sampleData) {
-      DisplayMeanShift.clusterer.mergeCanopy(
-          new MeanShiftCanopy(aRaw.get(), nextCanopyId++), canopies);
-    }
-    boolean done = false;
-    while (!done) { // shift canopies to their centroids
-      done = true;
-      List<MeanShiftCanopy> migratedCanopies = new ArrayList<MeanShiftCanopy>();
-      //List<Vector> centers = new ArrayList<Vector>();
-      for (MeanShiftCanopy canopy : canopies) {
-        //centers.add(canopy.getCenter());
-        done = DisplayMeanShift.clusterer.shiftToMean(canopy) && done;
-        DisplayMeanShift.clusterer.mergeCanopy(canopy, migratedCanopies);
-      }
-      //iterationCenters.add(centers);
-      canopies = migratedCanopies;
-    }
-  }
-  
   public static void main(String[] args) {
     RandomUtils.useTestSeed();
     DisplayDirichlet.generateSamples();
-    testReferenceImplementation();
+    List<Vector> points = new ArrayList<Vector>();
+    for (VectorWritable sample : sampleData)
+      points.add(sample.get());
+    canopies = MeanShiftCanopyClusterer.clusterPoints(points, new EuclideanDistanceMeasure(), 0.5, 1.0, 0.05,
+      10);
     for (MeanShiftCanopy canopy : canopies) {
       log.info(canopy.toString());
     }

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/meanshift/OutputMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/meanshift/OutputMapper.java?rev=917396&r1=917395&r2=917396&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/meanshift/OutputMapper.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/meanshift/OutputMapper.java Mon Mar  1 05:42:35 2010
@@ -39,8 +39,8 @@
   public void map(Text key, MeanShiftCanopy canopy, OutputCollector<Text,Text> output,
                   Reporter reporter) throws IOException {
     clusters++;
-    for (Vector point : canopy.getBoundPoints()) {
-      output.collect(key, new Text(point.asFormatString()));
+    for (int point : canopy.getBoundPoints().elements()) {
+      output.collect(key, new Text(String.valueOf(point)));
     }
   }