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

svn commit: r898669 [2/3] - in /lucene/mahout/trunk: core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/ core/src/main/java/org/apache/mahout/clustering/canopy/ core/src/main/java/org/apache/mahout/clustering/dirichlet/ core/src/main/java/org/ap...

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=898669&r1=898668&r2=898669&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 Wed Jan 13 08:01:34 2010
@@ -34,6 +34,7 @@
 import org.apache.mahout.common.distance.EuclideanDistanceMeasure;
 import org.apache.mahout.common.distance.ManhattanDistanceMeasure;
 import org.apache.mahout.common.distance.UserDefinedDistanceMeasure;
+import org.apache.mahout.math.VectorWritable;
 
 import java.io.File;
 import java.util.ArrayList;
@@ -64,13 +65,13 @@
     super(name);
   }
 
-  private static List<Vector> getPoints(double[][] raw) {
-    List<Vector> points = new ArrayList<Vector>();
+  private static List<VectorWritable> getPoints(double[][] raw) {
+    List<VectorWritable> points = new ArrayList<VectorWritable>();
     int i = 0;
     for (double[] fr : raw) {
       Vector vec = new SparseVector(String.valueOf(i++), fr.length);
       vec.assign(fr);
-      points.add(vec);
+      points.add(new VectorWritable(vec));
     }
     return points;
   }
@@ -170,7 +171,7 @@
    * @return the List<Canopy> created
    */
   static List<Canopy> populateCanopies(DistanceMeasure measure,
-                                       List<Vector> points, double t1, double t2) {
+                                       List<VectorWritable> points, double t1, double t2) {
     List<Canopy> canopies = new ArrayList<Canopy>();
     /**
      * Reference Implementation: Given a distance metric, one can create
@@ -184,13 +185,13 @@
      */
     int nextCanopyId = 0;
     while (!points.isEmpty()) {
-      Iterator<Vector> ptIter = points.iterator();
-      Vector p1 = ptIter.next();
+      Iterator<VectorWritable> ptIter = points.iterator();
+      Vector p1 = ptIter.next().get();
       ptIter.remove();
       Canopy canopy = new VisibleCanopy(p1, nextCanopyId++);
       canopies.add(canopy);
       while (ptIter.hasNext()) {
-        Vector p2 = ptIter.next();
+        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) {
@@ -251,12 +252,12 @@
 
   /** Story: User can cluster points without instantiating them all in memory at once */
   public void testIterativeManhattan() throws Exception {
-    List<Vector> points = getPoints(raw);
+    List<VectorWritable> points = getPoints(raw);
     CanopyClusterer clusterer = new CanopyClusterer(new ManhattanDistanceMeasure(), 3.1, 2.1);
 
     List<Canopy> canopies = new ArrayList<Canopy>();
-    for (Vector point : points) {
-      clusterer.addPointToCanopies(point, canopies);
+    for (VectorWritable point : points) {
+      clusterer.addPointToCanopies(point.get(), canopies);
     }
 
     System.out.println("testIterativeManhattan");
@@ -266,12 +267,12 @@
 
   /** Story: User can cluster points without instantiating them all in memory at once */
   public void testIterativeEuclidean() throws Exception {
-    List<Vector> points = getPoints(raw);
+    List<VectorWritable> points = getPoints(raw);
     CanopyClusterer clusterer = new CanopyClusterer(new EuclideanDistanceMeasure(), 3.1, 2.1);
 
     List<Canopy> canopies = new ArrayList<Canopy>();
-    for (Vector point : points) {
-      clusterer.addPointToCanopies(point, canopies);
+    for (VectorWritable point : points) {
+      clusterer.addPointToCanopies(point.get(), canopies);
     }
 
     System.out.println("testIterativeEuclidean");
@@ -291,21 +292,21 @@
     conf.set(CanopyConfigKeys.T2_KEY, String.valueOf(2.1));
     mapper.configure(conf);
     
-    DummyOutputCollector<Text, Vector> collector = new DummyOutputCollector<Text, Vector>();
-    List<Vector> points = getPoints(raw);
+    DummyOutputCollector<Text, VectorWritable> collector = new DummyOutputCollector<Text, VectorWritable>();
+    List<VectorWritable> points = getPoints(raw);
     // map the data
-    for (Vector point : points) {
+    for (VectorWritable point : points) {
       mapper.map(new Text(), point, collector, null);
     }
     mapper.close();
     assertEquals("Number of map results", 1, collector.getData().size());
     // now verify the output
-    List<Vector> data = collector.getValue("centroid");
+    List<VectorWritable> data = collector.getValue("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)
-              .asFormatString());
+          manhattanCentroids.get(i).asFormatString(),
+          data.get(i).get().asFormatString());
     }
   }
 
@@ -321,21 +322,21 @@
     conf.set(CanopyConfigKeys.T2_KEY, String.valueOf(2.1));
     mapper.configure(conf);
     
-    DummyOutputCollector<Text, Vector> collector = new DummyOutputCollector<Text, Vector>();
-    List<Vector> points = getPoints(raw);
+    DummyOutputCollector<Text, VectorWritable> collector = new DummyOutputCollector<Text, VectorWritable>();
+    List<VectorWritable> points = getPoints(raw);
     // map the data
-    for (Vector point : points) {
+    for (VectorWritable point : points) {
       mapper.map(new Text(), point, collector, null);
     }
     mapper.close();
     assertEquals("Number of map results", 1, collector.getData().size());
     // now verify the output
-    List<Vector> data = collector.getValue("centroid");
+    List<VectorWritable> data = collector.getValue("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)
-              .asFormatString());
+          euclideanCentroids.get(i).asFormatString(),
+          data.get(i).get().asFormatString());
     }
   }
 
@@ -352,7 +353,7 @@
     reducer.configure(conf);
     
     DummyOutputCollector<Text, Canopy> collector = new DummyOutputCollector<Text, Canopy>();
-    List<Vector> points = getPoints(raw);
+    List<VectorWritable> points = getPoints(raw);
     reducer.reduce(new Text("centroid"), points.iterator(), collector, null);
     reducer.close();
     Set<String> keys = collector.getKeys();
@@ -378,7 +379,7 @@
     reducer.configure(conf);
     
     DummyOutputCollector<Text, Canopy> collector = new DummyOutputCollector<Text, Canopy>();
-    List<Vector> points = getPoints(raw);
+    List<VectorWritable> points = getPoints(raw);
     reducer.reduce(new Text("centroid"), points.iterator(), collector, null);
     reducer.close();
     Set<String> keys = collector.getKeys();
@@ -393,7 +394,7 @@
 
   /** Story: User can produce final canopy centers using a Hadoop map/reduce job and a ManhattanDistanceMeasure. */
   public void testCanopyGenManhattanMR() throws Exception {
-    List<Vector> points = getPoints(raw);
+    List<VectorWritable> points = getPoints(raw);
     File testData = new File("testdata");
     if (!testData.exists()) {
       testData.mkdir();
@@ -402,8 +403,7 @@
     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, SparseVector.class);
+    CanopyDriver.runJob("testdata", "output/canopies", ManhattanDistanceMeasure.class.getName(), 3.1, 2.1);
 
     // verify output from sequence file
     Path path = new Path("output/canopies/part-00000");
@@ -427,7 +427,7 @@
 
   /** Story: User can produce final canopy centers using a Hadoop map/reduce job and a EuclideanDistanceMeasure. */
   public void testCanopyGenEuclideanMR() throws Exception {
-    List<Vector> points = getPoints(raw);
+    List<VectorWritable> points = getPoints(raw);
     File testData = new File("testdata");
     if (!testData.exists()) {
       testData.mkdir();
@@ -436,8 +436,7 @@
     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, SparseVector.class);
+    CanopyDriver.runJob("testdata", "output/canopies", EuclideanDistanceMeasure.class.getName(), 3.1, 2.1);
 
     // verify output from sequence file
     Path path = new Path("output/canopies/part-00000");
@@ -467,25 +466,25 @@
     mapper.configure(conf);
     
     List<Canopy> canopies = new ArrayList<Canopy>();
-    DummyOutputCollector<Text, Vector> collector = new DummyOutputCollector<Text, Vector>();
+    DummyOutputCollector<Text, VectorWritable> collector = new DummyOutputCollector<Text, VectorWritable>();
     int nextCanopyId = 0;
     for (Vector centroid : manhattanCentroids) {
       canopies.add(new Canopy(centroid, nextCanopyId++));
     }
     mapper.config(canopies);
-    List<Vector> points = getPoints(raw);
+    List<VectorWritable> points = getPoints(raw);
     // map the data
-    for (Vector point : points) {
+    for (VectorWritable point : points) {
       mapper.map(new Text(), point, collector, null);
     }
-    Map<String, List<Vector>> data = collector.getData();
+    Map<String, List<VectorWritable>> data = collector.getData();
     assertEquals("Number of map results", canopies.size(), data.size());
-    for (Map.Entry<String, List<Vector>> stringListEntry : data.entrySet()) {
+    for (Map.Entry<String, List<VectorWritable>> stringListEntry : data.entrySet()) {
       String key = stringListEntry.getKey();
       Canopy canopy = findCanopy(key, canopies);
-      List<Vector> pts = stringListEntry.getValue();
-      for (Vector ptDef : pts) {
-        assertTrue("Point not in canopy", mapper.canopyCovers(canopy, ptDef));
+      List<VectorWritable> pts = stringListEntry.getValue();
+      for (VectorWritable ptDef : pts) {
+        assertTrue("Point not in canopy", mapper.canopyCovers(canopy, ptDef.get()));
       }
     }
   }
@@ -509,25 +508,25 @@
     mapper.configure(conf);
     
     List<Canopy> canopies = new ArrayList<Canopy>();
-    DummyOutputCollector<Text, Vector> collector = new DummyOutputCollector<Text, Vector>();
+    DummyOutputCollector<Text, VectorWritable> collector = new DummyOutputCollector<Text, VectorWritable>();
     int nextCanopyId = 0;
     for (Vector centroid : euclideanCentroids) {
       canopies.add(new Canopy(centroid, nextCanopyId++));
     }
     mapper.config(canopies);
-    List<Vector> points = getPoints(raw);
+    List<VectorWritable> points = getPoints(raw);
     // map the data
-    for (Vector point : points) {
+    for (VectorWritable point : points) {
       mapper.map(new Text(), point, collector, null);
     }
-    Map<String, List<Vector>> data = collector.getData();
+    Map<String, List<VectorWritable>> data = collector.getData();
     assertEquals("Number of map results", canopies.size(), data.size());
-    for (Map.Entry<String, List<Vector>> stringListEntry : data.entrySet()) {
+    for (Map.Entry<String, List<VectorWritable>> stringListEntry : data.entrySet()) {
       String key = stringListEntry.getKey();
       Canopy canopy = findCanopy(key, canopies);
-      List<Vector> pts = stringListEntry.getValue();
-      for (Vector ptDef : pts) {
-        assertTrue("Point not in canopy", mapper.canopyCovers(canopy, ptDef));
+      List<VectorWritable> pts = stringListEntry.getValue();
+      for (VectorWritable ptDef : pts) {
+        assertTrue("Point not in canopy", mapper.canopyCovers(canopy, ptDef.get()));
       }
     }
   }
@@ -542,36 +541,36 @@
     mapper.configure(conf);
     
     List<Canopy> canopies = new ArrayList<Canopy>();
-    DummyOutputCollector<Text, Vector> collector = new DummyOutputCollector<Text, Vector>();
+    DummyOutputCollector<Text, VectorWritable> collector = new DummyOutputCollector<Text, VectorWritable>();
     int nextCanopyId = 0;
     for (Vector centroid : manhattanCentroids) {
       canopies.add(new Canopy(centroid, nextCanopyId++));
     }
     mapper.config(canopies);
-    List<Vector> points = getPoints(raw);
+    List<VectorWritable> points = getPoints(raw);
     // map the data
-    for (Vector point : points) {
+    for (VectorWritable point : points) {
       mapper.map(new Text(), point, collector, null);
     }
-    Map<String, List<Vector>> data = collector.getData();
+    Map<String, List<VectorWritable>> data = collector.getData();
     assertEquals("Number of map results", canopies.size(), data.size());
 
     // reduce the data
-    Reducer<Text, Vector, Text, Vector> reducer = new IdentityReducer<Text, Vector>();
-    collector = new DummyOutputCollector<Text, Vector>();
-    for (Map.Entry<String, List<Vector>> stringListEntry : data.entrySet()) {
+    Reducer<Text, VectorWritable, Text, VectorWritable> reducer = new IdentityReducer<Text, VectorWritable>();
+    collector = new DummyOutputCollector<Text, VectorWritable>();
+    for (Map.Entry<String, List<VectorWritable>> stringListEntry : data.entrySet()) {
       reducer.reduce(new Text(stringListEntry.getKey()), stringListEntry
           .getValue().iterator(), collector, null);
     }
 
     // check the output
     data = collector.getData();
-    for (Map.Entry<String, List<Vector>> stringListEntry : data.entrySet()) {
+    for (Map.Entry<String, List<VectorWritable>> stringListEntry : data.entrySet()) {
       String key = stringListEntry.getKey();
       Canopy canopy = findCanopy(key, canopies);
-      List<Vector> pts = stringListEntry.getValue();
-      for (Vector ptDef : pts) {
-        assertTrue("Point not in canopy", mapper.canopyCovers(canopy, ptDef));
+      List<VectorWritable> pts = stringListEntry.getValue();
+      for (VectorWritable ptDef : pts) {
+        assertTrue("Point not in canopy", mapper.canopyCovers(canopy, ptDef.get()));
       }
     }
   }
@@ -586,43 +585,43 @@
     mapper.configure(conf);
     
     List<Canopy> canopies = new ArrayList<Canopy>();
-    DummyOutputCollector<Text, Vector> collector = new DummyOutputCollector<Text, Vector>();
+    DummyOutputCollector<Text, VectorWritable> collector = new DummyOutputCollector<Text, VectorWritable>();
     int nextCanopyId = 0;
     for (Vector centroid : euclideanCentroids) {
       canopies.add(new Canopy(centroid, nextCanopyId++));
     }
     mapper.config(canopies);
-    List<Vector> points = getPoints(raw);
+    List<VectorWritable> points = getPoints(raw);
     // map the data
-    for (Vector point : points) {
+    for (VectorWritable point : points) {
       mapper.map(new Text(), point, collector, null);
     }
-    Map<String, List<Vector>> data = collector.getData();
+    Map<String, List<VectorWritable>> data = collector.getData();
 
     // reduce the data
-    Reducer<Text, Vector, Text, Vector> reducer = new IdentityReducer<Text, Vector>();
-    collector = new DummyOutputCollector<Text, Vector>();
-    for (Map.Entry<String, List<Vector>> stringListEntry : data.entrySet()) {
-      reducer.reduce(new Text(stringListEntry.getKey()), stringListEntry
-          .getValue().iterator(), collector, null);
+    Reducer<Text, VectorWritable, Text, VectorWritable> reducer = new IdentityReducer<Text, VectorWritable>();
+    collector = new DummyOutputCollector<Text, VectorWritable>();
+    for (Map.Entry<String, List<VectorWritable>> stringListEntry : data.entrySet()) {
+      reducer.reduce(new Text(stringListEntry.getKey()),
+          stringListEntry.getValue().iterator(), collector, null);
     }
 
     // check the output
     data = collector.getData();
     assertEquals("Number of map results", canopies.size(), data.size());
-    for (Map.Entry<String, List<Vector>> stringListEntry : data.entrySet()) {
+    for (Map.Entry<String, List<VectorWritable>> stringListEntry : data.entrySet()) {
       String key = stringListEntry.getKey();
       Canopy canopy = findCanopy(key, canopies);
-      List<Vector> pts = stringListEntry.getValue();
-      for (Vector ptDef : pts) {
-        assertTrue("Point not in canopy", mapper.canopyCovers(canopy, ptDef));
+      List<VectorWritable> pts = stringListEntry.getValue();
+      for (VectorWritable ptDef : pts) {
+        assertTrue("Point not in canopy", mapper.canopyCovers(canopy, ptDef.get()));
       }
     }
   }
 
   /** Story: User can produce final point clustering using a Hadoop map/reduce job and a ManhattanDistanceMeasure. */
   public void testClusteringManhattanMR() throws Exception {
-    List<Vector> points = getPoints(raw);
+    List<VectorWritable> points = getPoints(raw);
     File testData = new File("testdata");
     if (!testData.exists()) {
       testData.mkdir();
@@ -632,7 +631,7 @@
     ClusteringTestUtils.writePointsToFile(points, "testdata/file2", fs, conf);
     // now run the Job
     CanopyClusteringJob.runJob("testdata", "output",
-        ManhattanDistanceMeasure.class.getName(), 3.1, 2.1, SparseVector.class);
+        ManhattanDistanceMeasure.class.getName(), 3.1, 2.1);
     //TODO: change
     Path path = new Path("output/clusters/part-00000");
     SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, conf);
@@ -642,10 +641,10 @@
       count++;
     }*/
     Text txt = new Text();
-    SparseVector vector = new SparseVector();
+    VectorWritable vector = new VectorWritable();
     while (reader.next(txt, vector)) {
       count++;
-      System.out.println("Txt: " + txt + " Vec: " + vector.asFormatString());
+      System.out.println("Txt: " + txt + " Vec: " + vector.get().asFormatString());
     }
     // the point [3.0,3.0] is covered by both canopies
     assertEquals("number of points", 2 + 2 * points.size(), count);
@@ -654,7 +653,7 @@
 
   /** Story: User can produce final point clustering using a Hadoop map/reduce job and a EuclideanDistanceMeasure. */
   public void testClusteringEuclideanMR() throws Exception {
-    List<Vector> points = getPoints(raw);
+    List<VectorWritable> points = getPoints(raw);
     File testData = new File("testdata");
     if (!testData.exists()) {
       testData.mkdir();
@@ -664,7 +663,7 @@
     ClusteringTestUtils.writePointsToFile(points, "testdata/file2", fs, conf);
     // now run the Job
     CanopyClusteringJob.runJob("testdata", "output",
-        EuclideanDistanceMeasure.class.getName(), 3.1, 2.1, SparseVector.class);
+        EuclideanDistanceMeasure.class.getName(), 3.1, 2.1);
     Path path = new Path("output/clusters/part-00000");
     SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, conf);
     int count = 0;
@@ -673,7 +672,7 @@
       count++;
     }*/
     Text txt = new Text();
-    SparseVector can = new SparseVector();
+    VectorWritable can = new VectorWritable();
     while (reader.next(txt, can)) {
       count++;
     }
@@ -689,7 +688,7 @@
 
   /** Story: Clustering algorithm must support arbitrary user defined distance measure */
   public void testUserDefinedDistanceMeasure() throws Exception {
-    List<Vector> points = getPoints(raw);
+    List<VectorWritable> points = getPoints(raw);
     File testData = new File("testdata");
     if (!testData.exists()) {
       testData.mkdir();
@@ -699,8 +698,7 @@
     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, SparseVector.class);
+    CanopyDriver.runJob("testdata", "output/canopies", UserDefinedDistanceMeasure.class.getName(), 3.1, 2.1);
 
     // verify output from sequence file
     JobConf job = new JobConf(CanopyDriver.class);

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestDirichletClustering.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestDirichletClustering.java?rev=898669&r1=898668&r2=898669&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestDirichletClustering.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestDirichletClustering.java Wed Jan 13 08:01:34 2010
@@ -25,19 +25,20 @@
 import org.apache.mahout.math.DenseVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.common.RandomUtils;
+import org.apache.mahout.math.VectorWritable;
 
 import java.util.ArrayList;
 import java.util.List;
 
 public class TestDirichletClustering extends TestCase {
 
-  private List<Vector> sampleData;
+  private List<VectorWritable> sampleData;
 
   @Override
   protected void setUp() throws Exception {
     super.setUp();
     RandomUtils.useTestSeed();
-    sampleData = new ArrayList<Vector>();
+    sampleData = new ArrayList<VectorWritable>();
   }
 
   /**
@@ -52,17 +53,17 @@
     System.out.println("Generating " + num + " samples m=[" + mx + ", " + my
         + "] sd=" + sd);
     for (int i = 0; i < num; i++) {
-      sampleData.add(new DenseVector(new double[]{
+      sampleData.add(new VectorWritable(new DenseVector(new double[]{
           UncommonDistributions.rNorm(mx, sd),
-          UncommonDistributions.rNorm(my, sd)}));
+          UncommonDistributions.rNorm(my, sd)})));
     }
   }
 
-  private static void printResults(List<Model<Vector>[]> result, int significant) {
+  private static void printResults(List<Model<VectorWritable>[]> result, int significant) {
     int row = 0;
-    for (Model<Vector>[] r : result) {
+    for (Model<VectorWritable>[] r : result) {
       System.out.print("sample[" + row++ + "]= ");
-      for (Model<Vector> model : r) {
+      for (Model<VectorWritable> model : r) {
         if (model.count() > significant) {
           System.out.print(model.toString() + ", ");
         }
@@ -78,9 +79,9 @@
     generateSamples(30, 1, 0, 0.1);
     generateSamples(30, 0, 1, 0.1);
 
-    DirichletClusterer<Vector> dc = new DirichletClusterer<Vector>(sampleData,
+    DirichletClusterer<VectorWritable> dc = new DirichletClusterer<VectorWritable>(sampleData,
         new NormalModelDistribution(), 1.0, 10, 1, 0);
-    List<Model<Vector>[]> result = dc.cluster(30);
+    List<Model<VectorWritable>[]> result = dc.cluster(30);
     printResults(result, 2);
     assertNotNull(result);
   }
@@ -91,9 +92,9 @@
     generateSamples(30, 1, 0, 0.1);
     generateSamples(30, 0, 1, 0.1);
 
-    DirichletClusterer<Vector> dc = new DirichletClusterer<Vector>(sampleData,
+    DirichletClusterer<VectorWritable> dc = new DirichletClusterer<VectorWritable>(sampleData,
         new SampledNormalDistribution(), 1.0, 10, 1, 0);
-    List<Model<Vector>[]> result = dc.cluster(30);
+    List<Model<VectorWritable>[]> result = dc.cluster(30);
     printResults(result, 2);
     assertNotNull(result);
   }
@@ -104,9 +105,9 @@
     generateSamples(30, 1, 0, 0.1);
     generateSamples(30, 0, 1, 0.1);
 
-    DirichletClusterer<Vector> dc = new DirichletClusterer<Vector>(sampleData,
+    DirichletClusterer<VectorWritable> dc = new DirichletClusterer<VectorWritable>(sampleData,
         new AsymmetricSampledNormalDistribution(), 1.0, 10, 1, 0);
-    List<Model<Vector>[]> result = dc.cluster(30);
+    List<Model<VectorWritable>[]> result = dc.cluster(30);
     printResults(result, 2);
     assertNotNull(result);
   }
@@ -117,9 +118,9 @@
     generateSamples(300, 1, 0, 0.1);
     generateSamples(300, 0, 1, 0.1);
 
-    DirichletClusterer<Vector> dc = new DirichletClusterer<Vector>(sampleData,
+    DirichletClusterer<VectorWritable> dc = new DirichletClusterer<VectorWritable>(sampleData,
         new NormalModelDistribution(), 1.0, 10, 1, 0);
-    List<Model<Vector>[]> result = dc.cluster(30);
+    List<Model<VectorWritable>[]> result = dc.cluster(30);
     printResults(result, 20);
     assertNotNull(result);
   }
@@ -130,9 +131,9 @@
     generateSamples(300, 1, 0, 0.1);
     generateSamples(300, 0, 1, 0.1);
 
-    DirichletClusterer<Vector> dc = new DirichletClusterer<Vector>(sampleData,
+    DirichletClusterer<VectorWritable> dc = new DirichletClusterer<VectorWritable>(sampleData,
         new SampledNormalDistribution(), 1.0, 10, 1, 0);
-    List<Model<Vector>[]> result = dc.cluster(30);
+    List<Model<VectorWritable>[]> result = dc.cluster(30);
     printResults(result, 20);
     assertNotNull(result);
   }
@@ -143,9 +144,9 @@
     generateSamples(300, 1, 0, 0.1);
     generateSamples(300, 0, 1, 0.1);
 
-    DirichletClusterer<Vector> dc = new DirichletClusterer<Vector>(sampleData,
+    DirichletClusterer<VectorWritable> dc = new DirichletClusterer<VectorWritable>(sampleData,
         new AsymmetricSampledNormalDistribution(), 1.0, 10, 1, 0);
-    List<Model<Vector>[]> result = dc.cluster(30);
+    List<Model<VectorWritable>[]> result = dc.cluster(30);
     printResults(result, 20);
     assertNotNull(result);
   }
@@ -156,9 +157,9 @@
     generateSamples(3000, 1, 0, 0.1);
     generateSamples(3000, 0, 1, 0.1);
 
-    DirichletClusterer<Vector> dc = new DirichletClusterer<Vector>(sampleData,
+    DirichletClusterer<VectorWritable> dc = new DirichletClusterer<VectorWritable>(sampleData,
         new NormalModelDistribution(), 1.0, 10, 1, 0);
-    List<Model<Vector>[]> result = dc.cluster(30);
+    List<Model<VectorWritable>[]> result = dc.cluster(30);
     printResults(result, 200);
     assertNotNull(result);
   }
@@ -169,9 +170,9 @@
     generateSamples(3000, 1, 0, 0.1);
     generateSamples(3000, 0, 1, 0.1);
 
-    DirichletClusterer<Vector> dc = new DirichletClusterer<Vector>(sampleData,
+    DirichletClusterer<VectorWritable> dc = new DirichletClusterer<VectorWritable>(sampleData,
         new AsymmetricSampledNormalDistribution(), 1.0, 10, 1, 0);
-    List<Model<Vector>[]> result = dc.cluster(30);
+    List<Model<VectorWritable>[]> result = dc.cluster(30);
     printResults(result, 200);
     assertNotNull(result);
   }
@@ -182,9 +183,9 @@
     generateSamples(3000, 1, 0, 0.1);
     generateSamples(3000, 0, 1, 0.1);
 
-    DirichletClusterer<Vector> dc = new DirichletClusterer<Vector>(sampleData,
+    DirichletClusterer<VectorWritable> dc = new DirichletClusterer<VectorWritable>(sampleData,
         new SampledNormalDistribution(), 1.0, 10, 1, 0);
-    List<Model<Vector>[]> result = dc.cluster(30);
+    List<Model<VectorWritable>[]> result = dc.cluster(30);
     printResults(result, 200);
     assertNotNull(result);
   }

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=898669&r1=898668&r2=898669&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 Wed Jan 13 08:01:34 2010
@@ -33,10 +33,7 @@
 import org.apache.mahout.clustering.dirichlet.models.SampledNormalDistribution;
 import org.apache.mahout.clustering.dirichlet.models.SampledNormalModel;
 import org.apache.mahout.clustering.kmeans.KMeansDriver;
-import org.apache.mahout.math.DenseVector;
-import org.apache.mahout.math.JsonVectorAdapter;
-import org.apache.mahout.math.SparseVector;
-import org.apache.mahout.math.Vector;
+import org.apache.mahout.math.*;
 import org.apache.mahout.common.DummyOutputCollector;
 import org.apache.mahout.common.RandomUtils;
 
@@ -48,7 +45,7 @@
 
 public class TestMapReduce extends TestCase {
 
-  private List<Vector> sampleData = new ArrayList<Vector>();
+  private List<VectorWritable> sampleData = new ArrayList<VectorWritable>();
 
   private FileSystem fs;
 
@@ -79,7 +76,7 @@
     for (int j = 0; j < values.length; j++) {
       v.setQuick(j, values[j]);
     }
-    sampleData.add(v);
+    sampleData.add(new VectorWritable(v));
   }
 
   /**
@@ -114,16 +111,16 @@
   /** Test the basic Mapper */
   public void testMapper() throws Exception {
     generateSamples(10, 0, 0, 1);
-    DirichletState<Vector> state = new DirichletState<Vector>(
+    DirichletState<VectorWritable> state = new DirichletState<VectorWritable>(
         new NormalModelDistribution(), 5, 1, 0, 0);
     DirichletMapper mapper = new DirichletMapper();
     mapper.configure(state);
 
-    DummyOutputCollector<Text, Vector> collector = new DummyOutputCollector<Text, Vector>();
-    for (Vector v : sampleData) {
+    DummyOutputCollector<Text, VectorWritable> collector = new DummyOutputCollector<Text, VectorWritable>();
+    for (VectorWritable v : sampleData) {
       mapper.map(null, v, collector, null);
     }
-    Map<String, List<Vector>> data = collector.getData();
+    Map<String, List<VectorWritable>> data = collector.getData();
     // this seed happens to produce two partitions, but they work
     assertEquals("output size", 3, data.size());
   }
@@ -134,37 +131,38 @@
     generateSamples(100, 2, 0, 1);
     generateSamples(100, 0, 2, 1);
     generateSamples(100, 2, 2, 1);
-    DirichletState<Vector> state = new DirichletState<Vector>(
+    DirichletState<VectorWritable> state = new DirichletState<VectorWritable>(
         new SampledNormalDistribution(), 20, 1, 1, 0);
     DirichletMapper mapper = new DirichletMapper();
     mapper.configure(state);
 
-    DummyOutputCollector<Text, Vector> mapCollector = new DummyOutputCollector<Text, Vector>();
-    for (Vector v : sampleData) {
+    DummyOutputCollector<Text, VectorWritable> mapCollector = new DummyOutputCollector<Text, VectorWritable>();
+    for (VectorWritable v : sampleData) {
       mapper.map(null, v, mapCollector, null);
     }
-    Map<String, List<Vector>> data = mapCollector.getData();
+    Map<String, List<VectorWritable>> data = mapCollector.getData();
     // this seed happens to produce three partitions, but they work
     assertEquals("output size", 7, data.size());
 
     DirichletReducer reducer = new DirichletReducer();
     reducer.configure(state);
-    OutputCollector<Text, DirichletCluster<Vector>> reduceCollector = new DummyOutputCollector<Text, DirichletCluster<Vector>>();
+    OutputCollector<Text, DirichletCluster<VectorWritable>> reduceCollector =
+        new DummyOutputCollector<Text, DirichletCluster<VectorWritable>>();
     for (String key : mapCollector.getKeys()) {
       reducer.reduce(new Text(key), mapCollector.getValue(key).iterator(),
           reduceCollector, null);
     }
 
-    Model<Vector>[] newModels = reducer.getNewModels();
+    Model<VectorWritable>[] newModels = reducer.getNewModels();
     state.update(newModels);
   }
 
-  private static void printModels(Iterable<Model<Vector>[]> results, int significant) {
+  private static void printModels(Iterable<Model<VectorWritable>[]> results, int significant) {
     int row = 0;
-    for (Model<Vector>[] r : results) {
+    for (Model<VectorWritable>[] r : results) {
       System.out.print("sample[" + row++ + "]= ");
       for (int k = 0; k < r.length; k++) {
-        Model<Vector> model = r[k];
+        Model<VectorWritable> model = r[k];
         if (model.count() > significant) {
           System.out.print("m" + k + model.toString() + ", ");
         }
@@ -180,28 +178,29 @@
     generateSamples(100, 2, 0, 1);
     generateSamples(100, 0, 2, 1);
     generateSamples(100, 2, 2, 1);
-    DirichletState<Vector> state = new DirichletState<Vector>(
+    DirichletState<VectorWritable> state = new DirichletState<VectorWritable>(
         new SampledNormalDistribution(), 20, 1.0, 1, 0);
 
-    List<Model<Vector>[]> models = new ArrayList<Model<Vector>[]>();
+    List<Model<VectorWritable>[]> models = new ArrayList<Model<VectorWritable>[]>();
 
     for (int iteration = 0; iteration < 10; iteration++) {
       DirichletMapper mapper = new DirichletMapper();
       mapper.configure(state);
-      DummyOutputCollector<Text, Vector> mapCollector = new DummyOutputCollector<Text, Vector>();
-      for (Vector v : sampleData) {
+      DummyOutputCollector<Text, VectorWritable> mapCollector = new DummyOutputCollector<Text, VectorWritable>();
+      for (VectorWritable v : sampleData) {
         mapper.map(null, v, mapCollector, null);
       }
 
       DirichletReducer reducer = new DirichletReducer();
       reducer.configure(state);
-      OutputCollector<Text,DirichletCluster<Vector>> reduceCollector = new DummyOutputCollector<Text, DirichletCluster<Vector>>();
+      OutputCollector<Text,DirichletCluster<VectorWritable>> reduceCollector =
+          new DummyOutputCollector<Text, DirichletCluster<VectorWritable>>();
       for (String key : mapCollector.getKeys()) {
         reducer.reduce(new Text(key), mapCollector.getValue(key).iterator(),
             reduceCollector, null);
       }
 
-      Model<Vector>[] newModels = reducer.getNewModels();
+      Model<VectorWritable>[] newModels = reducer.getNewModels();
       state.update(newModels);
       models.add(newModels);
     }
@@ -247,12 +246,12 @@
 
   public void testSampledNormalDistributionSerialization() {
     SampledNormalDistribution dist = new SampledNormalDistribution();
-    Model<Vector>[] models = dist.sampleFromPrior(20);
+    Model<VectorWritable>[] models = dist.sampleFromPrior(20);
     GsonBuilder builder = new GsonBuilder();
     builder.registerTypeAdapter(Vector.class, new JsonVectorAdapter());
     Gson gson = builder.create();
     String jsonString = gson.toJson(models);
-    Model<Vector>[] models2 = gson.fromJson(jsonString, SampledNormalModel[].class);
+    Model<VectorWritable>[] models2 = gson.fromJson(jsonString, SampledNormalModel[].class);
     assertEquals("models", models.length, models2.length);
     for (int i = 0; i < models.length; i++) {
       assertEquals("model[" + i + ']', models[i].toString(), models2[i]
@@ -276,12 +275,12 @@
 
   public void testAsymmetricSampledNormalDistributionSerialization() {
     AsymmetricSampledNormalDistribution dist = new AsymmetricSampledNormalDistribution();
-    Model<Vector>[] models = dist.sampleFromPrior(20);
+    Model<VectorWritable>[] models = dist.sampleFromPrior(20);
     GsonBuilder builder = new GsonBuilder();
     builder.registerTypeAdapter(Vector.class, new JsonVectorAdapter());
     Gson gson = builder.create();
     String jsonString = gson.toJson(models);
-    Model<Vector>[] models2 = gson.fromJson(jsonString,
+    Model<VectorWritable>[] models2 = gson.fromJson(jsonString,
         AsymmetricSampledNormalModel[].class);
     assertEquals("models", models.length, models2.length);
     for (int i = 0; i < models.length; i++) {
@@ -297,7 +296,7 @@
         .registerTypeAdapter(ModelHolder.class, new JsonModelHolderAdapter());
     Gson gson = builder.create();
     double[] d = {1.1, 2.2};
-    ModelHolder<Vector> mh = new ModelHolder<Vector>(new NormalModel(new DenseVector(d), 3.3));
+    ModelHolder<VectorWritable> mh = new ModelHolder<VectorWritable>(new NormalModel(new DenseVector(d), 3.3));
     String format = gson.toJson(mh);
     ModelHolder<Vector> mh2 = gson.<ModelHolder<Vector>>fromJson(format, ModelHolder.class);
     assertEquals("mh", mh.getModel().toString(), mh2.getModel().toString());
@@ -311,7 +310,7 @@
     Gson gson = builder.create();
     double[] d = {1.1, 2.2};
     double[] s = {3.3, 4.4};
-    ModelHolder<Vector> mh = new ModelHolder<Vector>(new AsymmetricSampledNormalModel(
+    ModelHolder<VectorWritable> mh = new ModelHolder<VectorWritable>(new AsymmetricSampledNormalModel(
         new DenseVector(d), new DenseVector(s)));
     String format = gson.toJson(mh);
     ModelHolder<Vector> mh2 = gson.<ModelHolder<Vector>>fromJson(format, ModelHolder.class);
@@ -323,7 +322,7 @@
     builder.registerTypeAdapter(DirichletState.class,
         new JsonDirichletStateAdapter());
     Gson gson = builder.create();
-    DirichletState<Vector> state = new DirichletState<Vector>(new SampledNormalDistribution(),
+    DirichletState<VectorWritable> state = new DirichletState<VectorWritable>(new SampledNormalDistribution(),
         20, 1, 1, 0);
     String format = gson.toJson(state);
     DirichletState<?> state2 = gson.fromJson(format, DirichletState.class);
@@ -346,20 +345,17 @@
     generateSamples(100, 2, 0, 0.2);
     generateSamples(100, 0, 2, 0.3);
     generateSamples(100, 2, 2, 1);
-    ClusteringTestUtils.writePointsToFile(sampleData, "input/data.txt", fs,
-        conf);
+    ClusteringTestUtils.writePointsToFile(sampleData, "input/data.txt", fs, conf);
     // Now run the driver
-    DirichletDriver
-        .runJob(
+    DirichletDriver.runJob(
             "input",
             "output",
             "org.apache.mahout.clustering.dirichlet.models.SampledNormalDistribution",
             20, 10, 1.0, 1);
     // and inspect results
-    List<List<DirichletCluster<Vector>>> clusters = new ArrayList<List<DirichletCluster<Vector>>>();
+    List<List<DirichletCluster<VectorWritable>>> clusters = new ArrayList<List<DirichletCluster<VectorWritable>>>();
     JobConf conf = new JobConf(KMeansDriver.class);
-    conf
-        .set(DirichletDriver.MODEL_FACTORY_KEY,
+    conf.set(DirichletDriver.MODEL_FACTORY_KEY,
             "org.apache.mahout.clustering.dirichlet.models.SampledNormalDistribution");
     conf.set(DirichletDriver.NUM_CLUSTERS_KEY, Integer.toString(20));
     conf.set(DirichletDriver.ALPHA_0_KEY, Double.toString(1.0));
@@ -371,12 +367,12 @@
   }
 
   private static void printResults(
-      List<List<DirichletCluster<Vector>>> clusters, int significant) {
+      List<List<DirichletCluster<VectorWritable>>> clusters, int significant) {
     int row = 0;
-    for (List<DirichletCluster<Vector>> r : clusters) {
+    for (List<DirichletCluster<VectorWritable>> r : clusters) {
       System.out.print("sample[" + row++ + "]= ");
       for (int k = 0; k < r.size(); k++) {
-        Model<Vector> model = r.get(k).getModel();
+        Model<VectorWritable> model = r.get(k).getModel();
         if (model.count() > significant) {
           int total = (int) r.get(k).getTotalCount();
           System.out.print("m" + k + '(' + total + ')' + model.toString()
@@ -403,7 +399,7 @@
             "org.apache.mahout.clustering.dirichlet.models.SampledNormalDistribution",
             20, 15, 1.0, 1);
     // and inspect results
-    List<List<DirichletCluster<Vector>>> clusters = new ArrayList<List<DirichletCluster<Vector>>>();
+    List<List<DirichletCluster<VectorWritable>>> clusters = new ArrayList<List<DirichletCluster<VectorWritable>>>();
     JobConf conf = new JobConf(KMeansDriver.class);
     conf
         .set(DirichletDriver.MODEL_FACTORY_KEY,
@@ -421,15 +417,15 @@
     generateSamples(500, 0, 0, 0.5);
     ClusteringTestUtils.writePointsToFile(sampleData, "input/data1.txt", fs,
         conf);
-    sampleData = new ArrayList<Vector>();
+    sampleData = new ArrayList<VectorWritable>();
     generateSamples(500, 2, 0, 0.2);
     ClusteringTestUtils.writePointsToFile(sampleData, "input/data2.txt", fs,
         conf);
-    sampleData = new ArrayList<Vector>();
+    sampleData = new ArrayList<VectorWritable>();
     generateSamples(500, 0, 2, 0.3);
     ClusteringTestUtils.writePointsToFile(sampleData, "input/data3.txt", fs,
         conf);
-    sampleData = new ArrayList<Vector>();
+    sampleData = new ArrayList<VectorWritable>();
     generateSamples(500, 2, 2, 1);
     ClusteringTestUtils.writePointsToFile(sampleData, "input/data4.txt", fs,
         conf);
@@ -450,7 +446,7 @@
             "org.apache.mahout.clustering.dirichlet.models.SampledNormalDistribution",
             20, 15, 1.0, 2);
     // and inspect results
-    List<List<DirichletCluster<Vector>>> clusters = new ArrayList<List<DirichletCluster<Vector>>>();
+    List<List<DirichletCluster<VectorWritable>>> clusters = new ArrayList<List<DirichletCluster<VectorWritable>>>();
     JobConf conf = new JobConf(KMeansDriver.class);
     conf
         .set(DirichletDriver.MODEL_FACTORY_KEY,
@@ -473,15 +469,15 @@
     generateSamples(500, 0, 0, 0.5, 1.0);
     ClusteringTestUtils.writePointsToFile(sampleData, "input/data1.txt", fs,
         conf);
-    sampleData = new ArrayList<Vector>();
+    sampleData = new ArrayList<VectorWritable>();
     generateSamples(500, 2, 0, 0.2);
     ClusteringTestUtils.writePointsToFile(sampleData, "input/data2.txt", fs,
         conf);
-    sampleData = new ArrayList<Vector>();
+    sampleData = new ArrayList<VectorWritable>();
     generateSamples(500, 0, 2, 0.3);
     ClusteringTestUtils.writePointsToFile(sampleData, "input/data3.txt", fs,
         conf);
-    sampleData = new ArrayList<Vector>();
+    sampleData = new ArrayList<VectorWritable>();
     generateSamples(500, 2, 2, 1);
     ClusteringTestUtils.writePointsToFile(sampleData, "input/data4.txt", fs,
         conf);
@@ -493,7 +489,7 @@
             "org.apache.mahout.clustering.dirichlet.models.AsymmetricSampledNormalDistribution",
             20, 15, 1.0, 2);
     // and inspect results
-    List<List<DirichletCluster<Vector>>> clusters = new ArrayList<List<DirichletCluster<Vector>>>();
+    List<List<DirichletCluster<VectorWritable>>> clusters = new ArrayList<List<DirichletCluster<VectorWritable>>>();
     JobConf conf = new JobConf(KMeansDriver.class);
     conf
         .set(

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=898669&r1=898668&r2=898669&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 Wed Jan 13 08:01:34 2010
@@ -31,6 +31,7 @@
 import org.apache.mahout.common.distance.DistanceMeasure;
 import org.apache.mahout.common.DummyOutputCollector;
 import org.apache.mahout.common.distance.EuclideanDistanceMeasure;
+import org.apache.mahout.math.VectorWritable;
 
 import java.io.File;
 import java.util.ArrayList;
@@ -84,7 +85,7 @@
 
   }
 
-  public static void referenceFuzzyKMeans(List<Vector> points,
+  public static void referenceFuzzyKMeans(List<VectorWritable> points,
                                           List<SoftCluster> clusterList, Map<String, String> pointClusterInfo,
                                           String distanceMeasureClass, double threshold, double m, int numIter)
       throws Exception {
@@ -100,10 +101,11 @@
     computeCluster(points, clusterList, clusterer, pointClusterInfo);
   }
 
-  public static boolean iterateReference(List<Vector> points,
+  public static boolean iterateReference(List<VectorWritable> points,
                                          List<SoftCluster> clusterList, FuzzyKMeansClusterer clusterer) {
     // for each
-    for (Vector point : points) {
+    for (VectorWritable pointWritable : points) {
+      Vector point = pointWritable.get();
       List<Double> clusterDistanceList = new ArrayList<Double>();
       for (SoftCluster cluster : clusterList) {
         clusterDistanceList.add(clusterer.getMeasure().distance(point, cluster.getCenter()));
@@ -132,11 +134,12 @@
 
   }
 
-  public static void computeCluster(List<Vector> points,
+  public static void computeCluster(List<VectorWritable> points,
                                     List<SoftCluster> clusterList, FuzzyKMeansClusterer clusterer,
                                     Map<String, String> pointClusterInfo) {
 
-    for (Vector point : points) {
+    for (VectorWritable pointWritable : points) {
+      Vector point = pointWritable.get();
       StringBuilder outputValue = new StringBuilder("[");
       List<Double> clusterDistanceList = new ArrayList<Double>();
       for (SoftCluster cluster : clusterList) {
@@ -156,15 +159,14 @@
   }
 
   public void testReferenceImplementation() throws Exception {
-    List<Vector> points = TestKmeansClustering
-        .getPoints(TestKmeansClustering.reference);
+    List<VectorWritable> points = TestKmeansClustering.getPoints(TestKmeansClustering.reference);
     for (int k = 0; k < points.size(); k++) {
       System.out.println("test k= " + k);
 
       List<SoftCluster> clusterList = new ArrayList<SoftCluster>();
       // pick k initial cluster centers at random
       for (int i = 0; i < k + 1; i++) {
-        Vector vec = tweakValue(points.get(i));
+        Vector vec = tweakValue(points.get(i).get());
         SoftCluster cluster = new SoftCluster(vec);
         // add the center so the centroid will be correct upon output
         cluster.addPoint(cluster.getCenter(), 1);
@@ -197,8 +199,7 @@
   }
 
   public void testFuzzyKMeansMRJob() throws Exception {
-    List<Vector> points = TestKmeansClustering
-        .getPoints(TestKmeansClustering.reference);
+    List<VectorWritable> points = TestKmeansClustering.getPoints(TestKmeansClustering.reference);
     File testData = new File("testdata");
     if (!testData.exists()) {
       testData.mkdir();
@@ -233,7 +234,7 @@
       SequenceFile.Writer writer = new SequenceFile.Writer(fs, conf, new Path("testdata/clusters/part-00000"),
           Text.class, SoftCluster.class);
       for (int i = 0; i < k + 1; i++) {
-        Vector vec = tweakValue(points.get(i));
+        Vector vec = tweakValue(points.get(i).get());
 
         SoftCluster cluster = new SoftCluster(vec);
         // add the center so the centroid will be correct upon output
@@ -254,7 +255,7 @@
       // now run the Job      
       FuzzyKMeansDriver.runJob("testdata/points", "testdata/clusters",
           "output", EuclideanDistanceMeasure.class.getName(), 0.001, 2, 1,
-          k + 1, 2, SparseVector.class);
+          k + 1, 2);
 
       // now compare the expected clusters with actual
       File outDir = new File("output/points");
@@ -292,8 +293,7 @@
   }
 
   public void testFuzzyKMeansMapper() throws Exception {
-    List<Vector> points = TestKmeansClustering
-        .getPoints(TestKmeansClustering.reference);
+    List<VectorWritable> points = TestKmeansClustering.getPoints(TestKmeansClustering.reference);
 
     for (int k = 0; k < points.size(); k++) {
       System.out.println("testKFuzzyKMeansMRJob k= " + k);
@@ -301,7 +301,7 @@
       List<SoftCluster> clusterList = new ArrayList<SoftCluster>();
 
       for (int i = 0; i < k + 1; i++) {
-        Vector vec = tweakValue(points.get(i));
+        Vector vec = tweakValue(points.get(i).get());
 
         SoftCluster cluster = new SoftCluster(vec, i);
         cluster.addPoint(cluster.getCenter(), 1);
@@ -319,7 +319,7 @@
       mapper.configure(conf);
       
       DummyOutputCollector<Text, FuzzyKMeansInfo> mapCollector = new DummyOutputCollector<Text, FuzzyKMeansInfo>();
-      for (Vector point : points) {
+      for (VectorWritable point : points) {
         mapper.map(new Text(), point, mapCollector, null);
       }
 
@@ -354,8 +354,7 @@
   }
 
   public void testFuzzyKMeansCombiner() throws Exception {
-    List<Vector> points = TestKmeansClustering
-        .getPoints(TestKmeansClustering.reference);
+    List<VectorWritable> points = TestKmeansClustering.getPoints(TestKmeansClustering.reference);
 
     for (int k = 0; k < points.size(); k++) {
       System.out.println("testKFuzzyKMeansMRJob k= " + k);
@@ -363,7 +362,7 @@
       List<SoftCluster> clusterList = new ArrayList<SoftCluster>();
 
       for (int i = 0; i < k + 1; i++) {
-        Vector vec = tweakValue(points.get(i));
+        Vector vec = tweakValue(points.get(i).get());
 
         SoftCluster cluster = new SoftCluster(vec, i);
         cluster.addPoint(cluster.getCenter(), 1);
@@ -381,9 +380,8 @@
       mapper.configure(conf);
       
       DummyOutputCollector<Text, FuzzyKMeansInfo> mapCollector = new DummyOutputCollector<Text, FuzzyKMeansInfo>();
-      for (Vector point : points) {
-        mapper.map(new Text(), point, mapCollector,
-            null);
+      for (VectorWritable point : points) {
+        mapper.map(new Text(), point, mapCollector, null);
       }
 
       // run combiner
@@ -409,8 +407,7 @@
   }
 
   public void testFuzzyKMeansReducer() throws Exception {
-    List<Vector> points = TestKmeansClustering
-        .getPoints(TestKmeansClustering.reference);
+    List<VectorWritable> points = TestKmeansClustering.getPoints(TestKmeansClustering.reference);
 
     for (int k = 0; k < points.size(); k++) {
       System.out.println("testKFuzzyKMeansMRJob k= " + k);
@@ -418,7 +415,7 @@
       List<SoftCluster> clusterList = new ArrayList<SoftCluster>();
 
       for (int i = 0; i < k + 1; i++) {
-        Vector vec = tweakValue(points.get(i));
+        Vector vec = tweakValue(points.get(i).get());
 
         SoftCluster cluster = new SoftCluster(vec, i);
         // cluster.addPoint(cluster.getCenter(), 1);
@@ -436,7 +433,7 @@
       mapper.configure(conf);
       
       DummyOutputCollector<Text, FuzzyKMeansInfo> mapCollector = new DummyOutputCollector<Text, FuzzyKMeansInfo>();
-      for (Vector point : points) {
+      for (VectorWritable point : points) {
         mapper.map(new Text(), point, mapCollector,
             null);
       }
@@ -470,7 +467,7 @@
       // compute the reference result after one iteration and compare
       List<SoftCluster> reference = new ArrayList<SoftCluster>();
       for (int i = 0; i < k + 1; i++) {
-        Vector vec = tweakValue(points.get(i));
+        Vector vec = tweakValue(points.get(i).get());
         reference.add(new SoftCluster(vec, i));
       }
       
@@ -492,8 +489,7 @@
   }
 
   public void testFuzzyKMeansClusterMapper() throws Exception {
-    List<Vector> points = TestKmeansClustering
-        .getPoints(TestKmeansClustering.reference);
+    List<VectorWritable> points = TestKmeansClustering.getPoints(TestKmeansClustering.reference);
 
     for (int k = 0; k < points.size(); k++) {
       System.out.println("testKFuzzyKMeansMRJob k= " + k);
@@ -501,7 +497,7 @@
       List<SoftCluster> clusterList = new ArrayList<SoftCluster>();
 
       for (int i = 0; i < k + 1; i++) {
-        Vector vec = tweakValue(points.get(i));
+        Vector vec = tweakValue(points.get(i).get());
 
         SoftCluster cluster = new SoftCluster(vec, i);
         cluster.addPoint(cluster.getCenter(), 1);
@@ -519,9 +515,8 @@
       mapper.configure(conf);
       
       DummyOutputCollector<Text, FuzzyKMeansInfo> mapCollector = new DummyOutputCollector<Text, FuzzyKMeansInfo>();
-      for (Vector point : points) {
-        mapper.map(new Text(), point, mapCollector,
-            null);
+      for (VectorWritable point : points) {
+        mapper.map(new Text(), point, mapCollector, null);
       }
       for (SoftCluster softCluster : clusterList) {
         softCluster.recomputeCenter();
@@ -568,7 +563,7 @@
       clusterMapper.config(reducerCluster);
       clusterMapper.configure(conf);
       
-      for (Vector point : points) {
+      for (VectorWritable point : points) {
         clusterMapper.map(new Text(), point, clusterMapperCollector, null);
       }
 
@@ -577,7 +572,7 @@
       // compute the reference result after one iteration and compare
       List<SoftCluster> reference = new ArrayList<SoftCluster>();
       for (int i = 0; i < k + 1; i++) {
-        Vector vec = tweakValue(points.get(i));
+        Vector vec = tweakValue(points.get(i).get());
         reference.add(new SoftCluster(vec, i));
       }
       Map<String, String> pointClusterInfo = new HashMap<String, String>();

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=898669&r1=898668&r2=898669&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 Wed Jan 13 08:01:34 2010
@@ -26,10 +26,7 @@
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.mahout.clustering.ClusteringTestUtils;
 import org.apache.mahout.clustering.canopy.CanopyDriver;
-import org.apache.mahout.math.AbstractVector;
-import org.apache.mahout.math.DenseVector;
-import org.apache.mahout.math.SparseVector;
-import org.apache.mahout.math.Vector;
+import org.apache.mahout.math.*;
 import org.apache.mahout.common.distance.DistanceMeasure;
 import org.apache.mahout.common.DummyOutputCollector;
 import org.apache.mahout.common.distance.EuclideanDistanceMeasure;
@@ -87,7 +84,7 @@
    * @param maxIter  the maximum number of iterations
    * @param convergenceDelta threshold until cluster is considered stable
    */
-  private static void referenceKmeans(List<Vector> points, List<Cluster> clusters,
+  private static void referenceKmeans(List<VectorWritable> points, List<Cluster> clusters,
                                DistanceMeasure measure, int maxIter, double convergenceDelta) {
     boolean converged = false;
     int iteration = 0;
@@ -105,10 +102,11 @@
    * @param measure  a DistanceMeasure to use
    * @param convergenceDelta threshold until cluster is considered stable
    */
-  private static boolean iterateReference(List<Vector> points, List<Cluster> clusters,
+  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 (Vector point : points) {
+    for (VectorWritable pointWritable : points) {
+      Vector point = pointWritable.get();
       Cluster closestCluster = null;
       double closestDistance = Double.MAX_VALUE;
       for (Cluster cluster : clusters) {
@@ -136,20 +134,20 @@
     return converged;
   }
 
-  public static List<Vector> getPoints(double[][] raw) {
-    List<Vector> points = new ArrayList<Vector>();
+  public static List<VectorWritable> getPoints(double[][] raw) {
+    List<VectorWritable> points = new ArrayList<VectorWritable>();
     for (int i = 0; i < raw.length; i++) {
       double[] fr = raw[i];
       Vector vec = new SparseVector(String.valueOf(i), fr.length);
       vec.assign(fr);
-      points.add(vec);
+      points.add(new VectorWritable(vec));
     }
     return points;
   }
 
   /** Story: Test the reference implementation */
   public void testReferenceImplementation() throws Exception {
-    List<Vector> points = getPoints(reference);
+    List<VectorWritable> points = getPoints(reference);
     DistanceMeasure measure = new EuclideanDistanceMeasure();
     // try all possible values of k
     for (int k = 0; k < points.size(); k++) {
@@ -157,7 +155,7 @@
       // 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);
+        Vector vec = points.get(i).get();
         clusters.add(new VisibleCluster(vec));
       }
       // iterate clusters until they converge
@@ -189,14 +187,14 @@
     conf.set(KMeansConfigKeys.CLUSTER_CONVERGENCE_KEY, "0.001");
     conf.set(KMeansConfigKeys.CLUSTER_PATH_KEY, "");
     mapper.configure(conf);
-    List<Vector> points = getPoints(reference);
+    List<VectorWritable> points = getPoints(reference);
     for (int k = 0; k < points.size(); k++) {
       // pick k initial cluster centers at random
       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), i);
+        Cluster cluster = new Cluster(points.get(i).get(), i);
         // add the center so the centroid will be correct upon output
         cluster.addPoint(cluster.getCenter());
         clusters.add(cluster);
@@ -204,7 +202,7 @@
       mapper.config(clusters);
 
       // map the data
-      for (Vector point : points) {
+      for (VectorWritable point : points) {
         mapper.map(new Text(), point, collector, null);
       }
       assertEquals("Number of map results", k + 1, collector.getData().size());
@@ -234,13 +232,13 @@
     conf.set(KMeansConfigKeys.CLUSTER_CONVERGENCE_KEY, "0.001");
     conf.set(KMeansConfigKeys.CLUSTER_PATH_KEY, "");
     mapper.configure(conf);
-    List<Vector> points = getPoints(reference);
+    List<VectorWritable> points = getPoints(reference);
     for (int k = 0; k < points.size(); k++) {
       // pick k initial cluster centers at random
       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);
+        Vector vec = points.get(i).get();
 
         Cluster cluster = new Cluster(vec, i);
         // add the center so the centroid will be correct upon output
@@ -249,7 +247,7 @@
       }
       mapper.config(clusters);
       // map the data
-      for (Vector point : points) {
+      for (VectorWritable point : points) {
         mapper.map(new Text(), point, collector,
             null);
       }
@@ -289,14 +287,14 @@
     conf.set(KMeansConfigKeys.CLUSTER_CONVERGENCE_KEY, "0.001");
     conf.set(KMeansConfigKeys.CLUSTER_PATH_KEY, "");
     mapper.configure(conf);
-    List<Vector> points = getPoints(reference);
+    List<VectorWritable> points = getPoints(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>();
       List<Cluster> clusters = new ArrayList<Cluster>();
       for (int i = 0; i < k + 1; i++) {
-        Vector vec = points.get(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());
@@ -304,7 +302,7 @@
       }
       mapper.config(clusters);
       // map the data
-      for (Vector point : points) {
+      for (VectorWritable point : points) {
         mapper.map(new Text(), point, collector,
             null);
       }
@@ -331,7 +329,7 @@
       // 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);
+        Vector vec = points.get(i).get();
         reference.add(new Cluster(vec, i));
       }
       boolean converged = iterateReference(points, reference,
@@ -368,7 +366,7 @@
 
   /** Story: User wishes to run kmeans job on reference data */
   public void testKMeansMRJob() throws Exception {
-    List<Vector> points = getPoints(reference);
+    List<VectorWritable> points = getPoints(reference);
     File testData = new File("testdata");
     if (!testData.exists()) {
       testData.mkdir();
@@ -391,7 +389,7 @@
           Text.class, Cluster.class);
 
       for (int i = 0; i < k + 1; i++) {
-        Vector vec = points.get(i);
+        Vector vec = points.get(i).get();
 
         Cluster cluster = new Cluster(vec, i);
         // add the center so the centroid will be correct upon output
@@ -401,8 +399,13 @@
       writer.close();
       // now run the Job
       HadoopUtil.overwriteOutput("output");
-      KMeansDriver.runJob("testdata/points", "testdata/clusters", "output",
-          EuclideanDistanceMeasure.class.getName(), 0.001, 10, k + 1, SparseVector.class);
+      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());
@@ -440,7 +443,7 @@
 
   /** Story: User wants to use canopy clustering to input the initial clusters for kmeans job. */
   public void textKMeansWithCanopyClusterInput() throws Exception {
-    List<Vector> points = getPoints(reference);
+    List<VectorWritable> points = getPoints(reference);
     File testData = new File("testdata");
     if (!testData.exists()) {
       testData.mkdir();
@@ -454,12 +457,16 @@
     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, SparseVector.class);
+    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, SparseVector.class);
+    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");

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestRandomSeedGenerator.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestRandomSeedGenerator.java?rev=898669&r1=898668&r2=898669&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestRandomSeedGenerator.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestRandomSeedGenerator.java Wed Jan 13 08:01:34 2010
@@ -35,6 +35,7 @@
 import org.apache.mahout.clustering.ClusteringTestUtils;
 import org.apache.mahout.math.SparseVector;
 import org.apache.mahout.math.Vector;
+import org.apache.mahout.math.VectorWritable;
 
 public class TestRandomSeedGenerator extends TestCase {
   
@@ -43,13 +44,13 @@
   
   private FileSystem fs;
   
-  private static List<Vector> getPoints(double[][] raw) {
-    List<Vector> points = new ArrayList<Vector>();
+  private static List<VectorWritable> getPoints(double[][] raw) {
+    List<VectorWritable> points = new ArrayList<VectorWritable>();
     int i = 0;
     for (double[] fr : raw) {
       Vector vec = new SparseVector(String.valueOf(i++), fr.length);
       vec.assign(fr);
-      points.add(vec);
+      points.add(new VectorWritable(vec));
     }
     return points;
   }
@@ -77,7 +78,7 @@
   
   /** Story: test random seed generation generates 4 clusters with proper ids and data */
   public void testRandomSeedGenerator() throws Exception {
-    List<Vector> points = getPoints(raw);
+    List<VectorWritable> points = getPoints(raw);
     File testData = new File("testdata");
     if (!testData.exists()) {
       testData.mkdir();

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=898669&r1=898668&r2=898669&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 Wed Jan 13 08:01:34 2010
@@ -30,6 +30,7 @@
 import org.apache.mahout.common.distance.DistanceMeasure;
 import org.apache.mahout.common.distance.ManhattanDistanceMeasure;
 import org.apache.mahout.common.RandomUtils;
+import org.apache.mahout.math.VectorWritable;
 
 class DisplayCanopy extends DisplayDirichlet {
   DisplayCanopy() {
@@ -73,7 +74,7 @@
    * @return the List<Canopy> created
    */
   static List<Canopy> populateCanopies(DistanceMeasure measure,
-      List<Vector> points, double t1, double t2) {
+      List<VectorWritable> points, double t1, double t2) {
     List<Canopy> canopies = new ArrayList<Canopy>();
     /**
      * Reference Implementation: Given a distance metric, one can create
@@ -87,13 +88,13 @@
      */
     int nextCanopyId = 0;
     while (!points.isEmpty()) {
-      Iterator<Vector> ptIter = points.iterator();
-      Vector p1 = ptIter.next();
+      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();
+        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)
@@ -109,7 +110,7 @@
   public static void main(String[] args) {
     RandomUtils.useTestSeed();
     generateSamples();
-    List<Vector> points = new ArrayList<Vector>();
+    List<VectorWritable> points = new ArrayList<VectorWritable>();
     points.addAll(sampleData);
     canopies = populateCanopies(new ManhattanDistanceMeasure(), points, t1, t2);
     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=898669&r1=898668&r2=898669&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 Wed Jan 13 08:01:34 2010
@@ -27,12 +27,12 @@
 import org.apache.mahout.math.DenseVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.common.RandomUtils;
+import org.apache.mahout.math.VectorWritable;
 
 class Display2dASNDirichlet extends DisplayDirichlet {
   Display2dASNDirichlet() {
     initialize();
-    this
-        .setTitle("Dirichlet Process Clusters - 2-d Asymmetric Sampled Normal Distribution (>"
+    this.setTitle("Dirichlet Process Clusters - 2-d Asymmetric Sampled Normal Distribution (>"
             + (int) (significance * 100) + "% of population)");
   }
 
@@ -43,10 +43,10 @@
 
     Vector dv = new DenseVector(2);
     int i = result.size() - 1;
-    for (Model<Vector>[] models : result) {
+    for (Model<VectorWritable>[] models : result) {
       g2.setStroke(new BasicStroke(i == 0 ? 3 : 1));
       g2.setColor(colors[Math.min(colors.length - 1, i--)]);
-      for (Model<Vector> m : models) {
+      for (Model<VectorWritable> m : models) {
         AsymmetricSampledNormalModel mm = (AsymmetricSampledNormalModel) m;
         dv.assign(mm.getStdDev().times(3));
         if (isSignificant(mm))

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayASNDirichlet.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayASNDirichlet.java?rev=898669&r1=898668&r2=898669&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayASNDirichlet.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayASNDirichlet.java Wed Jan 13 08:01:34 2010
@@ -27,6 +27,7 @@
 import org.apache.mahout.math.DenseVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.common.RandomUtils;
+import org.apache.mahout.math.VectorWritable;
 
 class DisplayASNDirichlet extends DisplayDirichlet {
   DisplayASNDirichlet() {
@@ -43,10 +44,10 @@
 
     Vector dv = new DenseVector(2);
     int i = result.size() - 1;
-    for (Model<Vector>[] models : result) {
+    for (Model<VectorWritable>[] models : result) {
       g2.setStroke(new BasicStroke(i == 0 ? 3 : 1));
       g2.setColor(colors[Math.min(colors.length - 1, i--)]);
-      for (Model<Vector> m : models) {
+      for (Model<VectorWritable> m : models) {
         AsymmetricSampledNormalModel mm = (AsymmetricSampledNormalModel) m;
         dv.assign(mm.getStdDev().times(3));
         if (isSignificant(mm))

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayASNOutputState.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayASNOutputState.java?rev=898669&r1=898668&r2=898669&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayASNOutputState.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayASNOutputState.java Wed Jan 13 08:01:34 2010
@@ -35,6 +35,7 @@
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.common.RandomUtils;
 import org.apache.mahout.common.FileLineIterable;
+import org.apache.mahout.math.VectorWritable;
 
 class DisplayASNOutputState extends DisplayDirichlet {
   DisplayASNOutputState() {
@@ -50,10 +51,10 @@
 
     Vector dv = new DenseVector(2);
     int i = result.size() - 1;
-    for (Model<Vector>[] models : result) {
+    for (Model<VectorWritable>[] models : result) {
       g2.setStroke(new BasicStroke(i == 0 ? 3 : 1));
       g2.setColor(colors[Math.min(colors.length - 1, i--)]);
-      for (Model<Vector> m : models) {
+      for (Model<VectorWritable> m : models) {
         AsymmetricSampledNormalModel mm = (AsymmetricSampledNormalModel) m;
         dv.set(0, mm.getStdDev().get(0) * 3);
         dv.set(1, mm.getStdDev().get(1) * 3);
@@ -72,10 +73,10 @@
    * @throws IOException
    *             if there is an error
    */
-  public static List<Vector> readFile(String fileName) throws IOException {
-    List<Vector> results = new ArrayList<Vector>();
+  public static List<VectorWritable> readFile(String fileName) throws IOException {
+    List<VectorWritable> results = new ArrayList<VectorWritable>();
     for (String line : new FileLineIterable(new File(fileName))) {
-      results.add(AbstractVector.decodeVector(line));
+      results.add(new VectorWritable(AbstractVector.decodeVector(line)));
     }
     return results;
   }
@@ -87,7 +88,7 @@
   }
 
   private static void getResults() throws IOException {
-    result = new ArrayList<Model<Vector>[]>();
+    result = new ArrayList<Model<VectorWritable>[]>();
     JobConf conf = new JobConf(KMeansDriver.class);
     conf
         .set(DirichletDriver.MODEL_FACTORY_KEY,
@@ -97,8 +98,7 @@
     File f = new File("output");
     for (File g : f.listFiles()) {
       conf.set(DirichletDriver.STATE_IN_KEY, g.getCanonicalPath());
-      DirichletState<Vector> dirichletState = DirichletMapper
-          .getDirichletState(conf);
+      DirichletState<VectorWritable> dirichletState = DirichletMapper.getDirichletState(conf);
       result.add(dirichletState.getModels());
     }
   }

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=898669&r1=898668&r2=898669&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 Wed Jan 13 08:01:34 2010
@@ -37,6 +37,7 @@
 import org.apache.mahout.math.TimesFunction;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.common.RandomUtils;
+import org.apache.mahout.math.VectorWritable;
 
 public class DisplayDirichlet extends Frame {
 
@@ -46,9 +47,9 @@
 
   protected static final int size = 8; // screen size in inches
 
-  protected static final List<Vector> sampleData = new ArrayList<Vector>();
+  protected static final List<VectorWritable> sampleData = new ArrayList<VectorWritable>();
 
-  protected static List<Model<Vector>[]> result;
+  protected static List<Model<VectorWritable>[]> result;
 
   protected static final double significance = 0.05;
 
@@ -133,8 +134,8 @@
     // plot the sample data
     g2.setColor(Color.DARK_GRAY);
     dv.assign(0.03);
-    for (Vector v : sampleData)
-      plotRectangle(g2, v, dv);
+    for (VectorWritable v : sampleData)
+      plotRectangle(g2, v.get(), dv);
   }
 
   /**
@@ -172,12 +173,12 @@
             * ds));
   }
 
-  private static void printModels(List<Model<Vector>[]> results, int significant) {
+  private static void printModels(List<Model<VectorWritable>[]> results, int significant) {
     int row = 0;
-    for (Model<Vector>[] r : results) {
+    for (Model<VectorWritable>[] r : results) {
       System.out.print("sample[" + row++ + "]= ");
       for (int k = 0; k < r.length; k++) {
-        Model<Vector> model = r[k];
+        Model<VectorWritable> model = r[k];
         if (model.count() > significant) {
           System.out.print("m" + k + model.toString() + ", ");
         }
@@ -212,9 +213,9 @@
     System.out.println("Generating " + num + " samples m=[" + mx + ", " + my
         + "] sd=" + sd);
     for (int i = 0; i < num; i++)
-      sampleData.add(new DenseVector(new double[] {
+      sampleData.add(new VectorWritable(new DenseVector(new double[] {
           UncommonDistributions.rNorm(mx, sd),
-          UncommonDistributions.rNorm(my, sd) }));
+          UncommonDistributions.rNorm(my, sd) })));
   }
 
   /**
@@ -232,19 +233,19 @@
     System.out.println("Generating " + num + " samples m=[" + mx + ", " + my
         + "] sd=[" + sdx + ", " + sdy + ']');
     for (int i = 0; i < num; i++)
-      sampleData.add(new DenseVector(new double[] {
+      sampleData.add(new VectorWritable(new DenseVector(new double[] {
           UncommonDistributions.rNorm(mx, sdx),
-          UncommonDistributions.rNorm(my, sdy) }));
+          UncommonDistributions.rNorm(my, sdy) })));
   }
 
-  public static void generateResults(ModelDistribution<Vector> modelDist) {
-    DirichletClusterer<Vector> dc = new DirichletClusterer<Vector>(sampleData,
+  public static void generateResults(ModelDistribution<VectorWritable> modelDist) {
+    DirichletClusterer<VectorWritable> dc = new DirichletClusterer<VectorWritable>(sampleData,
         modelDist, 1.0, 10, 2, 2);
     result = dc.cluster(20);
     printModels(result, 5);
   }
 
-  public static boolean isSignificant(Model<Vector> model) {
+  public static boolean isSignificant(Model<VectorWritable> model) {
     return (((double) model.count() / sampleData.size()) > significance);
   }
 

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayNDirichlet.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayNDirichlet.java?rev=898669&r1=898668&r2=898669&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayNDirichlet.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayNDirichlet.java Wed Jan 13 08:01:34 2010
@@ -27,6 +27,7 @@
 import org.apache.mahout.math.DenseVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.common.RandomUtils;
+import org.apache.mahout.math.VectorWritable;
 
 class DisplayNDirichlet extends DisplayDirichlet {
   DisplayNDirichlet() {
@@ -42,10 +43,10 @@
 
     Vector dv = new DenseVector(2);
     int i = result.size() - 1;
-    for (Model<Vector>[] models : result) {
+    for (Model<VectorWritable>[] models : result) {
       g2.setStroke(new BasicStroke(i == 0 ? 3 : 1));
       g2.setColor(colors[Math.min(colors.length - 1, i--)]);
-      for (Model<Vector> m : models) {
+      for (Model<VectorWritable> m : models) {
         NormalModel mm = (NormalModel) m;
         dv.assign(mm.getStdDev() * 3);
         if (isSignificant(mm))

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayOutputState.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayOutputState.java?rev=898669&r1=898668&r2=898669&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayOutputState.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayOutputState.java Wed Jan 13 08:01:34 2010
@@ -35,6 +35,7 @@
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.common.RandomUtils;
 import org.apache.mahout.common.FileLineIterable;
+import org.apache.mahout.math.VectorWritable;
 
 class DisplayOutputState extends DisplayDirichlet {
   DisplayOutputState() {
@@ -50,10 +51,10 @@
 
     Vector dv = new DenseVector(2);
     int i = result.size() - 1;
-    for (Model<Vector>[] models : result) {
+    for (Model<VectorWritable>[] models : result) {
       g2.setStroke(new BasicStroke(i == 0 ? 3 : 1));
       g2.setColor(colors[Math.min(colors.length - 1, i--)]);
-      for (Model<Vector> m : models) {
+      for (Model<VectorWritable> m : models) {
         NormalModel mm = (NormalModel) m;
         dv.assign(mm.getStdDev() * 3);
         if (isSignificant(mm))
@@ -71,10 +72,10 @@
    * @throws IOException
    *             if there is an error
    */
-  public static List<Vector> readFile(String fileName) throws IOException {
-    List<Vector> results = new ArrayList<Vector>();
+  public static List<VectorWritable> readFile(String fileName) throws IOException {
+    List<VectorWritable> results = new ArrayList<VectorWritable>();
     for (String line : new FileLineIterable(new File(fileName))) {
-      results.add(AbstractVector.decodeVector(line));
+      results.add(new VectorWritable(AbstractVector.decodeVector(line)));
     }
     return results;
   }
@@ -86,7 +87,7 @@
   }
 
   private static void getResults() throws IOException {
-    result = new ArrayList<Model<Vector>[]>();
+    result = new ArrayList<Model<VectorWritable>[]>();
     JobConf conf = new JobConf(KMeansDriver.class);
     conf.set(DirichletDriver.MODEL_FACTORY_KEY,
         "org.apache.mahout.clustering.dirichlet.models.SampledNormalDistribution");
@@ -95,8 +96,7 @@
     File f = new File("output");
     for (File g : f.listFiles()) {
       conf.set(DirichletDriver.STATE_IN_KEY, g.getCanonicalPath());
-      DirichletState<Vector> dirichletState = DirichletMapper
-          .getDirichletState(conf);
+      DirichletState<VectorWritable> dirichletState = DirichletMapper.getDirichletState(conf);
       result.add(dirichletState.getModels());
     }
   }

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplaySNDirichlet.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplaySNDirichlet.java?rev=898669&r1=898668&r2=898669&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplaySNDirichlet.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplaySNDirichlet.java Wed Jan 13 08:01:34 2010
@@ -27,6 +27,7 @@
 import org.apache.mahout.math.DenseVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.common.RandomUtils;
+import org.apache.mahout.math.VectorWritable;
 
 class DisplaySNDirichlet extends DisplayDirichlet {
   DisplaySNDirichlet() {
@@ -42,10 +43,10 @@
 
     Vector dv = new DenseVector(2);
     int i = result.size() - 1;
-    for (Model<Vector>[] models : result) {
+    for (Model<VectorWritable>[] models : result) {
       g2.setStroke(new BasicStroke(i == 0 ? 3 : 1));
       g2.setColor(colors[Math.min(colors.length - 1, i--)]);
-      for (Model<Vector> m : models) {
+      for (Model<VectorWritable> m : models) {
         NormalModel mm = (NormalModel) m;
         dv.assign(mm.getStdDev() * 3);
         if (isSignificant(mm))

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=898669&r1=898668&r2=898669&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 Wed Jan 13 08:01:34 2010
@@ -32,6 +32,7 @@
 import org.apache.mahout.common.distance.DistanceMeasure;
 import org.apache.mahout.common.distance.ManhattanDistanceMeasure;
 import org.apache.mahout.common.RandomUtils;
+import org.apache.mahout.math.VectorWritable;
 
 class DisplayFuzzyKMeans extends DisplayDirichlet {
   DisplayFuzzyKMeans() {
@@ -63,7 +64,7 @@
     }
   }
 
-  public static void referenceFuzzyKMeans(List<Vector> points,
+  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;
@@ -84,22 +85,21 @@
    * 
    * @param points the List<Vector> having the input points
    * @param clusterList the List<Cluster> clusters
-   * @param measure a DistanceMeasure to use
    * @return
    */
-  public static boolean iterateReference(List<Vector> points,
+  public static boolean iterateReference(List<VectorWritable> points,
       List<SoftCluster> clusterList, FuzzyKMeansClusterer clusterer) {
     // for each
-    for (Vector point : points) {
+    for (VectorWritable point : points) {
       List<Double> clusterDistanceList = new ArrayList<Double>();
       for (SoftCluster cluster : clusterList) {
-        clusterDistanceList.add(clusterer.getMeasure().distance(point, cluster.getCenter()));
+        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,
+        clusterList.get(i).addPoint(point.get(),
             Math.pow(probWeight, clusterer.getM()));
       }
     }
@@ -130,7 +130,7 @@
    * @return the List<Canopy> created
    */
   static List<Canopy> populateCanopies(DistanceMeasure measure,
-      List<Vector> points, double t1, double t2) {
+      List<VectorWritable> points, double t1, double t2) {
     List<Canopy> canopies = new ArrayList<Canopy>();
     /**
      * Reference Implementation: Given a distance metric, one can create
@@ -144,13 +144,13 @@
      */
     int nextCanopyId = 0;
     while (!points.isEmpty()) {
-      Iterator<Vector> ptIter = points.iterator();
-      Vector p1 = ptIter.next();
+      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();
+        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)
@@ -166,7 +166,7 @@
   public static void main(String[] args) {
     RandomUtils.useTestSeed();
     generateSamples();
-    List<Vector> points = new ArrayList<Vector>();
+    List<VectorWritable> points = new ArrayList<VectorWritable>();
     points.addAll(sampleData);
     List<Canopy> canopies = populateCanopies(new ManhattanDistanceMeasure(), points, t1, t2);
     DistanceMeasure measure = new ManhattanDistanceMeasure();

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=898669&r1=898668&r2=898669&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 Wed Jan 13 08:01:34 2010
@@ -31,6 +31,7 @@
 import org.apache.mahout.common.distance.DistanceMeasure;
 import org.apache.mahout.common.distance.ManhattanDistanceMeasure;
 import org.apache.mahout.common.RandomUtils;
+import org.apache.mahout.math.VectorWritable;
 
 class DisplayKMeans extends DisplayDirichlet {
   DisplayKMeans() {
@@ -72,7 +73,7 @@
    * @param measure the DistanceMeasure to use
    * @param maxIter the maximum number of iterations
    */
-  private static void referenceKmeans(List<Vector> points,
+  private static void referenceKmeans(List<VectorWritable> points,
       List<List<Cluster>> clusters, DistanceMeasure measure, int maxIter) {
     boolean converged = false;
     int iteration = 0;
@@ -95,20 +96,20 @@
    * @param measure a DistanceMeasure to use
    * @return
    */
-  private static boolean iterateReference(List<Vector> points,
+  private static boolean iterateReference(List<VectorWritable> points,
       List<Cluster> clusters, DistanceMeasure measure) {
     // iterate through all points, assigning each to the nearest cluster
-    for (Vector point : points) {
+    for (VectorWritable point : points) {
       Cluster closestCluster = null;
       double closestDistance = Double.MAX_VALUE;
       for (Cluster cluster : clusters) {
-        double distance = measure.distance(cluster.getCenter(), point);
+        double distance = measure.distance(cluster.getCenter(), point.get());
         if (closestCluster == null || closestDistance > distance) {
           closestCluster = cluster;
           closestDistance = distance;
         }
       }
-      closestCluster.addPoint(point);
+      closestCluster.addPoint(point.get());
     }
     // test for convergence
     boolean converged = true;
@@ -137,7 +138,7 @@
    * @return the List<Canopy> created
    */
   static List<Canopy> populateCanopies(DistanceMeasure measure,
-      List<Vector> points, double t1, double t2) {
+      List<VectorWritable> points, double t1, double t2) {
     List<Canopy> canopies = new ArrayList<Canopy>();
     /**
      * Reference Implementation: Given a distance metric, one can create
@@ -151,13 +152,13 @@
      */
     int nextCanopyId = 0;
     while (!points.isEmpty()) {
-      Iterator<Vector> ptIter = points.iterator();
-      Vector p1 = ptIter.next();
+      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();
+        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)
@@ -173,7 +174,7 @@
   public static void main(String[] args) {
     RandomUtils.useTestSeed();
     generateSamples();
-    List<Vector> points = new ArrayList<Vector>();
+    List<VectorWritable> points = new ArrayList<VectorWritable>();
     points.addAll(sampleData);
     List<Canopy> canopies = populateCanopies(new ManhattanDistanceMeasure(), points, t1, t2);
     DistanceMeasure measure = new ManhattanDistanceMeasure();