You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by gs...@apache.org on 2009/06/25 22:46:43 UTC

svn commit: r788504 - in /lucene/mahout/trunk/core/src/main/java/org/apache/mahout: clustering/canopy/ clustering/kmeans/ matrix/ utils/

Author: gsingers
Date: Thu Jun 25 20:46:42 2009
New Revision: 788504

URL: http://svn.apache.org/viewvc?rev=788504&view=rev
Log:
MAHOUT-121: Take advantage of sparseness, also intro Euclidean base class that doesn't do the square root

Added:
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/SquaredEuclideanDistanceMeasure.java
Modified:
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/Canopy.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/Cluster.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/DenseVector.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/SparseVector.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/Vector.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/VectorView.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/CosineDistanceMeasure.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/DistanceMeasure.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/EuclideanDistanceMeasure.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/ManhattanDistanceMeasure.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/TanimotoDistanceMeasure.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/WeightedEuclideanDistanceMeasure.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/WeightedManhattanDistanceMeasure.java

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/Canopy.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/Canopy.java?rev=788504&r1=788503&r2=788504&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/Canopy.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/Canopy.java Thu Jun 25 20:46:42 2009
@@ -143,7 +143,7 @@
   public static void addPointToCanopies(Vector point, List<Canopy> canopies) {
     boolean pointStronglyBound = false;
     for (Canopy canopy : canopies) {
-      double dist = measure.distance(canopy.getCenter(), point);
+      double dist = measure.distance(canopy.getCenter().getLengthSquared(), canopy.getCenter(), point);
       if (dist < t1)
         canopy.addPoint(point);
       pointStronglyBound = pointStronglyBound || (dist < t2);
@@ -166,7 +166,7 @@
       throws IOException {
     boolean pointStronglyBound = false;
     for (Canopy canopy : canopies) {
-      double dist = measure.distance(canopy.getCenter(), point);
+      double dist = measure.distance(canopy.getCenter().getLengthSquared(), canopy.getCenter(), point);
       if (dist < t1)
         canopy.emitPoint(point, collector);
       pointStronglyBound = pointStronglyBound || (dist < t2);
@@ -195,7 +195,7 @@
     Canopy closest = null;
     boolean isCovered = false;
     for (Canopy canopy : canopies) {
-      double dist = measure.distance(canopy.getCenter(), point);
+      double dist = measure.distance(canopy.getCenter().getLengthSquared(), canopy.getCenter(), point);
       if (dist < t1) {
         isCovered = true;
         collector.collect(new Text(canopy.getIdentifier()), point);
@@ -308,6 +308,6 @@
    * @return if the point is covered
    */
   public boolean covers(Vector point) {
-    return measure.distance(center, point) < t1;
+    return measure.distance(center.getLengthSquared(), center, point) < t1;
   }
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/Cluster.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/Cluster.java?rev=788504&r1=788503&r2=788504&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/Cluster.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/Cluster.java Thu Jun 25 20:46:42 2009
@@ -225,7 +225,8 @@
   /**
    * Construct a new cluster with the given point as its center
    * 
-   * @param center the center point
+   * @param center
+   *            the center point
    */
   public Cluster(Vector center) {
     super();
@@ -300,8 +301,8 @@
       pointTotal = delta.clone();
       pointSquaredTotal = delta.times(delta);
     } else {
-      pointTotal = pointTotal.plus(delta);
-      pointSquaredTotal = pointSquaredTotal.plus(delta.times(delta));
+      delta.addTo(pointTotal);
+      delta.times(delta).addTo(pointSquaredTotal);
     }
   }
 
@@ -324,7 +325,7 @@
    */
   public boolean computeConvergence() {
     Vector centroid = computeCentroid();
-    converged = measure.distance(centroid, center) <= convergenceDelta;
+    converged = measure.distance(centroid.getLengthSquared(), centroid, center) <= convergenceDelta;
     return converged;
   }
 

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/DenseVector.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/DenseVector.java?rev=788504&r1=788503&r2=788504&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/DenseVector.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/DenseVector.java Thu Jun 25 20:46:42 2009
@@ -306,4 +306,38 @@
 
     return result;
   }
+
+  private Double lengthSquared = null; 
+
+  @Override
+  public double getLengthSquared() {
+  	if (lengthSquared != null) {
+  		return lengthSquared;
+  	}	
+
+  	double result = 0.0;
+  	for (int i = 0; i < values.length; i++) {
+  		result += values[i] * values[i];
+  	
+  	}
+  	lengthSquared = new Double(result);
+  	return result;
+  }
+
+  @Override
+  public double getDistanceSquared(Vector v) {
+  	double result = 0.0;
+  	double delta = 0.0;
+  	for (int i = 0; i < values.length; i++) {
+  		delta = values[i] - v.getQuick(i);  
+  		result += delta * delta;	
+  	}	
+  	return result;
+  }
+
+  @Override
+  public void addTo(Vector v) {
+    for (int i = 0; i < size(); i++)
+      v.setQuick(i, get(i) + v.get(i));  	
+  }
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/SparseVector.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/SparseVector.java?rev=788504&r1=788503&r2=788504&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/SparseVector.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/SparseVector.java Thu Jun 25 20:46:42 2009
@@ -20,15 +20,17 @@
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.util.NoSuchElementException;
 import java.util.Iterator;
+import java.util.NoSuchElementException;
 
 /**
  * Implements vector that only stores non-zero doubles
  */
 public class SparseVector extends AbstractVector {
 
-  /** For serialization purposes only. */
+  /**
+   * For serialization purposes only.
+   */
   public SparseVector() {
   }
 
@@ -36,6 +38,9 @@
 
   private int cardinality;
 
+  public SparseVector(int cardinality, int size) {
+    this(null, cardinality, size);
+  }
 
   public SparseVector(String name, int cardinality, int size) {
     super(name);
@@ -45,17 +50,17 @@
 
   public SparseVector(String name, int cardinality) {
     this(name, cardinality, cardinality / 8); // arbitrary estimate of
-                                              // 'sparseness'
+    // 'sparseness'
   }
 
   public SparseVector(int cardinality) {
     this(null, cardinality, cardinality / 8); // arbitrary estimate of
-                                              // 'sparseness'
+    // 'sparseness'
   }
 
   @Override
   protected Matrix matrixLike(int rows, int columns) {
-    int[] cardinality = { rows, columns };
+    int[] cardinality = {rows, columns};
     return new SparseRowMatrix(cardinality);
   }
 
@@ -105,19 +110,27 @@
 
   @Override
   public SparseVector like() {
-    return new SparseVector(cardinality);
+    int numValues = 256;
+    if (values != null) {
+      numValues = values.getNumMappings();
+    }
+    return new SparseVector(cardinality, numValues);
   }
 
   @Override
   public Vector like(int newCardinality) {
-    return new SparseVector(newCardinality);
+    int numValues = 256;
+    if (values != null) {
+      numValues = values.getNumMappings();
+    }
+    return new SparseVector(newCardinality, numValues);
   }
 
   /**
    * NOTE: this implementation reuses the Vector.Element instance for each call of next(). If you
    * need to preserve the instance, you need to make a copy of it
-   * @return an {@link org.apache.mahout.matrix.SparseVector.NonZeroIterator} over the Elements.
    *
+   * @return an {@link org.apache.mahout.matrix.SparseVector.NonZeroIterator} over the Elements.
    * @see #getElement(int)
    */
   public java.util.Iterator<Vector.Element> iterateNonZero() {
@@ -133,11 +146,11 @@
    * Indicate whether the two objects are the same or not. Two
    * {@link org.apache.mahout.matrix.Vector}s can be equal even if the
    * underlying implementation is not equal.
-   * 
+   *
    * @param o The object to compare
    * @return true if the objects have the same cell values and same name, false
    *         otherwise.
-   * 
+   *         <p/>
    *         * @see AbstractVector#strictEquivalence(Vector, Vector)
    * @see AbstractVector#equivalent(Vector, Vector)
    */
@@ -154,7 +167,7 @@
 
     if (that instanceof SparseVector) {
       return (values == null ? ((SparseVector) that).values == null : values
-          .equals(((SparseVector) that).values));
+              .equals(((SparseVector) that).values));
     } else {
       return equivalent(this, that);
     }
@@ -169,7 +182,7 @@
     return result;
   }
 
-  private class AllIterator implements java.util.Iterator<Vector.Element>{
+  private class AllIterator implements java.util.Iterator<Vector.Element> {
     private int offset = 0;
     private Element element = new Element(0);
 
@@ -194,6 +207,7 @@
   private class NonZeroIterator implements java.util.Iterator<Vector.Element> {
     private int offset = 0;
     private Element element = new Element(0);
+
     @Override
     public boolean hasNext() {
       return offset < values.getNumMappings();
@@ -243,7 +257,6 @@
   }
 
 
- 
   @Override
   public void write(DataOutput dataOutput) throws IOException {
     dataOutput.writeUTF(this.name == null ? "" : this.name);
@@ -258,7 +271,7 @@
       dataOutput.writeDouble(element.get());
       count++;
     }
-    assert(nde == count);
+    assert (nde == count);
   }
 
   @Override
@@ -271,10 +284,50 @@
     for (; i < size; i++) {
       values.set(dataInput.readInt(), dataInput.readDouble());
     }
-    assert(i == size);
+    assert (i == size);
     this.cardinality = cardinality;
     this.values = values;
   }
 
+  private Double lengthSquared = null;
+
+  @Override
+  public double getLengthSquared() {
+    if (lengthSquared != null) {
+      return lengthSquared;
+    }
+    double result = 0.0f;
+    for (double val : values.getValues()) {
+      result += val * val;
+    }
+    lengthSquared = new Double(result);
+    return result;
+  }
+
+  @Override
+  public double getDistanceSquared(Vector v) {
+    //TODO: Check sizes?
+
+    double result = 0.0f;
+    double delta = 0.0f;
+    double centroidValue = 0.0f;
+    Iterator<Vector.Element> iter = iterateNonZero();
+    while (iter.hasNext()) {
+      Vector.Element elt = iter.next();
+      centroidValue = v.getQuick(elt.index());
+      delta = elt.get() - centroidValue;
+      result += (delta * delta) - (centroidValue * centroidValue);
+    }
+    return result;
+  }
+
+  @Override
+  public void addTo(Vector v) {
+    Iterator<Vector.Element> iter = iterateNonZero();
+    while (iter.hasNext()) {
+      Vector.Element elt = iter.next();
+      v.setQuick(elt.index(), elt.get() + v.get(elt.index()));
+    }
+  }
 
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/Vector.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/Vector.java?rev=788504&r1=788503&r2=788504&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/Vector.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/Vector.java Thu Jun 25 20:46:42 2009
@@ -425,4 +425,23 @@
   // DoubleDoubleFunction map);
   // NewVector assign(Vector y, DoubleDoubleFunction function, IntArrayList
   // nonZeroIndexes);
+  
+
+  /**
+   * Return the sum of squares of all elements in the vector. Square root of this value is the length of the vector.
+   */
+  public abstract double getLengthSquared();
+    
+  /**
+   * Get the square of the distance between this vector and the other vector. 
+   * @param v
+   * @return
+   */
+  public abstract double getDistanceSquared(Vector v);
+  
+  /**
+   * Add the elements to the other vector and results are stored in that vector.
+   */
+  public abstract void addTo(Vector v);
+  
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/VectorView.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/VectorView.java?rev=788504&r1=788503&r2=788504&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/VectorView.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/VectorView.java Thu Jun 25 20:46:42 2009
@@ -21,6 +21,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.NoSuchElementException;
+import java.util.Iterator;
 
 /**
  * Implements subset view of a Vector
@@ -313,4 +314,35 @@
     result = 31 * result + cardinality;
     return result;
   }
+
+  @Override
+  public double getLengthSquared() {
+  	double result = 0.0;
+  	for (int i = 0; i < cardinality; i++) {
+  		double value = getQuick(i); 
+  		result +=  value * value;
+  	}
+  	return result;
+  }
+
+  @Override
+  public double getDistanceSquared(Vector v) {  	
+  	double result = 0.0;
+  	double delta = 0.0;
+  	for (int i = 0; i < cardinality; i++) {
+  		delta = getQuick(i) - v.getQuick(i);
+  		result += delta * delta;
+  	}
+  	return result;
+  }
+
+  @Override
+  public void addTo(Vector v) {
+    Iterator<Vector.Element> iter = iterateNonZero();
+    while (iter.hasNext()) {
+      Vector.Element elt = iter.next();
+  	  v.set(elt.index(), elt.get() + v.get(elt.index()));
+    }
+  }
+
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/CosineDistanceMeasure.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/CosineDistanceMeasure.java?rev=788504&r1=788503&r2=788504&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/CosineDistanceMeasure.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/CosineDistanceMeasure.java Thu Jun 25 20:46:42 2009
@@ -92,4 +92,9 @@
     return 1.0 - (dotProduct / denominator);
   }
 
+  @Override
+   public double distance(double centroidLengthSquare, Vector centroid, Vector v) {	 
+     return distance(centroid, v); // TODO
+   }
+
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/DistanceMeasure.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/DistanceMeasure.java?rev=788504&r1=788503&r2=788504&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/DistanceMeasure.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/DistanceMeasure.java Thu Jun 25 20:46:42 2009
@@ -36,5 +36,16 @@
    * @throws CardinalityException
    */
   double distance(Vector v1, Vector v2);
+  
+  /**
+   * Optimized version of distance metric for sparse vectors. This distance computation requires  
+   * operations proportional to the number of non-zero elements in the vector instead of the cardinality 
+   * of the vector.
+   * @param centroidLengthSquare Square of the length of centroid
+   * @param centroid Centroid vector
+   * @param v 
+   * @return
+   */
+  double distance(double centroidLengthSquare, Vector centroid, Vector v);
 
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/EuclideanDistanceMeasure.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/EuclideanDistanceMeasure.java?rev=788504&r1=788503&r2=788504&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/EuclideanDistanceMeasure.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/EuclideanDistanceMeasure.java Thu Jun 25 20:46:42 2009
@@ -17,60 +17,24 @@
 
 package org.apache.mahout.utils;
 
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.mahout.matrix.CardinalityException;
 import org.apache.mahout.matrix.Vector;
-import org.apache.mahout.utils.parameters.Parameter;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
 
 /**
  * This class implements a Euclidian distance metric by summing the square root
- * of the squared differences between each coordinate
+ * of the squared differences between each coordinate.
+ * <p/>
+ * If you don't care about the true distance and only need the values for comparison, then
+ * the base class, {@link SquaredEuclideanDistanceMeasure}, will be faster since it doesn't do
+ * the actual square root of the squared differences.
  */
-public class EuclideanDistanceMeasure implements DistanceMeasure {
-
+public class EuclideanDistanceMeasure extends SquaredEuclideanDistanceMeasure {
   @Override
-  public void configure(JobConf job) {
-    // nothing to do
-  }
-
-  @Override
-  public Collection<Parameter<?>> getParameters() {
-    return Collections.emptyList();
-  }
-
-  @Override
-  public void createParameters(String prefix, JobConf jobConf) {
-    // nothing to do
-  }
-
-  public static double distance(double[] p1, double[] p2) {
-    double result = 0.0;
-    for (int i = 0; i < p1.length; i++) {
-      double delta = p2[i] - p1[i];
-      result += delta * delta;
-    }
-    //TODO: Do we really need to return the square root?
-    return Math.sqrt(result);
+  public double distance(Vector v1, Vector v2) {
+    return Math.sqrt(super.distance(v1, v2));
   }
 
   @Override
-  public double distance(Vector v1, Vector v2) {
-    if (v1.size() != v2.size())
-      throw new CardinalityException();
-    double result = 0;
-    Vector vector = v1.plus(v2);
-    Iterator<Vector.Element> iter = vector.iterateNonZero();//this contains all non zero elements between the two
-    while (iter.hasNext()) {
-      Vector.Element e = iter.next();
-      double delta = v2.getQuick(e.index()) - v1.getQuick(e.index());
-      result += delta * delta;
-    }
-    //TODO: Do we really need to return the square root?
-    return Math.sqrt(result);
+  public double distance(double centroidLengthSquare, Vector centroid, Vector v) {
+    return Math.sqrt(super.distance(centroidLengthSquare, centroid, v));
   }
-
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/ManhattanDistanceMeasure.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/ManhattanDistanceMeasure.java?rev=788504&r1=788503&r2=788504&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/ManhattanDistanceMeasure.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/ManhattanDistanceMeasure.java Thu Jun 25 20:46:42 2009
@@ -68,4 +68,9 @@
     return result;
   }
 
+ @Override
+ public double distance(double centroidLengthSquare, Vector centroid, Vector v) {	 
+  return distance(centroid, v); // TODO
+ }
+ 
 }

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/SquaredEuclideanDistanceMeasure.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/SquaredEuclideanDistanceMeasure.java?rev=788504&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/SquaredEuclideanDistanceMeasure.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/SquaredEuclideanDistanceMeasure.java Thu Jun 25 20:46:42 2009
@@ -0,0 +1,88 @@
+package org.apache.mahout.utils;
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.mahout.utils.parameters.Parameter;
+import org.apache.mahout.matrix.Vector;
+import org.apache.mahout.matrix.CardinalityException;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+
+
+/**
+ * Like {@link org.apache.mahout.utils.EuclideanDistanceMeasure} but it does not take the square root.
+ * <p/>
+ * Thus, it is not actually the Euclidean Distance, but it is saves on computation when you only need the distance for
+ * comparison and don't care about the actual value as a distance.
+ *
+ **/
+public class SquaredEuclideanDistanceMeasure implements DistanceMeasure {
+
+  @Override
+  public void configure(JobConf job) {
+    // nothing to do
+  }
+
+  @Override
+  public Collection<Parameter<?>> getParameters() {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public void createParameters(String prefix, JobConf jobConf) {
+    // nothing to do
+  }
+
+  public static double distance(double[] p1, double[] p2) {
+    double result = 0.0;
+    for (int i = 0; i < p1.length; i++) {
+      double delta = p2[i] - p1[i];
+      result += delta * delta;
+    }
+
+    return result;
+  }
+
+  @Override
+  public double distance(Vector v1, Vector v2) {
+    if (v1.size() != v2.size())
+      throw new CardinalityException();
+    double result = 0;
+    Vector vector = v1.plus(v2);
+    Iterator<Vector.Element> iter = vector.iterateNonZero();//this contains all non zero elements between the two
+    while (iter.hasNext()) {
+      Vector.Element e = iter.next();
+      double delta = v2.getQuick(e.index()) - v1.getQuick(e.index());
+      result += delta * delta;
+    }
+
+    return result;
+  }
+
+  @Override
+  public double distance(double centroidLengthSquare, Vector centroid, Vector v) {
+	    if (centroid.size() != centroid.size())
+	        throw new CardinalityException();
+
+	    double result = centroidLengthSquare;
+	    result += v.getDistanceSquared(centroid);
+	    return result;
+  }
+}

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/TanimotoDistanceMeasure.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/TanimotoDistanceMeasure.java?rev=788504&r1=788503&r2=788504&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/TanimotoDistanceMeasure.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/TanimotoDistanceMeasure.java Thu Jun 25 20:46:42 2009
@@ -93,5 +93,10 @@
     return ((a2 + b2 - ab) / ab) - 1.0;
   }
 
+  @Override
+  public double distance(double centroidLengthSquare, Vector centroid, Vector v) {	 
+   return distance(centroid, v); // TODO
+  }
+
 }
 

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/WeightedEuclideanDistanceMeasure.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/WeightedEuclideanDistanceMeasure.java?rev=788504&r1=788503&r2=788504&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/WeightedEuclideanDistanceMeasure.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/WeightedEuclideanDistanceMeasure.java Thu Jun 25 20:46:42 2009
@@ -47,4 +47,9 @@
     return Math.sqrt(result);
   }
 
+  @Override
+  public double distance(double centroidLengthSquare, Vector centroid, Vector v) {	 
+   return distance(centroid, v); // TODO
+  }
+
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/WeightedManhattanDistanceMeasure.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/WeightedManhattanDistanceMeasure.java?rev=788504&r1=788503&r2=788504&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/WeightedManhattanDistanceMeasure.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/utils/WeightedManhattanDistanceMeasure.java Thu Jun 25 20:46:42 2009
@@ -51,4 +51,9 @@
     return result;
   }
 
+  @Override
+  public double distance(double centroidLengthSquare, Vector centroid, Vector v) {	 
+   return distance(centroid, v); // TODO
+  }
+
 }