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

svn commit: r649728 - in /lucene/mahout/trunk/src: main/java/org/apache/mahout/clustering/canopy/ main/java/org/apache/mahout/clustering/kmeans/ main/java/org/apache/mahout/utils/ test/java/org/apache/mahout/clustering/canopy/ test/java/org/apache/maho...

Author: jeastman
Date: Fri Apr 18 17:10:38 2008
New Revision: 649728

URL: http://svn.apache.org/viewvc?rev=649728&view=rev
Log:
Committed Isabel's vectorClustering.txt patch to replace Float[] with Vectors. All unit tests run.

Modified:
    lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/Canopy.java
    lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyClusteringJob.java
    lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyCombiner.java
    lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyDriver.java
    lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyMapper.java
    lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyReducer.java
    lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/ClusterDriver.java
    lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/ClusterMapper.java
    lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/kmeans/Cluster.java
    lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/kmeans/KMeansMapper.java
    lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/kmeans/KMeansReducer.java
    lucene/mahout/trunk/src/main/java/org/apache/mahout/utils/Point.java
    lucene/mahout/trunk/src/test/java/org/apache/mahout/clustering/canopy/TestCanopyCreation.java
    lucene/mahout/trunk/src/test/java/org/apache/mahout/clustering/canopy/VisibleCanopy.java
    lucene/mahout/trunk/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java
    lucene/mahout/trunk/src/test/java/org/apache/mahout/clustering/kmeans/VisibleCluster.java
    lucene/mahout/trunk/src/test/java/org/apache/mahout/utils/TestPoint.java

Modified: lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/Canopy.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/Canopy.java?rev=649728&r1=649727&r2=649728&view=diff
==============================================================================
--- lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/Canopy.java (original)
+++ lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/Canopy.java Fri Apr 18 17:10:38 2008
@@ -19,6 +19,8 @@
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.mahout.matrix.SparseVector;
+import org.apache.mahout.matrix.Vector;
 import org.apache.mahout.utils.DistanceMeasure;
 import org.apache.mahout.utils.Point;
 
@@ -46,10 +48,10 @@
   private static int nextCanopyId = 0;
 
   // the T1 distance threshold
-  private static float t1;
+  private static double t1;
 
   // the T2 distance threshold
-  private static float t2;
+  private static double t2;
 
   // the distance measure
   private static DistanceMeasure measure;
@@ -58,38 +60,38 @@
   private int canopyId;
 
   // the current center
-  private Float[] center = new Float[0];
+  private Vector center = new SparseVector(0);
 
   // the number of points in the canopy
   private int numPoints = 0;
 
   // the total of all points added to the canopy
-  private Float[] pointTotal = null;
+  private Vector pointTotal = null;
 
   /**
    * Create a new Canopy containing the given point
    *
-   * @param point a Float[]
+   * @param point a point in vector space
    */
-  public Canopy(Float[] point) {
+  public Canopy(Vector point) {
     super();
     this.canopyId = nextCanopyId++;
     this.center = point;
-    this.pointTotal = point.clone();
+    this.pointTotal = point.copy();
     this.numPoints = 1;
   }
 
   /**
    * Create a new Canopy containing the given point and canopyId
    *
-   * @param point    a Float[]
+   * @param point    a point in vector space
    * @param canopyId an int identifying the canopy local to this process only
    */
-  public Canopy(Float[] point, int canopyId) {
+  public Canopy(Vector point, int canopyId) {
     super();
     this.canopyId = canopyId;
     this.center = point;
-    this.pointTotal = point.clone();
+    this.pointTotal = point.copy();
     this.numPoints = 1;
   }
 
@@ -108,8 +110,8 @@
       throw new RuntimeException(e);
     }
     nextCanopyId = 0;
-    t1 = new Float(job.get(T1_KEY));
-    t2 = new Float(job.get(T2_KEY));
+    t1 = new Double(job.get(T1_KEY));
+    t2 = new Double(job.get(T2_KEY));
   }
 
   /**
@@ -119,7 +121,7 @@
    * @param aT1
    * @param aT2
    */
-  public static void config(DistanceMeasure aMeasure, float aT1, float aT2) {
+  public static void config(DistanceMeasure aMeasure, double aT1, double aT2) {
     nextCanopyId = 0;
     measure = aMeasure;
     t1 = aT1;
@@ -134,13 +136,13 @@
    * <p/>
    * This method is used by the CanopyReducer.
    *
-   * @param point    the Float[] defining the point to be added
+   * @param point    the point to be added
    * @param canopies the List<Canopy> to be appended
    */
-  public static void addPointToCanopies(Float[] point, List<Canopy> canopies) {
+  public static void addPointToCanopies(Vector point, List<Canopy> canopies) {
     boolean pointStronglyBound = false;
     for (Canopy canopy : canopies) {
-      float dist = measure.distance(canopy.getCenter(), point);
+      double dist = measure.distance(canopy.getCenter(), point);
       if (dist < t1)
         canopy.addPoint(point);
       pointStronglyBound = pointStronglyBound | (dist < t2);
@@ -154,15 +156,15 @@
    * and to emit the point and its covering canopies to the output. The
    * CanopyCombiner will then sum the canopy points and produce the centroids.
    *
-   * @param point     the Float[] defining the point to be added
+   * @param point     the point to be added
    * @param canopies  the List<Canopy> to be appended
    * @param collector an OutputCollector in which to emit the point
    */
-  public static void emitPointToNewCanopies(Float[] point,
+  public static void emitPointToNewCanopies(Vector point,
                                             List<Canopy> canopies, OutputCollector<Text, Text> collector) throws IOException {
     boolean pointStronglyBound = false;
     for (Canopy canopy : canopies) {
-      float dist = measure.distance(canopy.getCenter(), point);
+      double dist = measure.distance(canopy.getCenter(), point);
       if (dist < t1)
         canopy.emitPoint(point, collector);
       pointStronglyBound = pointStronglyBound | (dist < t2);
@@ -180,20 +182,20 @@
    * point is not covered by any canopies (due to canopy centroid clustering),
    * emit the point to the closest covering canopy.
    *
-   * @param point     the Float[] defining the point to be added
+   * @param point     the point to be added
    * @param canopies  the List<Canopy> to be appended
    * @param writable  the original Writable from the input, may include arbitrary
    *                  payload information after the point [...]<payload>
    * @param collector an OutputCollector in which to emit the point
    */
-  public static void emitPointToExistingCanopies(Float[] point,
+  public static void emitPointToExistingCanopies(Vector point,
                                                  List<Canopy> canopies, Text writable, OutputCollector<Text, Text> collector)
           throws IOException {
-    float minDist = Float.MAX_VALUE;
+    double minDist = Double.MAX_VALUE;
     Canopy closest = null;
     boolean isCovered = false;
     for (Canopy canopy : canopies) {
-      float dist = measure.distance(canopy.getCenter(), point);
+      double dist = measure.distance(canopy.getCenter(), point);
       if (dist < t1) {
         isCovered = true;
         collector.collect(new Text(Canopy.formatCanopy(canopy)), writable);
@@ -231,7 +233,7 @@
     String centroid = formattedString.substring(beginIndex);
     if (id.startsWith("C")) {
       int canopyId = new Integer(formattedString.substring(1, beginIndex - 2));
-      Float[] canopyCentroid = Point.decodePoint(centroid);
+      Vector canopyCentroid = Point.decodePoint(centroid);
       return new Canopy(canopyCentroid, canopyId);
     }
     return null;
@@ -240,32 +242,27 @@
   /**
    * Add a point to the canopy
    *
-   * @param point a Float[]
+   * @param point some point to add
    */
-  public void addPoint(Float[] point) {
+  public void addPoint(Vector point) {
     numPoints++;
-    for (int i = 0; i < point.length; i++)
-      pointTotal[i] = new Float(point[i] + pointTotal[i]);
+    for (int i = 0; i < point.cardinality(); i++)
+      pointTotal.set(i, new Double(point.get(i) + pointTotal.get(i)));
   }
 
   /**
    * Emit the point to the collector, keyed by the canopy's formatted
    * representation
    *
-   * @param point a Float[]
+   * @param point a point to emit.
    */
-  public void emitPoint(Float[] point, OutputCollector<Text, Text> collector)
+  public void emitPoint(Vector point, OutputCollector<Text, Text> collector)
           throws IOException {
     collector.collect(new Text(formatCanopy(this)), new Text(Point.ptOut("",
             point)));
   }
 
-  /**
-   * Return a printable representation of this object, using the user supplied
-   * identifier
-   *
-   * @return
-   */
+  @Override
   public String toString() {
     return getIdentifier() + " - " + Point.ptOut("", getCenter());
   }
@@ -281,16 +278,16 @@
   /**
    * Return the center point
    *
-   * @return a Float[]
+   * @return the center of the Canopy
    */
-  public Float[] getCenter() {
+  public Vector getCenter() {
     return center;
   }
 
   /**
    * Return the number of points in the Canopy
    *
-   * @return
+   * @return the number of points in the canopy.
    */
   public int getNumPoints() {
     return numPoints;
@@ -299,22 +296,22 @@
   /**
    * Compute the centroid by averaging the pointTotals
    *
-   * @return a Float[] which is the new centroid
+   * @return a point which is the new centroid
    */
-  public Float[] computeCentroid() {
-    Float[] result = new Float[pointTotal.length];
-    for (int i = 0; i < pointTotal.length; i++)
-      result[i] = new Float(pointTotal[i] / numPoints);
+  public Vector computeCentroid() {
+    Vector result = new SparseVector(pointTotal.cardinality());
+    for (int i = 0; i < pointTotal.cardinality(); i++)
+      result.set(i, new Double(pointTotal.get(i) / numPoints));
     return result;
   }
 
   /**
    * Return if the point is covered by this canopy
    *
-   * @param point a Float[] point
+   * @param point a point
    * @return if the point is covered
    */
-  public boolean covers(Float[] point) {
+  public boolean covers(Vector point) {
     return measure.distance(center, point) < t1;
   }
 }

Modified: lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyClusteringJob.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyClusteringJob.java?rev=649728&r1=649727&r2=649728&view=diff
==============================================================================
--- lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyClusteringJob.java (original)
+++ lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyClusteringJob.java Fri Apr 18 17:10:38 2008
@@ -27,8 +27,8 @@
     String input = args[0];
     String output = args[1];
     String measureClassName = args[2];
-    float t1 = new Float(args[3]);
-    float t2 = new Float(args[4]);
+    double t1 = new Double(args[3]);
+    double t2 = new Double(args[4]);
     runJob(input, output, measureClassName, t1, t2);
   }
 
@@ -42,7 +42,7 @@
    * @param t2               the T2 distance threshold
    */
   public static void runJob(String input, String output,
-                            String measureClassName, float t1, float t2) {
+                            String measureClassName, double t1, double t2) {
     CanopyDriver.runJob(input, output + "/canopies", measureClassName, t1, t2);
     ClusterDriver.runJob(input, output + "/canopies", output, measureClassName, t1, t2);
   }

Modified: lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyCombiner.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyCombiner.java?rev=649728&r1=649727&r2=649728&view=diff
==============================================================================
--- lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyCombiner.java (original)
+++ lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyCombiner.java Fri Apr 18 17:10:38 2008
@@ -23,6 +23,7 @@
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reporter;
+import org.apache.mahout.matrix.Vector;
 import org.apache.mahout.utils.Point;
 
 import java.io.IOException;
@@ -33,11 +34,11 @@
   public void reduce(Text key, Iterator<Text> values,
                      OutputCollector<Text, Text> output, Reporter reporter) throws IOException {
     Writable value = (Writable) values.next();
-    Float[] center = Point.decodePoint(value.toString());
+    Vector center = Point.decodePoint(value.toString());
     Canopy canopy = new Canopy(center);
     while (values.hasNext()) {
       value = (Writable) values.next();
-      Float[] point = Point.decodePoint(value.toString());
+      Vector point = Point.decodePoint(value.toString());
       canopy.addPoint(point);
     }
     output.collect(new Text("centroid"), new Text(Point.formatPoint(canopy

Modified: lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyDriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyDriver.java?rev=649728&r1=649727&r2=649728&view=diff
==============================================================================
--- lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyDriver.java (original)
+++ lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyDriver.java Fri Apr 18 17:10:38 2008
@@ -29,8 +29,8 @@
     String input = args[0];
     String output = args[1];
     String measureClassName = args[2];
-    float t1 = new Float(args[3]);
-    float t2 = new Float(args[4]);
+    double t1 = new Double(args[3]);
+    double t2 = new Double(args[4]);
     runJob(input, output, measureClassName, t1, t2);
   }
 
@@ -44,7 +44,7 @@
    * @param t2               the T2 distance threshold
    */
   public static void runJob(String input, String output,
-                            String measureClassName, float t1, float t2) {
+                            String measureClassName, double t1, double t2) {
     JobClient client = new JobClient();
     JobConf conf = new JobConf(
             org.apache.mahout.clustering.canopy.CanopyDriver.class);

Modified: lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyMapper.java?rev=649728&r1=649727&r2=649728&view=diff
==============================================================================
--- lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyMapper.java (original)
+++ lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyMapper.java Fri Apr 18 17:10:38 2008
@@ -23,6 +23,7 @@
 import org.apache.hadoop.mapred.Mapper;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reporter;
+import org.apache.mahout.matrix.Vector;
 import org.apache.mahout.utils.Point;
 
 import java.io.IOException;
@@ -44,7 +45,7 @@
    */
   public void map(WritableComparable key, Text values,
                   OutputCollector<Text, Text> output, Reporter reporter) throws IOException {
-    Float[] point = Point.decodePoint(values.toString());
+    Vector point = Point.decodePoint(values.toString());
     Canopy.emitPointToNewCanopies(point, canopies, output);
   }
 

Modified: lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyReducer.java?rev=649728&r1=649727&r2=649728&view=diff
==============================================================================
--- lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyReducer.java (original)
+++ lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/CanopyReducer.java Fri Apr 18 17:10:38 2008
@@ -22,6 +22,7 @@
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reporter;
+import org.apache.mahout.matrix.Vector;
 import org.apache.mahout.utils.Point;
 
 import java.io.IOException;
@@ -45,7 +46,7 @@
                      OutputCollector<Text, Text> output, Reporter reporter) throws IOException {
     while (values.hasNext()) {
       Text value = (Text) values.next();
-      Float[] point = Point.decodePoint(value.toString());
+      Vector point = Point.decodePoint(value.toString());
       Canopy.addPointToCanopies(point, canopies);
     }
     for (Canopy canopy : canopies)

Modified: lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/ClusterDriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/ClusterDriver.java?rev=649728&r1=649727&r2=649728&view=diff
==============================================================================
--- lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/ClusterDriver.java (original)
+++ lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/ClusterDriver.java Fri Apr 18 17:10:38 2008
@@ -30,8 +30,8 @@
     String canopies = args[1];
     String output = args[2];
     String measureClassName = args[3];
-    float t1 = new Float(args[4]);
-    float t2 = new Float(args[5]);
+    double t1 = new Double(args[4]);
+    double t2 = new Double(args[5]);
     runJob(points, canopies, output, measureClassName, t1, t2);
   }
 
@@ -46,7 +46,7 @@
    * @param t2               the T2 distance threshold
    */
   public static void runJob(String points, String canopies, String output,
-                            String measureClassName, float t1, float t2) {
+                            String measureClassName, double t1, double t2) {
     JobClient client = new JobClient();
     JobConf conf = new JobConf(
             org.apache.mahout.clustering.canopy.ClusterDriver.class);

Modified: lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/ClusterMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/ClusterMapper.java?rev=649728&r1=649727&r2=649728&view=diff
==============================================================================
--- lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/ClusterMapper.java (original)
+++ lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/canopy/ClusterMapper.java Fri Apr 18 17:10:38 2008
@@ -26,6 +26,7 @@
 import org.apache.hadoop.mapred.Mapper;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reporter;
+import org.apache.mahout.matrix.Vector;
 import org.apache.mahout.utils.Point;
 
 import java.io.IOException;
@@ -39,7 +40,7 @@
 
   public void map(WritableComparable key, Text values,
                   OutputCollector<Text, Text> output, Reporter reporter) throws IOException {
-    Float[] point = Point.decodePoint(values.toString());
+    Vector point = Point.decodePoint(values.toString());
     Canopy.emitPointToExistingCanopies(point, canopies, values, output);
   }
 

Modified: lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/kmeans/Cluster.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/kmeans/Cluster.java?rev=649728&r1=649727&r2=649728&view=diff
==============================================================================
--- lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/kmeans/Cluster.java (original)
+++ lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/kmeans/Cluster.java Fri Apr 18 17:10:38 2008
@@ -19,6 +19,8 @@
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.mahout.matrix.SparseVector;
+import org.apache.mahout.matrix.Vector;
 import org.apache.mahout.utils.DistanceMeasure;
 import org.apache.mahout.utils.Point;
 
@@ -39,23 +41,23 @@
   private int clusterId;
 
   // the current center
-  private Float[] center = new Float[0];
+  private Vector center = new SparseVector(0);
 
   // the current centroid is lazy evaluated and may be null
-  private Float[] centroid = null;
+  private Vector centroid = null;
 
   // the number of points in the cluster
   private int numPoints = 0;
 
   // the total of all points added to the cluster
-  private Float[] pointTotal = null;
+  private Vector pointTotal = null;
 
   // has the centroid converged with the center?
   private boolean converged = false;
 
   private static DistanceMeasure measure;
 
-  private static float convergenceDelta = 0;
+  private static double convergenceDelta = 0;
 
   /**
    * Format the cluster for output
@@ -80,7 +82,7 @@
     String center = formattedString.substring(beginIndex);
     if (id.startsWith("C") || id.startsWith("V")) {
       int clusterId = new Integer(formattedString.substring(1, beginIndex - 2));
-      Float[] clusterCenter = Point.decodePoint(center);
+      Vector clusterCenter = Point.decodePoint(center);
       Cluster cluster = new Cluster(clusterCenter, clusterId);
       cluster.converged = id.startsWith("V");
       return cluster;
@@ -99,7 +101,7 @@
       Class cl = ccl.loadClass(job.get(DISTANCE_MEASURE_KEY));
       measure = (DistanceMeasure) cl.newInstance();
       measure.configure(job);
-      convergenceDelta = new Float(job.get(CLUSTER_CONVERGENCE_KEY));
+      convergenceDelta = new Double(job.get(CLUSTER_CONVERGENCE_KEY));
       nextClusterId = 0;
     } catch (Exception e) {
       throw new RuntimeException(e);
@@ -110,9 +112,9 @@
    * Configure the distance measure directly. Used by unit tests.
    *
    * @param aMeasure          the DistanceMeasure
-   * @param aConvergenceDelta the float delta value used to define convergence
+   * @param aConvergenceDelta the delta value used to define convergence
    */
-  public static void config(DistanceMeasure aMeasure, float aConvergenceDelta) {
+  public static void config(DistanceMeasure aMeasure, double aConvergenceDelta) {
     measure = aMeasure;
     convergenceDelta = aConvergenceDelta;
     nextClusterId = 0;
@@ -121,20 +123,20 @@
   /**
    * Emit the point to the nearest cluster center
    *
-   * @param point    a Float[] representing the point
+   * @param point    a point
    * @param clusters a List<Cluster> to test
    * @param values   a Writable containing the input point and possible other
    *                 values of interest (payload)
    * @param output   the OutputCollector to emit into
    * @throws IOException
    */
-  public static void emitPointToNearestCluster(Float[] point,
+  public static void emitPointToNearestCluster(Vector point,
                                                List<Cluster> clusters, Text values, OutputCollector<Text, Text> output)
           throws IOException {
     Cluster nearestCluster = null;
-    float nearestDistance = Float.MAX_VALUE;
+    double nearestDistance = Double.MAX_VALUE;
     for (Cluster cluster : clusters) {
-      float distance = measure.distance(point, cluster.getCenter());
+      double distance = measure.distance(point, cluster.getCenter());
       if (nearestCluster == null || distance < nearestDistance) {
         nearestCluster = cluster;
         nearestDistance = distance;
@@ -146,16 +148,14 @@
   /**
    * Compute the centroid by averaging the pointTotals
    *
-   * @return a Float[] which is the new centroid
+   * @return the new centroid
    */
-  private Float[] computeCentroid() {
+  private Vector computeCentroid() {
     if (numPoints == 0)
       return pointTotal;
     else if (centroid == null) {
       // lazy compute new centroid
-      centroid = new Float[pointTotal.length];
-      for (int i = 0; i < pointTotal.length; i++)
-        centroid[i] = new Float(pointTotal[i] / numPoints);
+      centroid = pointTotal.divide(numPoints);
     }
     return centroid;
   }
@@ -163,35 +163,30 @@
   /**
    * Construct a new cluster with the given point as its center
    *
-   * @param center a Float[] center point
+   * @param center the center point
    */
-  public Cluster(Float[] center) {
+  public Cluster(Vector center) {
     super();
     this.clusterId = nextClusterId++;
     this.center = center;
     this.numPoints = 0;
-    this.pointTotal = Point.origin(center.length);
+    this.pointTotal = Point.origin(center.cardinality());
   }
 
   /**
    * Construct a new cluster with the given point as its center
    *
-   * @param center a Float[] center point
+   * @param center the center point
    */
-  public Cluster(Float[] center, int clusterId) {
+  public Cluster(Vector center, int clusterId) {
     super();
     this.clusterId = clusterId;
     this.center = center;
     this.numPoints = 0;
-    this.pointTotal = Point.origin(center.length);
+    this.pointTotal = Point.origin(center.cardinality());
   }
 
-  /**
-   * Return a printable representation of this object, using the user supplied
-   * identifier
-   *
-   * @return
-   */
+  @Override
   public String toString() {
     return getIdentifier() + " - " + Point.formatPoint(center);
   }
@@ -206,35 +201,33 @@
   /**
    * Add the point to the cluster
    *
-   * @param point a Float[] point to add
+   * @param point a point to add
    */
-  public void addPoint(Float[] point) {
+  public void addPoint(Vector point) {
     centroid = null;
     numPoints++;
     if (pointTotal == null)
-      pointTotal = point.clone();
+      pointTotal = point.copy();
     else
-      for (int i = 0; i < point.length; i++)
-        pointTotal[i] = new Float(point[i] + pointTotal[i]);
+      pointTotal = point.plus(pointTotal);
   }
 
   /**
    * Add the point to the cluster
    *
    * @param count the number of points in the delta
-   * @param delta a Float[] point to add
+   * @param delta a point to add
    */
-  public void addPoints(int count, Float[] delta) {
+  public void addPoints(int count, Vector delta) {
     centroid = null;
     numPoints += count;
     if (pointTotal == null)
-      pointTotal = delta.clone();
+      pointTotal = delta.copy();
     else
-      for (int i = 0; i < delta.length; i++)
-        pointTotal[i] = new Float(delta[i] + pointTotal[i]);
+      pointTotal = delta.plus(pointTotal);
   }
 
-  public Float[] getCenter() {
+  public Vector getCenter() {
     return center;
   }
 
@@ -248,7 +241,7 @@
   public void recomputeCenter() {
     center = computeCentroid();
     numPoints = 0;
-    pointTotal = Point.origin(center.length);
+    pointTotal = Point.origin(center.cardinality());
   }
 
   /**
@@ -257,12 +250,12 @@
    * @return if the cluster is converged
    */
   public boolean computeConvergence() {
-    Float[] centroid = computeCentroid();
+    Vector centroid = computeCentroid();
     converged = measure.distance(centroid, center) <= convergenceDelta;
     return converged;
   }
 
-  public Float[] getPointTotal() {
+  public Vector getPointTotal() {
     return pointTotal;
   }
 

Modified: lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/kmeans/KMeansMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/kmeans/KMeansMapper.java?rev=649728&r1=649727&r2=649728&view=diff
==============================================================================
--- lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/kmeans/KMeansMapper.java (original)
+++ lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/kmeans/KMeansMapper.java Fri Apr 18 17:10:38 2008
@@ -26,6 +26,7 @@
 import org.apache.hadoop.mapred.Mapper;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reporter;
+import org.apache.mahout.matrix.Vector;
 import org.apache.mahout.utils.Point;
 
 import java.io.IOException;
@@ -39,7 +40,7 @@
 
   public void map(WritableComparable key, Text values,
                   OutputCollector<Text, Text> output, Reporter reporter) throws IOException {
-    Float[] point = Point.decodePoint(values.toString());
+    Vector point = Point.decodePoint(values.toString());
     Cluster.emitPointToNearestCluster(point, clusters, values, output);
   }
 

Modified: lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/kmeans/KMeansReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/kmeans/KMeansReducer.java?rev=649728&r1=649727&r2=649728&view=diff
==============================================================================
--- lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/kmeans/KMeansReducer.java (original)
+++ lucene/mahout/trunk/src/main/java/org/apache/mahout/clustering/kmeans/KMeansReducer.java Fri Apr 18 17:10:38 2008
@@ -22,6 +22,7 @@
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reporter;
+import org.apache.mahout.matrix.Vector;
 import org.apache.mahout.utils.Point;
 
 import java.io.IOException;
@@ -30,7 +31,7 @@
 public class KMeansReducer extends MapReduceBase implements
         Reducer<Text, Text, Text, Text> {
 
-  float delta = 0;
+  double delta = 0;
 
   public void reduce(Text key, Iterator<Text> values,
                      OutputCollector<Text, Text> output, Reporter reporter) throws IOException {
@@ -39,7 +40,7 @@
       String value = values.next().toString();
       int ix = value.indexOf(',');
       int count = new Integer(value.substring(0, ix));
-      Float[] total = Point.decodePoint(value.substring(ix + 2));
+      Vector total = Point.decodePoint(value.substring(ix + 2));
       cluster.addPoints(count, total);
     }
     // force convergence calculation

Modified: lucene/mahout/trunk/src/main/java/org/apache/mahout/utils/Point.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/src/main/java/org/apache/mahout/utils/Point.java?rev=649728&r1=649727&r2=649728&view=diff
==============================================================================
--- lucene/mahout/trunk/src/main/java/org/apache/mahout/utils/Point.java (original)
+++ lucene/mahout/trunk/src/main/java/org/apache/mahout/utils/Point.java Fri Apr 18 17:10:38 2008
@@ -22,6 +22,9 @@
 import java.util.List;
 import java.util.regex.Pattern;
 
+import org.apache.mahout.matrix.SparseVector;
+import org.apache.mahout.matrix.Vector;
+
 public class Point {
   /**
    * Split pattern for {@link #decodePoint(String)}.
@@ -31,19 +34,19 @@
   /**
    * Format the point for input to a Mapper or Reducer
    *
-   * @param point a Float[]
+   * @param point a point to format
    * @return a String
    */
-  public static String formatPoint(Float[] point) {
-    if (point.length == 0) {
+  public static String formatPoint(Vector point) {
+    if (point.cardinality() == 0) {
       return "[]";
     }
 
     final StringBuilder out = new StringBuilder();
     out.append('[');
-    for (int i = 0; i < point.length; i++) {
+    for (int i = 0; i < point.cardinality(); i++) {
       if (i > 0) out.append(", ");
-      out.append(point[i]);
+      out.append(point.get(i));
     }
     out.append(']');
     return out.toString();
@@ -55,9 +58,9 @@
    * @param formattedString a comma-terminated String of the form 
    *    "[v1,v2,...,vn]payload". Note the payload remainder: it is optional,
    *    but can be present.
-   * @return the Float[] defining an n-dimensional point
+   * @return the n-dimensional point
    */
-  public static Float[] decodePoint(String formattedString) {
+  public static Vector decodePoint(String formattedString) {
     final int closingBracketIndex = formattedString.indexOf(']'); 
     if (formattedString.charAt(0) != '[' || closingBracketIndex < 0) {
       throw new IllegalArgumentException(formattedString);
@@ -66,10 +69,11 @@
     formattedString = formattedString.substring(1, closingBracketIndex);
 
     final String[] pts = splitPattern.split(formattedString);
-    final Float[] point = new Float[pts.length];
-    for (int i = 0; i < point.length; i++) {
-      point[i] = new Float(pts[i]);
+    final Vector point = new SparseVector(pts.length);
+    for (int i = 0; i < point.cardinality(); i++) {
+      point.set(i, Double.parseDouble(pts[i]));
     }
+
     return point;
   }
 
@@ -77,10 +81,10 @@
    * Returns a print string for the point
    *
    * @param out a String to append to
-   * @param pt  the Float[] point
+   * @param pt  the point
    * @return
    */
-  public static String ptOut(String out, Float[] pt) {
+  public static String ptOut(String out, Vector pt) {
     return out + formatPoint(pt);
   }
 
@@ -88,38 +92,37 @@
    * Return a point with length dimensions and zero values
    *
    * @param length
-   * @return a Float[] representing [0,0,0,...,0]
+   * @return a point representing [0,0,0,...,0]
    */
-  public static Float[] origin(int length) {
-    Float[] result = new Float[length];
-    for (int i = 0; i < length; i++)
-      result[i] = new Float(0);
-    return result;
+  public static Vector origin(int length) {
+    
+    Vector point = new SparseVector(length);
+    point.assign(0);
+
+    return point;
   }
 
   /**
    * Return the sum of the two points
    *
-   * @param pt1 a Float[] point
-   * @param pt2 a Float[] point
+   * @param pt1 first point to add
+   * @param pt2 second point to add
    * @return
    */
-  public static Float[] sum(Float[] pt1, Float[] pt2) {
-    Float[] result = pt1.clone();
-    for (int i = 0; i < pt1.length; i++)
-      result[i] += pt2[i];
-    return result;
+  public static Vector sum(Vector v1, Vector v2) {
+    Vector sum = v1.plus(v2);
+    return sum;
   }
 
-  public static void writePointsToFile(List<Float[]> points, String fileName)
+  public static void writePointsToFile(List<Vector> points, String fileName)
           throws IOException {
     writePointsToFileWithPayload(points, fileName, "");
   }
 
-  public static void writePointsToFileWithPayload(List<Float[]> points,
+  public static void writePointsToFileWithPayload(List<Vector> points,
                                                   String fileName, String payload) throws IOException {
     BufferedWriter output = new BufferedWriter(new FileWriter(fileName));
-    for (Float[] point : points) {
+    for (Vector point : points) {
       output.write(org.apache.mahout.utils.Point.formatPoint(point));
       output.write(payload);
       output.write("\n");

Modified: lucene/mahout/trunk/src/test/java/org/apache/mahout/clustering/canopy/TestCanopyCreation.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/src/test/java/org/apache/mahout/clustering/canopy/TestCanopyCreation.java?rev=649728&r1=649727&r2=649728&view=diff
==============================================================================
--- lucene/mahout/trunk/src/test/java/org/apache/mahout/clustering/canopy/TestCanopyCreation.java (original)
+++ lucene/mahout/trunk/src/test/java/org/apache/mahout/clustering/canopy/TestCanopyCreation.java Fri Apr 18 17:10:38 2008
@@ -25,6 +25,8 @@
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
+import org.apache.mahout.matrix.SparseVector;
+import org.apache.mahout.matrix.Vector;
 import org.apache.mahout.utils.DistanceMeasure;
 import org.apache.mahout.utils.EuclideanDistanceMeasure;
 import org.apache.mahout.utils.ManhattanDistanceMeasure;
@@ -44,40 +46,39 @@
 import java.util.Set;
 
 public class TestCanopyCreation extends TestCase {
-  static final float[][] raw = {{1, 1}, {2, 1}, {1, 2}, {2, 2},
+  static final double[][] raw = {{1, 1}, {2, 1}, {1, 2}, {2, 2},
           {3, 3}, {4, 4}, {5, 4}, {4, 5}, {5, 5}};
 
   List<Canopy> referenceManhattan;
 
   DistanceMeasure manhattanDistanceMeasure = new ManhattanDistanceMeasure();
 
-  List<Float[]> manhattanCentroids;
+  List<Vector> manhattanCentroids;
 
   List<Canopy> referenceEuclidean;
 
   DistanceMeasure euclideanDistanceMeasure = new EuclideanDistanceMeasure();
 
-  List<Float[]> euclideanCentroids;
+  List<Vector> euclideanCentroids;
 
   public TestCanopyCreation(String name) {
     super(name);
   }
 
-  private List<Float[]> getPoints(float[][] raw) {
-    List<Float[]> points = new ArrayList<Float[]>();
+  private List<Vector> getPoints(double[][] raw) {
+    List<Vector> points = new ArrayList<Vector>();
     for (int i = 0; i < raw.length; i++) {
-      float[] fr = raw[i];
-      Float[] fs = new Float[fr.length];
-      for (int j = 0; j < fs.length; j++)
-        fs[j] = fr[j];
-      points.add(fs);
+      double[] fr = raw[i];
+      Vector vec = new SparseVector(fr.length);
+      vec.assign(fr);
+      points.add(vec);
     }
     return points;
   }
 
-  private List<Text> getFormattedPoints(List<Float[]> points) {
+  private List<Text> getFormattedPoints(List<Vector> points) {
     List<Text> result = new ArrayList<Text>();
-    for (Float[] point : points)
+    for (Vector point : points)
       result.add(new Text(Point.formatPoint(point)));
     return result;
   }
@@ -114,11 +115,11 @@
       Canopy testCanopy = canopies.get(canopyIx);
       assertEquals("canopy points " + canopyIx, refCanopy.getNumPoints(),
               testCanopy.getNumPoints());
-      Float[] refCentroid = refCanopy.computeCentroid();
-      Float[] testCentroid = testCanopy.computeCentroid();
-      for (int pointIx = 0; pointIx < refCentroid.length; pointIx++) {
+      Vector refCentroid = refCanopy.computeCentroid();
+      Vector testCentroid = testCanopy.computeCentroid();
+      for (int pointIx = 0; pointIx < refCentroid.cardinality(); pointIx++) {
         assertEquals("canopy centroid " + canopyIx + "[" + pointIx + "]",
-                refCentroid[pointIx], testCentroid[pointIx]);
+                refCentroid.get(pointIx), testCentroid.get(pointIx));
       }
     }
   }
@@ -134,15 +135,15 @@
     }
   }
 
-  private void writePointsToFile(List<Float[]> points, String fileName)
+  private void writePointsToFile(List<Vector> points, String fileName)
           throws IOException {
     writePointsToFileWithPayload(points, fileName, "");
   }
 
-  private void writePointsToFileWithPayload(List<Float[]> points,
+  private void writePointsToFileWithPayload(List<Vector> points,
                                             String fileName, String payload) throws IOException {
     BufferedWriter output = new BufferedWriter(new FileWriter(fileName));
-    for (Float[] point : points) {
+    for (Vector point : points) {
       output.write(Point.formatPoint(point));
       output.write(payload);
       output.write("\n");
@@ -168,10 +169,10 @@
     rmr("output");
     rmr("testdata");
     referenceManhattan = populateCanopies(manhattanDistanceMeasure,
-            getPoints(raw), (float) 3.1, (float) 2.1);
+            getPoints(raw), 3.1, 2.1);
     manhattanCentroids = populateCentroids(referenceManhattan);
     referenceEuclidean = populateCanopies(euclideanDistanceMeasure,
-            getPoints(raw), (float) 3.1, (float) 2.1);
+            getPoints(raw), 3.1, 2.1);
     euclideanCentroids = populateCentroids(referenceEuclidean);
   }
 
@@ -179,10 +180,10 @@
    * Iterate through the canopies, adding their centroids to a list
    *
    * @param canopies a List<Canopy>
-   * @return the List<Float[]>
+   * @return the List<Vector>
    */
-  List<Float[]> populateCentroids(List<Canopy> canopies) {
-    List<Float[]> result = new ArrayList<Float[]>();
+  List<Vector> populateCentroids(List<Canopy> canopies) {
+    List<Vector> result = new ArrayList<Vector>();
     for (Canopy canopy : canopies)
       result.add(canopy.computeCentroid());
     return result;
@@ -192,13 +193,13 @@
    * Iterate through the points, adding new canopies. Return the canopies.
    *
    * @param measure a DistanceMeasure to use
-   * @param points  a list<Float[]> defining the points to be clustered
+   * @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
    */
-  List<Canopy> populateCanopies(DistanceMeasure measure, List<Float[]> points,
-                                float t1, float t2) {
+  List<Canopy> populateCanopies(DistanceMeasure measure, List<Vector> points,
+                                double t1, double t2) {
     List<Canopy> canopies = new ArrayList<Canopy>();
     Canopy.config(measure, t1, t2);
     /**
@@ -212,14 +213,14 @@
      * the list is empty.
      */
     while (!points.isEmpty()) {
-      Iterator<Float[]> ptIter = points.iterator();
-      Float[] p1 = ptIter.next();
+      Iterator<Vector> ptIter = points.iterator();
+      Vector p1 = ptIter.next();
       ptIter.remove();
       Canopy canopy = new VisibleCanopy(p1);
       canopies.add(canopy);
       while (ptIter.hasNext()) {
-        Float[] p2 = ptIter.next();
-        float dist = measure.distance(p1, p2);
+        Vector p2 = ptIter.next();
+        double dist = measure.distance(p1, p2);
         // Put all points that are within distance threshold T1 into the canopy
         if (dist < t1)
           canopy.addPoint(p2);
@@ -249,16 +250,16 @@
     for (int canopyIx = 0; canopyIx < referenceManhattan.size(); canopyIx++) {
       Canopy testCanopy = referenceManhattan.get(canopyIx);
       int[] expectedNumPoints = {4, 4, 3};
-      float[][] expectedCentroids = {{(float) 1.5, (float) 1.5},
-              {(float) 4.0, (float) 4.0},
-              {(float) 4.6666665, (float) 4.6666665}};
+      double[][] expectedCentroids = {{1.5, 1.5},
+              {4.0, 4.0},
+              {4.666666666666667, 4.6666666666666667}};
       assertEquals("canopy points " + canopyIx, expectedNumPoints[canopyIx],
               testCanopy.getNumPoints());
-      float[] refCentroid = expectedCentroids[canopyIx];
-      Float[] testCentroid = testCanopy.computeCentroid();
+      double[] refCentroid = expectedCentroids[canopyIx];
+      Vector testCentroid = testCanopy.computeCentroid();
       for (int pointIx = 0; pointIx < refCentroid.length; pointIx++) {
         assertEquals("canopy centroid " + canopyIx + "[" + pointIx + "]",
-                refCentroid[pointIx], testCentroid[pointIx]);
+                refCentroid[pointIx], testCentroid.get(pointIx));
       }
     }
   }
@@ -277,16 +278,16 @@
     for (int canopyIx = 0; canopyIx < referenceManhattan.size(); canopyIx++) {
       Canopy testCanopy = referenceEuclidean.get(canopyIx);
       int[] expectedNumPoints = {5, 5, 3};
-      float[][] expectedCentroids = {{(float) 1.8, (float) 1.8},
-              {(float) 4.2, (float) 4.2},
-              {(float) 4.6666665, (float) 4.6666665}};
+      double[][] expectedCentroids = {{1.8, 1.8},
+              {4.2, 4.2},
+              {4.666666666666667, 4.666666666666667}};
       assertEquals("canopy points " + canopyIx, expectedNumPoints[canopyIx],
               testCanopy.getNumPoints());
-      float[] refCentroid = expectedCentroids[canopyIx];
-      Float[] testCentroid = testCanopy.computeCentroid();
+      double[] refCentroid = expectedCentroids[canopyIx];
+      Vector testCentroid = testCanopy.computeCentroid();
       for (int pointIx = 0; pointIx < refCentroid.length; pointIx++) {
         assertEquals("canopy centroid " + canopyIx + "[" + pointIx + "]",
-                refCentroid[pointIx], testCentroid[pointIx]);
+                refCentroid[pointIx], testCentroid.get(pointIx));
       }
     }
   }
@@ -298,11 +299,11 @@
    * @throws Exception
    */
   public void testIterativeManhattan() throws Exception {
-    List<Float[]> points = getPoints(raw);
-    Canopy.config(new ManhattanDistanceMeasure(), (float) 3.1, (float) 2.1);
+    List<Vector> points = getPoints(raw);
+    Canopy.config(new ManhattanDistanceMeasure(), 3.1, 2.1);
 
     List<Canopy> canopies = new ArrayList<Canopy>();
-    for (Float[] point : points)
+    for (Vector point : points)
       Canopy.addPointToCanopies(point, canopies);
 
     System.out.println("testIterativeManhattan");
@@ -317,11 +318,11 @@
    * @throws Exception
    */
   public void testIterativeEuclidean() throws Exception {
-    List<Float[]> points = getPoints(raw);
-    Canopy.config(new EuclideanDistanceMeasure(), (float) 3.1, (float) 2.1);
+    List<Vector> points = getPoints(raw);
+    Canopy.config(new EuclideanDistanceMeasure(), 3.1, 2.1);
 
     List<Canopy> canopies = new ArrayList<Canopy>();
-    for (Float[] point : points)
+    for (Vector point : points)
       Canopy.addPointToCanopies(point, canopies);
 
     System.out.println("testIterativeEuclidean");
@@ -340,15 +341,15 @@
     CanopyMapper mapper = new CanopyMapper();
     CanopyCombiner combiner = new CanopyCombiner();
     DummyOutputCollector collector = new DummyOutputCollector();
-    Canopy.config(manhattanDistanceMeasure, ((float) 3.1), ((float) 2.1));
-    List<Float[]> points = getPoints(raw);
+    Canopy.config(manhattanDistanceMeasure, (3.1), (2.1));
+    List<Vector> points = getPoints(raw);
     // map the data
-    for (Float[] point : points)
+    for (Vector point : points)
       mapper.map(new Text(), new Text(Point.formatPoint(point)), collector,
               null);
     assertEquals("Number of map results", 3, collector.getData().size());
     // now combine the mapper output
-    Canopy.config(manhattanDistanceMeasure, ((float) 3.1), ((float) 2.1));
+    Canopy.config(manhattanDistanceMeasure, (3.1), (2.1));
     Map<String, List<Text>> mapData = collector.getData();
     collector = new DummyOutputCollector();
     for (String key : mapData.keySet())
@@ -374,15 +375,15 @@
     CanopyMapper mapper = new CanopyMapper();
     CanopyCombiner combiner = new CanopyCombiner();
     DummyOutputCollector collector = new DummyOutputCollector();
-    Canopy.config(euclideanDistanceMeasure, ((float) 3.1), ((float) 2.1));
-    List<Float[]> points = getPoints(raw);
+    Canopy.config(euclideanDistanceMeasure, (3.1), (2.1));
+    List<Vector> points = getPoints(raw);
     // map the data
-    for (Float[] point : points)
+    for (Vector point : points)
       mapper.map(new Text(), new Text(Point.formatPoint(point)), collector,
               null);
     assertEquals("Number of map results", 3, collector.getData().size());
     // now combine the mapper output
-    Canopy.config(euclideanDistanceMeasure, ((float) 3.1), ((float) 2.1));
+    Canopy.config(euclideanDistanceMeasure, (3.1), (2.1));
     Map<String, List<Text>> mapData = collector.getData();
     collector = new DummyOutputCollector();
     for (String key : mapData.keySet())
@@ -407,8 +408,8 @@
   public void testCanopyReducerManhattan() throws Exception {
     CanopyReducer reducer = new CanopyReducer();
     DummyOutputCollector collector = new DummyOutputCollector();
-    Canopy.config(manhattanDistanceMeasure, ((float) 3.1), ((float) 2.1));
-    List<Float[]> points = getPoints(raw);
+    Canopy.config(manhattanDistanceMeasure, (3.1), (2.1));
+    List<Vector> points = getPoints(raw);
     List<Text> texts = getFormattedPoints(points);
     reducer.reduce(new Text("centroid"), texts.iterator(), collector, null);
     reducer.close();
@@ -434,8 +435,8 @@
   public void testCanopyReducerEuclidean() throws Exception {
     CanopyReducer reducer = new CanopyReducer();
     DummyOutputCollector collector = new DummyOutputCollector();
-    Canopy.config(euclideanDistanceMeasure, ((float) 3.1), ((float) 2.1));
-    List<Float[]> points = getPoints(raw);
+    Canopy.config(euclideanDistanceMeasure, (3.1), (2.1));
+    List<Vector> points = getPoints(raw);
     List<Text> texts = getFormattedPoints(points);
     reducer.reduce(new Text("centroid"), texts.iterator(), collector, null);
     reducer.close();
@@ -458,7 +459,7 @@
    * @throws Exception
    */
   public void testCanopyGenManhattanMR() throws Exception {
-    List<Float[]> points = getPoints(raw);
+    List<Vector> points = getPoints(raw);
     File testData = new File("testdata");
     if (!testData.exists())
       testData.mkdir();
@@ -466,7 +467,7 @@
     writePointsToFile(points, "testdata/file2");
     // now run the Canopy Driver
     CanopyDriver.runJob("testdata", "output/canopies",
-            ManhattanDistanceMeasure.class.getName(), (float) 3.1, (float) 2.1);
+            ManhattanDistanceMeasure.class.getName(), 3.1, 2.1);
 
     // verify output from sequence file
     JobConf job = new JobConf(
@@ -481,7 +482,7 @@
     assertEquals("1st value", "C0: [1.5, 1.5]", value.toString());
     assertTrue("more to come", reader.next(key, value));
     assertEquals("2nd key", "C1", key.toString());
-    assertEquals("2nd value", "C1: [4.333333, 4.333333]", value.toString());
+    assertEquals("2nd value", "C1: [4.333333333333334, 4.333333333333334]", value.toString());
     assertFalse("more to come", reader.next(key, value));
     reader.close();
   }
@@ -493,7 +494,7 @@
    * @throws Exception
    */
   public void testCanopyGenEuclideanMR() throws Exception {
-    List<Float[]> points = getPoints(raw);
+    List<Vector> points = getPoints(raw);
     File testData = new File("testdata");
     if (!testData.exists())
       testData.mkdir();
@@ -501,7 +502,7 @@
     writePointsToFile(points, "testdata/file2");
     // now run the Canopy Driver
     CanopyDriver.runJob("testdata", "output/canopies",
-            EuclideanDistanceMeasure.class.getName(), (float) 3.1, (float) 2.1);
+            EuclideanDistanceMeasure.class.getName(), 3.1, 2.1);
 
     // verify output from sequence file
     JobConf job = new JobConf(
@@ -516,7 +517,7 @@
     assertEquals("1st value", "C0: [1.8, 1.8]", value.toString());
     assertTrue("more to come", reader.next(key, value));
     assertEquals("2nd key", "C1", key.toString());
-    assertEquals("2nd value", "C1: [4.4333334, 4.4333334]", value.toString());
+    assertEquals("2nd value", "C1: [4.433333333333334, 4.433333333333334]", value.toString());
     assertFalse("more to come", reader.next(key, value));
     reader.close();
   }
@@ -528,16 +529,16 @@
    * @throws Exception
    */
   public void testClusterMapperManhattan() throws Exception {
-    Canopy.config(manhattanDistanceMeasure, ((float) 3.1), ((float) 2.1));
+    Canopy.config(manhattanDistanceMeasure, (3.1), (2.1));
     ClusterMapper mapper = new ClusterMapper();
     List<Canopy> canopies = new ArrayList<Canopy>();
     DummyOutputCollector collector = new DummyOutputCollector();
-    for (Float[] centroid : manhattanCentroids)
+    for (Vector centroid : manhattanCentroids)
       canopies.add(new Canopy(centroid));
     mapper.config(canopies);
-    List<Float[]> points = getPoints(raw);
+    List<Vector> points = getPoints(raw);
     // map the data
-    for (Float[] point : points)
+    for (Vector point : points)
       mapper.map(new Text(), new Text(Point.formatPoint(point)), collector,
               null);
     Map<String, List<Text>> data = collector.getData();
@@ -558,16 +559,16 @@
    * @throws Exception
    */
   public void testClusterMapperEuclidean() throws Exception {
-    Canopy.config(euclideanDistanceMeasure, ((float) 3.1), ((float) 2.1));
+    Canopy.config(euclideanDistanceMeasure, (3.1), (2.1));
     ClusterMapper mapper = new ClusterMapper();
     List<Canopy> canopies = new ArrayList<Canopy>();
     DummyOutputCollector collector = new DummyOutputCollector();
-    for (Float[] centroid : euclideanCentroids)
+    for (Vector centroid : euclideanCentroids)
       canopies.add(new Canopy(centroid));
     mapper.config(canopies);
-    List<Float[]> points = getPoints(raw);
+    List<Vector> points = getPoints(raw);
     // map the data
-    for (Float[] point : points)
+    for (Vector point : points)
       mapper.map(new Text(), new Text(Point.formatPoint(point)), collector,
               null);
     Map<String, List<Text>> data = collector.getData();
@@ -588,16 +589,16 @@
    * @throws Exception
    */
   public void testClusterReducerManhattan() throws Exception {
-    Canopy.config(manhattanDistanceMeasure, ((float) 3.1), ((float) 2.1));
+    Canopy.config(manhattanDistanceMeasure, (3.1), (2.1));
     ClusterMapper mapper = new ClusterMapper();
     List<Canopy> canopies = new ArrayList<Canopy>();
     DummyOutputCollector collector = new DummyOutputCollector();
-    for (Float[] centroid : manhattanCentroids)
+    for (Vector centroid : manhattanCentroids)
       canopies.add(new Canopy(centroid));
     mapper.config(canopies);
-    List<Float[]> points = getPoints(raw);
+    List<Vector> points = getPoints(raw);
     // map the data
-    for (Float[] point : points)
+    for (Vector point : points)
       mapper.map(new Text(), new Text(Point.formatPoint(point)), collector,
               null);
     Map<String, List<Text>> data = collector.getData();
@@ -627,16 +628,16 @@
    * @throws Exception
    */
   public void testClusterReducerEuclidean() throws Exception {
-    Canopy.config(euclideanDistanceMeasure, ((float) 3.1), ((float) 2.1));
+    Canopy.config(euclideanDistanceMeasure, (3.1), (2.1));
     ClusterMapper mapper = new ClusterMapper();
     List<Canopy> canopies = new ArrayList<Canopy>();
     DummyOutputCollector collector = new DummyOutputCollector();
-    for (Float[] centroid : euclideanCentroids)
+    for (Vector centroid : euclideanCentroids)
       canopies.add(new Canopy(centroid));
     mapper.config(canopies);
-    List<Float[]> points = getPoints(raw);
+    List<Vector> points = getPoints(raw);
     // map the data
-    for (Float[] point : points)
+    for (Vector point : points)
       mapper.map(new Text(), new Text(Point.formatPoint(point)), collector,
               null);
     Map<String, List<Text>> data = collector.getData();
@@ -666,7 +667,7 @@
    * @throws Exception
    */
   public void testClusteringManhattanMR() throws Exception {
-    List<Float[]> points = getPoints(raw);
+    List<Vector> points = getPoints(raw);
     File testData = new File("testdata");
     if (!testData.exists())
       testData.mkdir();
@@ -674,7 +675,7 @@
     writePointsToFile(points, "testdata/file2");
     // now run the Job
     CanopyClusteringJob.runJob("testdata", "output",
-            ManhattanDistanceMeasure.class.getName(), (float) 3.1, (float) 2.1);
+            ManhattanDistanceMeasure.class.getName(), 3.1,  2.1);
     BufferedReader reader = new BufferedReader(new FileReader(
             "output/clusters/part-00000"));
     int count = 0;
@@ -694,7 +695,7 @@
    * @throws Exception
    */
   public void testClusteringEuclideanMR() throws Exception {
-    List<Float[]> points = getPoints(raw);
+    List<Vector> points = getPoints(raw);
     File testData = new File("testdata");
     if (!testData.exists())
       testData.mkdir();
@@ -702,7 +703,7 @@
     writePointsToFile(points, "testdata/file2");
     // now run the Job
     CanopyClusteringJob.runJob("testdata", "output",
-            EuclideanDistanceMeasure.class.getName(), (float) 3.1, (float) 2.1);
+            EuclideanDistanceMeasure.class.getName(),  3.1,  2.1);
     BufferedReader reader = new BufferedReader(new FileReader(
             "output/clusters/part-00000"));
     int count = 0;
@@ -724,7 +725,7 @@
    * @throws Exception
    */
   public void testClusteringManhattanMRWithPayload() throws Exception {
-    List<Float[]> points = getPoints(raw);
+    List<Vector> points = getPoints(raw);
     File testData = new File("testdata");
     if (!testData.exists())
       testData.mkdir();
@@ -732,7 +733,7 @@
     writePointsToFileWithPayload(points, "testdata/file2", "file2");
     // now run the Job
     CanopyClusteringJob.runJob("testdata", "output",
-            ManhattanDistanceMeasure.class.getName(), (float) 3.1, (float) 2.1);
+            ManhattanDistanceMeasure.class.getName(),  3.1,  2.1);
     BufferedReader reader = new BufferedReader(new FileReader(
             "output/clusters/part-00000"));
     int count = 0;
@@ -756,7 +757,7 @@
    * @throws Exception
    */
   public void testClusteringEuclideanMRWithPayload() throws Exception {
-    List<Float[]> points = getPoints(raw);
+    List<Vector> points = getPoints(raw);
     File testData = new File("testdata");
     if (!testData.exists())
       testData.mkdir();
@@ -764,7 +765,7 @@
     writePointsToFileWithPayload(points, "testdata/file2", "file2");
     // now run the Job
     CanopyClusteringJob.runJob("testdata", "output",
-            EuclideanDistanceMeasure.class.getName(), (float) 3.1, (float) 2.1);
+            EuclideanDistanceMeasure.class.getName(),  3.1,  2.1);
     BufferedReader reader = new BufferedReader(new FileReader(
             "output/clusters/part-00000"));
     int count = 0;
@@ -786,7 +787,7 @@
    * @throws Exception
    */
   public void testUserDefinedDistanceMeasure() throws Exception {
-    List<Float[]> points = getPoints(raw);
+    List<Vector> points = getPoints(raw);
     File testData = new File("testdata");
     if (!testData.exists())
       testData.mkdir();
@@ -795,7 +796,7 @@
     // 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(), (float) 3.1, (float) 2.1);
+            UserDefinedDistanceMeasure.class.getName(),  3.1,  2.1);
 
     // verify output from sequence file
     JobConf job = new JobConf(
@@ -810,7 +811,7 @@
     assertEquals("1st value", "C0: [1.5, 1.5]", value.toString());
     assertTrue("more to come", reader.next(key, value));
     assertEquals("2nd key", "C1", key.toString());
-    assertEquals("2nd value", "C1: [4.333333, 4.333333]", value.toString());
+    assertEquals("2nd value", "C1: [4.333333333333334, 4.333333333333334]", value.toString());
     assertFalse("more to come", reader.next(key, value));
     reader.close();
   }

Modified: lucene/mahout/trunk/src/test/java/org/apache/mahout/clustering/canopy/VisibleCanopy.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/src/test/java/org/apache/mahout/clustering/canopy/VisibleCanopy.java?rev=649728&r1=649727&r2=649728&view=diff
==============================================================================
--- lucene/mahout/trunk/src/test/java/org/apache/mahout/clustering/canopy/VisibleCanopy.java (original)
+++ lucene/mahout/trunk/src/test/java/org/apache/mahout/clustering/canopy/VisibleCanopy.java Fri Apr 18 17:10:38 2008
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 
+import org.apache.mahout.matrix.Vector;
 import org.apache.mahout.utils.Point;
 
 import java.util.ArrayList;
@@ -27,9 +28,9 @@
  * for production use since it holds onto all its points.
  */
 public class VisibleCanopy extends Canopy {
-  private List<Float[]> points = new ArrayList<Float[]>();
+  private List<Vector> points = new ArrayList<Vector>();
 
-  public VisibleCanopy(Float[] point) {
+  public VisibleCanopy(Vector point) {
     super(point);
     points.add(point);
   }
@@ -37,9 +38,9 @@
   /**
    * Add a point to the canopy
    *
-   * @param point a Float[]
+   * @param point a point
    */
-  public void addPoint(Float[] point) {
+  public void addPoint(Vector point) {
     super.addPoint(point);
     points.add(point);
   }
@@ -52,7 +53,7 @@
    */
   public String toString() {
     String out = super.toString() + ": ";
-    for (Float[] pt : points)
+    for (Vector pt : points)
       out = Point.ptOut(out, pt);
     return out;
   }

Modified: lucene/mahout/trunk/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java?rev=649728&r1=649727&r2=649728&view=diff
==============================================================================
--- lucene/mahout/trunk/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java (original)
+++ lucene/mahout/trunk/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java Fri Apr 18 17:10:38 2008
@@ -26,6 +26,8 @@
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.mahout.clustering.canopy.CanopyDriver;
 import org.apache.mahout.clustering.canopy.DummyOutputCollector;
+import org.apache.mahout.matrix.SparseVector;
+import org.apache.mahout.matrix.Vector;
 import org.apache.mahout.utils.DistanceMeasure;
 import org.apache.mahout.utils.EuclideanDistanceMeasure;
 import org.apache.mahout.utils.ManhattanDistanceMeasure;
@@ -39,7 +41,7 @@
 
 public class TestKmeansClustering extends TestCase {
 
-  static final float[][] reference = {{1, 1}, {2, 1}, {1, 2}, {2, 2},
+  static final double[][] reference = {{1, 1}, {2, 1}, {1, 2}, {2, 2},
           {3, 3}, {4, 4}, {5, 4}, {4, 5}, {5, 5}};
 
   static int[][] expectedNumPoints = {{9}, {4, 5}, {4, 5, 0},
@@ -75,12 +77,12 @@
    * over the points and clusters until their centers converge or until the
    * maximum number of iterations is exceeded.
    *
-   * @param points   the input List<Float[]> of points
+   * @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 void referenceKmeans(List<Float[]> points, List<Cluster> clusters,
+  private void referenceKmeans(List<Vector> points, List<Cluster> clusters,
                                DistanceMeasure measure, int maxIter) {
     boolean converged = false;
     int iteration = 0;
@@ -93,21 +95,21 @@
    * Perform a single iteration over the points and clusters, assigning points
    * to clusters and returning if the iterations are completed.
    *
-   * @param points   the List<Float[]> having the input points
+   * @param points   the List<Vector> having the input points
    * @param clusters the List<Cluster> clusters
    * @param measure  a DistanceMeasure to use
    * @return
    */
-  private boolean iterateReference(List<Float[]> points,
+  private boolean iterateReference(List<Vector> points,
                                    List<Cluster> clusters, DistanceMeasure measure) {
     boolean converged;
     converged = true;
     // iterate through all points, assigning each to the nearest cluster
-    for (Float[] point : points) {
+    for (Vector point : points) {
       Cluster closestCluster = null;
-      float closestDistance = Float.MAX_VALUE;
+      double closestDistance = Double.MAX_VALUE;
       for (Cluster cluster : clusters) {
-        float distance = measure.distance(cluster.getCenter(), point);
+        double distance = measure.distance(cluster.getCenter(), point);
         if (closestCluster == null || closestDistance > distance) {
           closestCluster = cluster;
           closestDistance = distance;
@@ -127,14 +129,13 @@
     return converged;
   }
 
-  private List<Float[]> getPoints(float[][] raw) {
-    List<Float[]> points = new ArrayList<Float[]>();
+  private List<Vector> getPoints(double[][] raw) {
+    List<Vector> points = new ArrayList<Vector>();
     for (int i = 0; i < raw.length; i++) {
-      float[] fr = raw[i];
-      Float[] fs = new Float[fr.length];
-      for (int j = 0; j < fs.length; j++)
-        fs[j] = fr[j];
-      points.add(fs);
+      double[] fr = raw[i];
+      Vector vec = new SparseVector(fr.length);
+      vec.assign(fr);
+      points.add(vec);
     }
     return points;
   }
@@ -145,16 +146,18 @@
    * @throws Exception
    */
   public void testReferenceImplementation() throws Exception {
-    List<Float[]> points = getPoints(reference);
+    List<Vector> points = getPoints(reference);
     DistanceMeasure measure = new EuclideanDistanceMeasure();
-    Cluster.config(measure, (float) 0.001);
+    Cluster.config(measure, 0.001);
     // try all possible values of k
     for (int k = 0; k < points.size(); k++) {
       System.out.println("Test k=" + (k + 1) + ":");
       // pick k initial cluster centers at random
       List<Cluster> clusters = new ArrayList<Cluster>();
-      for (int i = 0; i < k + 1; i++)
-        clusters.add(new VisibleCluster(points.get(i)));
+      for (int i = 0; i < k + 1; i++) {
+        Vector vec = points.get(i);
+        clusters.add(new VisibleCluster(vec));
+      }
       // iterate clusters until they converge
       int maxIter = 10;
       referenceKmeans(points, clusters, measure, maxIter);
@@ -175,8 +178,8 @@
   public void testKMeansMapper() throws Exception {
     KMeansMapper mapper = new KMeansMapper();
     EuclideanDistanceMeasure euclideanDistanceMeasure = new EuclideanDistanceMeasure();
-    Cluster.config(euclideanDistanceMeasure, (float) 0.001);
-    List<Float[]> points = getPoints(reference);
+    Cluster.config(euclideanDistanceMeasure, 0.001);
+    List<Vector> points = getPoints(reference);
     for (int k = 0; k < points.size(); k++) {
       // pick k initial cluster centers at random
       DummyOutputCollector collector = new DummyOutputCollector();
@@ -189,17 +192,18 @@
       }
       mapper.config(clusters);
       // map the data
-      for (Float[] point : points)
+      for (Vector point : points) {
         mapper.map(new Text(), new Text(Point.formatPoint(point)), collector,
                 null);
+      }
       assertEquals("Number of map results", k + 1, collector.getData().size());
       // now verify that all points are correctly allocated
       for (String key : collector.getKeys()) {
         Cluster cluster = Cluster.decodeCluster(key);
         List<Text> values = collector.getValue(key);
         for (Writable value : values) {
-          Float[] point = Point.decodePoint(value.toString());
-          float distance = euclideanDistanceMeasure.distance(cluster
+          Vector point = Point.decodePoint(value.toString());
+          double distance = euclideanDistanceMeasure.distance(cluster
                   .getCenter(), point);
           for (Cluster c : clusters)
             assertTrue("distance error", distance <= euclideanDistanceMeasure
@@ -218,24 +222,26 @@
   public void testKMeansCombiner() throws Exception {
     KMeansMapper mapper = new KMeansMapper();
     EuclideanDistanceMeasure euclideanDistanceMeasure = new EuclideanDistanceMeasure();
-    Cluster.config(euclideanDistanceMeasure, (float) 0.001);
-    List<Float[]> points = getPoints(reference);
+    Cluster.config(euclideanDistanceMeasure, 0.001);
+    List<Vector> points = getPoints(reference);
     for (int k = 0; k < points.size(); k++) {
       // pick k initial cluster centers at random
       DummyOutputCollector collector = new DummyOutputCollector();
       List<Cluster> clusters = new ArrayList<Cluster>();
       for (int i = 0; i < k + 1; i++) {
-        Cluster cluster = new Cluster(points.get(i));
+        Vector vec = points.get(i);
+
+        Cluster cluster = new Cluster(vec);
         // add the center so the centroid will be correct upon output
         cluster.addPoint(cluster.getCenter());
         clusters.add(cluster);
       }
       mapper.config(clusters);
       // map the data
-      for (Float[] point : points)
+      for (Vector point : points) {
         mapper.map(new Text(), new Text(Point.formatPoint(point)), collector,
                 null);
-
+      }
       // now combine the data
       KMeansCombiner combiner = new KMeansCombiner();
       DummyOutputCollector collector2 = new DummyOutputCollector();
@@ -246,7 +252,7 @@
       assertEquals("Number of map results", k + 1, collector2.getData().size());
       // now verify that all points are accounted for
       int count = 0;
-      Float[] total = Point.origin(2);
+      Vector total = Point.origin(2);
       for (String key : collector2.getKeys()) {
         List<Text> values = collector2.getValue(key);
         assertEquals("too many values", 1, values.size());
@@ -256,8 +262,8 @@
         total = Point.sum(total, Point.decodePoint(value.substring(ix + 2)));
       }
       assertEquals("total points", 9, count);
-      assertEquals("point total[0]", 27, total[0].intValue());
-      assertEquals("point total[1]", 27, total[1].intValue());
+      assertEquals("point total[0]", 27, (new Double (total.get(0))).intValue());
+      assertEquals("point total[1]", 27, (new Double (total.get(1))).intValue());
     }
   }
 
@@ -270,25 +276,26 @@
   public void testKMeansReducer() throws Exception {
     KMeansMapper mapper = new KMeansMapper();
     EuclideanDistanceMeasure euclideanDistanceMeasure = new EuclideanDistanceMeasure();
-    Cluster.config(euclideanDistanceMeasure, (float) 0.001);
-    List<Float[]> points = getPoints(reference);
+    Cluster.config(euclideanDistanceMeasure, 0.001);
+    List<Vector> points = getPoints(reference);
     for (int k = 0; k < points.size(); k++) {
       System.out.println("K = " + k);
       // pick k initial cluster centers at random
       DummyOutputCollector collector = new DummyOutputCollector();
       List<Cluster> clusters = new ArrayList<Cluster>();
       for (int i = 0; i < k + 1; i++) {
-        Cluster cluster = new Cluster(points.get(i), i);
+        Vector vec = points.get(i);
+        Cluster cluster = new Cluster(vec, i);
         // add the center so the centroid will be correct upon output
         cluster.addPoint(cluster.getCenter());
         clusters.add(cluster);
       }
       mapper.config(clusters);
       // map the data
-      for (Float[] point : points)
+      for (Vector point : points) {
         mapper.map(new Text(), new Text(Point.formatPoint(point)), collector,
                 null);
-
+      }
       // now combine the data
       KMeansCombiner combiner = new KMeansCombiner();
       DummyOutputCollector collector2 = new DummyOutputCollector();
@@ -307,8 +314,10 @@
 
       // compute the reference result after one iteration and compare
       List<Cluster> reference = new ArrayList<Cluster>();
-      for (int i = 0; i < k + 1; i++)
-        reference.add(new Cluster(points.get(i), i));
+      for (int i = 0; i < k + 1; i++) {
+        Vector vec = points.get(i);
+        reference.add(new Cluster(vec, i));
+      }
       boolean converged = iterateReference(points, reference,
               euclideanDistanceMeasure);
       if (k == 8)
@@ -327,10 +336,10 @@
         converged = converged && cluster.isConverged();
         System.out.println("ref= " + ref.toString() + " cluster= "
                 + cluster.toString());
-        assertEquals(k + " center[" + key + "][0]", ref.getCenter()[0], cluster
-                .getCenter()[0]);
-        assertEquals(k + " center[" + key + "][1]", ref.getCenter()[1], cluster
-                .getCenter()[1]);
+        assertEquals(k + " center[" + key + "][0]", ref.getCenter().get(0), cluster
+                .getCenter().get(0));
+        assertEquals(k + " center[" + key + "][1]", ref.getCenter().get(1), cluster
+                .getCenter().get(1));
       }
       if (k == 8)
         assertTrue("not converged? " + k, converged);
@@ -345,13 +354,14 @@
    * @throws Exception
    */
   public void testKMeansMRJob() throws Exception {
-    List<Float[]> points = getPoints(reference);
+    List<Vector> points = getPoints(reference);
     File testData = new File("testdata");
     if (!testData.exists())
       testData.mkdir();
     testData = new File("testdata/points");
     if (!testData.exists())
       testData.mkdir();
+    
     Point.writePointsToFile(points, "testdata/points/file1");
     Point.writePointsToFile(points, "testdata/points/file2");
     for (int k = 0; k < points.size(); k++) {
@@ -363,7 +373,9 @@
       SequenceFile.Writer writer = new SequenceFile.Writer(fs, job, path,
               Text.class, Text.class);
       for (int i = 0; i < k + 1; i++) {
-        Cluster cluster = new Cluster(points.get(i));
+        Vector vec = points.get(i);
+
+        Cluster cluster = new Cluster(vec);
         // add the center so the centroid will be correct upon output
         cluster.addPoint(cluster.getCenter());
         writer.append(new Text(cluster.getIdentifier()), new Text(Cluster
@@ -409,7 +421,7 @@
    * @throws Exception
    */
   public void textKMeansWithCanopyClusterInput() throws Exception {
-    List<Float[]> points = getPoints(reference);
+    List<Vector> points = getPoints(reference);
     File testData = new File("testdata");
     if (!testData.exists())
       testData.mkdir();
@@ -421,7 +433,7 @@
 
     // now run the Canopy job
     CanopyDriver.runJob("testdata/points", "testdata/canopies",
-            ManhattanDistanceMeasure.class.getName(), (float) 3.1, (float) 2.1);
+            ManhattanDistanceMeasure.class.getName(), 3.1, 2.1);
 
     // now run the KMeans job
     KMeansDriver.runJob("testdata/points", "testdata/canopies", "output",

Modified: lucene/mahout/trunk/src/test/java/org/apache/mahout/clustering/kmeans/VisibleCluster.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/src/test/java/org/apache/mahout/clustering/kmeans/VisibleCluster.java?rev=649728&r1=649727&r2=649728&view=diff
==============================================================================
--- lucene/mahout/trunk/src/test/java/org/apache/mahout/clustering/kmeans/VisibleCluster.java (original)
+++ lucene/mahout/trunk/src/test/java/org/apache/mahout/clustering/kmeans/VisibleCluster.java Fri Apr 18 17:10:38 2008
@@ -16,6 +16,7 @@
  */
 package org.apache.mahout.clustering.kmeans;
 
+import org.apache.mahout.matrix.Vector;
 import org.apache.mahout.utils.Point;
 
 import java.util.ArrayList;
@@ -28,24 +29,24 @@
  */
 public class VisibleCluster extends Cluster {
 
-  private List<Float[]> points = new ArrayList<Float[]>();
+  private List<Vector> points = new ArrayList<Vector>();
 
   @Override
   public void recomputeCenter() {
     super.recomputeCenter();
-    points = new ArrayList<Float[]>();
+    points = new ArrayList<Vector>();
   }
 
-  public VisibleCluster(Float[] point) {
+  public VisibleCluster(Vector point) {
     super(point);
   }
 
   /**
    * Add a point to the canopy
    *
-   * @param point a Float[]
+   * @param point a Double[]
    */
-  public void addPoint(Float[] point) {
+  public void addPoint(Vector point) {
     super.addPoint(point);
     points.add(point);
   }
@@ -58,7 +59,7 @@
    */
   public String toString() {
     String out = super.toString() + ": ";
-    for (Float[] pt : points)
+    for (Vector pt : points)
       out += Point.formatPoint(pt);
     return out;
   }

Modified: lucene/mahout/trunk/src/test/java/org/apache/mahout/utils/TestPoint.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/src/test/java/org/apache/mahout/utils/TestPoint.java?rev=649728&r1=649727&r2=649728&view=diff
==============================================================================
--- lucene/mahout/trunk/src/test/java/org/apache/mahout/utils/TestPoint.java (original)
+++ lucene/mahout/trunk/src/test/java/org/apache/mahout/utils/TestPoint.java Fri Apr 18 17:10:38 2008
@@ -2,6 +2,9 @@
 
 import java.util.Arrays;
 
+import org.apache.mahout.matrix.SparseVector;
+import org.apache.mahout.matrix.Vector;
+
 import junit.framework.TestCase;
 
 /**
@@ -23,22 +26,26 @@
 
 public class TestPoint extends TestCase {
   public void testFormatPoint() {
-    assertEquals("[1.0, 1.5]", Point.formatPoint(new Float [] {1.0f, 1.5f}));
+    Vector vec = new SparseVector(2);
+    vec.assign(new double [] {1.0, 1.5});
+    assertEquals("[1.0, 1.5]", Point.formatPoint(vec));
   }
   
   public void testPtOut() {
-    assertEquals("abc[1.0, 1.5]", Point.ptOut("abc", new Float [] {1.0f, 1.5f}));
+    Vector vec = new SparseVector(2);
+    vec.assign(new double [] {1.0, 1.5});
+    assertEquals("abc[1.0, 1.5]", Point.ptOut("abc", vec));
   }
   
   public void testDecodePoint() {
-    assertEquals(
-        Arrays.asList(new Float [] {1.0f, 2.5f}), 
-        Arrays.asList(Point.decodePoint("[1.0, 2.5]")));
+    assertTrue(
+        Arrays.equals((new double [] {1.0, 2.5}), 
+        Point.decodePoint("[1.0, 2.5]").toArray()));
   }
   
   public void testDecodePointWithPayload() {
-    assertEquals(
-        Arrays.asList(new Float [] {1.0f, 2.5f}), 
-        Arrays.asList(Point.decodePoint("[1.0, 2.5] payloadhere, blah [][]")));
+    assertTrue(
+        Arrays.equals(new double [] {1.0, 2.5}, 
+        Point.decodePoint("[1.0, 2.5] payloadhere, blah [][]").toArray()));
   }
 }