You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by sr...@apache.org on 2010/04/21 09:41:56 UTC

svn commit: r936183 [1/3] - in /lucene/mahout/trunk: core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/ core/src/main/java/org/apache/mahout/clustering/canopy/ core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/ core/src/main/java/org/...

Author: srowen
Date: Wed Apr 21 07:41:55 2010
New Revision: 936183

URL: http://svn.apache.org/viewvc?rev=936183&view=rev
Log:
Huge patch for MAHOUT-379. See JIRA for more complete details.

Added:
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/NamedVectorWritable.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/math/VectorTest.java
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/NamedVector.java
    lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/AbstractTestVector.java   (contents, props changed)
      - copied, changed from r935497, lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/TestSparseVector.java
    lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/TestRandomAccessSparseVector.java
    lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/TestSequentialAccessSparseVector.java
Removed:
    lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/TestSparseVector.java
Modified:
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/ToUserVectorReducer.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/UserVectorToCooccurrenceReducer.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyClusterer.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyMapper.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansClusterMapper.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansClusterer.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansClusterMapper.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansClusterer.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/DenseVectorWritable.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/RandomAccessSparseVectorWritable.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/SequentialAccessSparseVectorWritable.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/VectorWritable.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/MatrixMultiplicationJob.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/TimesSquaredJob.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/decomposer/DistributedLanczosSolver.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/decomposer/EigenVector.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/decomposer/EigenVerificationJob.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/canopy/TestCanopyCreation.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/fuzzykmeans/TestFuzzyKmeansClustering.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestRandomSeedGenerator.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/lda/TestMapReduce.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/math/hadoop/TestDistributedRowMatrix.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayDirichlet.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/fuzzykmeans/DisplayFuzzyKMeans.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/kmeans/DisplayKMeans.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/InputMapper.java
    lucene/mahout/trunk/examples/src/test/java/org/apache/mahout/ga/watchmaker/cd/CDCrossoverTest.java
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/AbstractMatrix.java
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/AbstractVector.java
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/DenseVector.java
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/OrderedIntDoubleMapping.java
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/RandomAccessSparseVector.java
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/SequentialAccessSparseVector.java
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/Vector.java
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/VectorView.java
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/decomposer/hebbian/TrainingState.java
    lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/TestDenseVector.java
    lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/TestVectorView.java
    lucene/mahout/trunk/math/src/test/java/org/apache/mahout/math/VectorTest.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/VectorHelper.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/common/PartialVectorMergeReducer.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/lucene/LuceneIterable.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/term/TFPartialVectorReducer.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/tfidf/TFIDFPartialVectorReducer.java

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/ToUserVectorReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/ToUserVectorReducer.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/ToUserVectorReducer.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/ToUserVectorReducer.java Wed Apr 21 07:41:55 2010
@@ -61,7 +61,6 @@ public final class ToUserVectorReducer e
   
   public static final int MAX_PREFS_CONSIDERED = 20;
   
-  private final VectorWritable vectorWritable = new VectorWritable();
   private boolean booleanData;
 
   @Override
@@ -102,9 +101,9 @@ public final class ToUserVectorReducer e
         }
         userVector = filteredVector;
       }
-      
-      vectorWritable.set(userVector);
-      output.collect(userID, vectorWritable);
+
+      VectorWritable writable = new VectorWritable(userVector);
+      output.collect(userID, writable);
     }
   }
   

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/UserVectorToCooccurrenceReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/UserVectorToCooccurrenceReducer.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/UserVectorToCooccurrenceReducer.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/UserVectorToCooccurrenceReducer.java Wed Apr 21 07:41:55 2010
@@ -31,9 +31,7 @@ import org.apache.mahout.math.VectorWrit
 
 public final class UserVectorToCooccurrenceReducer extends MapReduceBase implements
     Reducer<IntWritable,IntWritable,IntWritable,VectorWritable> {
-  
-  private final VectorWritable vectorWritable = new VectorWritable();
-  
+
   @Override
   public void reduce(IntWritable index1,
                      Iterator<IntWritable> index2s,
@@ -52,8 +50,8 @@ public final class UserVectorToCooccurre
           element.set(0.0);
         }
       }
-      vectorWritable.set(cooccurrenceRow);
-      output.collect(index1, vectorWritable);
+      VectorWritable writable = new VectorWritable(cooccurrenceRow);
+      output.collect(index1, writable);
     }
   }
   

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyClusterer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyClusterer.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyClusterer.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyClusterer.java Wed Apr 21 07:41:55 2010
@@ -169,12 +169,11 @@ public class CanopyClusterer {
     double minDist = Double.MAX_VALUE;
     Canopy closest = null;
     boolean isCovered = false;
-    VectorWritable vw = new VectorWritable();
     for (Canopy canopy : canopies) {
       double dist = measure.distance(canopy.getCenter().getLengthSquared(), canopy.getCenter(), point);
       if (dist < t1) {
         isCovered = true;
-        vw.set(point);
+        VectorWritable vw = new VectorWritable(point);
         collector.collect(new Text(canopy.getIdentifier()), vw);
         reporter.setStatus("Emit Canopy ID:" + canopy.getIdentifier());
       } else if (dist < minDist) {
@@ -184,8 +183,8 @@ public class CanopyClusterer {
     }
     // if the point is not contained in any canopies (due to canopy centroid
     // clustering), emit the point to the closest covering canopy.
-    vw.set(point);
     if (!isCovered) {
+      VectorWritable vw = new VectorWritable(point);
       collector.collect(new Text(closest.getIdentifier()), vw);
       reporter.setStatus("Emit Closest Canopy ID:" + closest.getIdentifier());
     }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyMapper.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyMapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyMapper.java Wed Apr 21 07:41:55 2010
@@ -59,8 +59,7 @@ public class CanopyMapper extends MapRed
   public void close() throws IOException {
     for (Canopy canopy : canopies) {
       Vector centroid = canopy.computeCentroid();
-      VectorWritable vw = new VectorWritable();
-      vw.set(centroid);
+      VectorWritable vw = new VectorWritable(centroid);
       outputCollector.collect(new Text("centroid"), vw);
     }
     super.close();

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansClusterMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansClusterMapper.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansClusterMapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansClusterMapper.java Wed Apr 21 07:41:55 2010
@@ -28,6 +28,7 @@ import org.apache.hadoop.mapred.MapReduc
 import org.apache.hadoop.mapred.Mapper;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reporter;
+import org.apache.mahout.math.NamedVector;
 import org.apache.mahout.math.VectorWritable;
 
 public class FuzzyKMeansClusterMapper extends MapReduceBase implements
@@ -41,7 +42,7 @@ public class FuzzyKMeansClusterMapper ex
                   VectorWritable point,
                   OutputCollector<Text,FuzzyKMeansOutput> output,
                   Reporter reporter) throws IOException {
-    clusterer.outputPointWithClusterProbabilities(key.toString(), point.get(), clusters, output);
+    clusterer.outputPointWithClusterProbabilities(key.toString(), (NamedVector) point.get(), clusters, output);
   }
   
   /**

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansClusterer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansClusterer.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansClusterer.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansClusterer.java Wed Apr 21 07:41:55 2010
@@ -25,6 +25,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.mahout.common.distance.DistanceMeasure;
+import org.apache.mahout.math.NamedVector;
 import org.apache.mahout.math.Vector;
 
 public class FuzzyKMeansClusterer {
@@ -123,7 +124,7 @@ public class FuzzyKMeansClusterer {
    *          the OutputCollector to emit into
    */
   public void outputPointWithClusterProbabilities(String key,
-                                                  Vector point,
+                                                  NamedVector point,
                                                   List<SoftCluster> clusters,
                                                   OutputCollector<Text,FuzzyKMeansOutput> output) throws IOException {
     
@@ -137,8 +138,7 @@ public class FuzzyKMeansClusterer {
       double probWeight = computeProbWeight(clusterDistanceList.get(i), clusterDistanceList);
       fOutput.add(i, clusters.get(i), probWeight);
     }
-    String name = point.getName();
-    output.collect(new Text((name != null) && (name.length() != 0) ? name : point.asFormatString()), fOutput);
+    output.collect(new Text(point.getName()), fOutput);
   }
   
   /** Computes the probability of a point belonging to a cluster */
@@ -188,7 +188,7 @@ public class FuzzyKMeansClusterer {
    * @param numIter
    *          the maximum number of iterations
    */
-  public static List<List<SoftCluster>> clusterPoints(List<Vector> points,
+  public static List<List<SoftCluster>> clusterPoints(List<NamedVector> points,
                                                       List<SoftCluster> clusters,
                                                       DistanceMeasure measure,
                                                       double threshold,
@@ -221,7 +221,7 @@ public class FuzzyKMeansClusterer {
    *          the List<Cluster> clusters
    * @return
    */
-  public static boolean runFuzzyKMeansIteration(List<Vector> points,
+  public static boolean runFuzzyKMeansIteration(List<NamedVector> points,
                                                 List<SoftCluster> clusterList,
                                                 FuzzyKMeansClusterer clusterer) {
     // for each

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansClusterMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansClusterMapper.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansClusterMapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansClusterMapper.java Wed Apr 21 07:41:55 2010
@@ -29,6 +29,7 @@ import org.apache.hadoop.mapred.Mapper;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.mahout.common.distance.DistanceMeasure;
+import org.apache.mahout.math.NamedVector;
 import org.apache.mahout.math.VectorWritable;
 
 public class KMeansClusterMapper extends MapReduceBase implements
@@ -42,18 +43,7 @@ public class KMeansClusterMapper extends
                   VectorWritable point,
                   OutputCollector<Text,Text> output,
                   Reporter reporter) throws IOException {
-    this.clusterer.outputPointWithClusterInfo(point.get(), clusters, output);
-  }
-  
-  /**
-   * Configure the mapper by providing its clusters. Used by unit tests.
-   * 
-   * @param clusters
-   *          a List<Cluster>
-   */
-  void config(List<Cluster> clusters) {
-    this.clusters.clear();
-    this.clusters.addAll(clusters);
+    clusterer.outputPointWithClusterInfo((NamedVector) point.get(), clusters, output);
   }
   
   @Override

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansClusterer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansClusterer.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansClusterer.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansClusterer.java Wed Apr 21 07:41:55 2010
@@ -23,6 +23,7 @@ import java.util.List;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.mahout.common.distance.DistanceMeasure;
+import org.apache.mahout.math.NamedVector;
 import org.apache.mahout.math.Vector;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -78,7 +79,7 @@ public class KMeansClusterer {
     output.collect(new Text(nearestCluster.getIdentifier()), new KMeansInfo(1, point));
   }
   
-  public void outputPointWithClusterInfo(Vector point,
+  public void outputPointWithClusterInfo(NamedVector point,
                                          List<Cluster> clusters,
                                          OutputCollector<Text,Text> output) throws IOException {
     Cluster nearestCluster = null;
@@ -92,9 +93,7 @@ public class KMeansClusterer {
       }
     }
     
-    String name = point.getName();
-    String key = (name != null) && (name.length() != 0) ? name : point.asFormatString();
-    output.collect(new Text(key), new Text(String.valueOf(nearestCluster.getId())));
+    output.collect(new Text(point.getName()), new Text(String.valueOf(nearestCluster.getId())));
   }
   
   /**
@@ -110,7 +109,7 @@ public class KMeansClusterer {
    * @param maxIter
    *          the maximum number of iterations
    */
-  public static List<List<Cluster>> clusterPoints(List<Vector> points,
+  public static List<List<Cluster>> clusterPoints(List<NamedVector> points,
                                                   List<Cluster> clusters,
                                                   DistanceMeasure measure,
                                                   int maxIter,
@@ -145,7 +144,7 @@ public class KMeansClusterer {
    *          a DistanceMeasure to use
    * @return
    */
-  public static boolean runKMeansIteration(List<Vector> points,
+  public static boolean runKMeansIteration(List<NamedVector> points,
                                            List<Cluster> clusters,
                                            DistanceMeasure measure,
                                            double distanceThreshold) {

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/DenseVectorWritable.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/DenseVectorWritable.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/DenseVectorWritable.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/DenseVectorWritable.java Wed Apr 21 07:41:55 2010
@@ -17,52 +17,38 @@
 
 package org.apache.mahout.math;
 
-import org.apache.hadoop.io.Writable;
-
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.util.Iterator;
 
-public class DenseVectorWritable extends DenseVector implements Writable {
+public class DenseVectorWritable extends VectorWritable {
 
-  public DenseVectorWritable() {
-    
+  public DenseVectorWritable(DenseVector vector) {
+    super(vector);
   }
 
-  public DenseVectorWritable(DenseVector v) {
-    setName(v.getName());
-    values = v.values;
-    lengthSquared = v.lengthSquared;
+  public DenseVectorWritable() {
   }
 
   @Override
   public void write(DataOutput dataOutput) throws IOException {
-    dataOutput.writeUTF(getClass().getName());
-    dataOutput.writeUTF(this.getName() == null ? "" : this.getName());
-    dataOutput.writeInt(size());
-    dataOutput.writeDouble(lengthSquared);
-    Iterator<Vector.Element> iter = iterateAll();
-    while (iter.hasNext()) {
-      Vector.Element element = iter.next();
+    DenseVector denseVector = (DenseVector) get();
+    dataOutput.writeInt(denseVector.size());
+    dataOutput.writeDouble(denseVector.getLengthSquared());
+    for (Vector.Element element : denseVector) {
       dataOutput.writeDouble(element.get());
     }
   }
 
   @Override
-  public void readFields(DataInput dataInput) throws IOException {
-    String className = dataInput.readUTF();
-    if(className.equals(getClass().getName())) {
-      this.setName(dataInput.readUTF());
-    } else {
-      setName(className); // we have already read the class name in VectorWritable
-    }
-    double[] values = new double[dataInput.readInt()];
-    lengthSquared = dataInput.readDouble();
+  public void readFields(DataInput in) throws IOException {
+    int size = in.readInt();
+    double[] values = new double[size];
+    double lengthSquared = in.readDouble();
     for (int i = 0; i < values.length; i++) {
-      values[i] = dataInput.readDouble();
+      values[i] = in.readDouble();
     }
-    this.values = values;
+    set(new DenseVector(values));
   }
   
 }

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/NamedVectorWritable.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/NamedVectorWritable.java?rev=936183&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/NamedVectorWritable.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/NamedVectorWritable.java Wed Apr 21 07:41:55 2010
@@ -0,0 +1,50 @@
+/**
+ * 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.
+ */
+
+package org.apache.mahout.math;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class NamedVectorWritable extends VectorWritable {
+
+  public NamedVectorWritable(NamedVector vector) {
+    super(vector);
+  }
+
+  public NamedVectorWritable() {
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    NamedVector namedVector = (NamedVector) get();
+    out.writeUTF(namedVector.getName());
+    VectorWritable writable = new VectorWritable(namedVector.getDelegate());
+    writable.write(out);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    String name = in.readUTF();
+    VectorWritable writable = new VectorWritable();
+    writable.readFields(in);
+    Vector delegate = writable.get();
+    set(new NamedVector(delegate, name));
+  }
+
+}

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/RandomAccessSparseVectorWritable.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/RandomAccessSparseVectorWritable.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/RandomAccessSparseVectorWritable.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/RandomAccessSparseVectorWritable.java Wed Apr 21 07:41:55 2010
@@ -17,7 +17,6 @@
 
 package org.apache.mahout.math;
 
-import org.apache.hadoop.io.Writable;
 import org.apache.mahout.math.map.OpenIntDoubleHashMap;
 
 import java.io.DataInput;
@@ -25,57 +24,39 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Iterator;
 
+public class RandomAccessSparseVectorWritable extends VectorWritable {
 
-public class RandomAccessSparseVectorWritable extends RandomAccessSparseVector implements Writable {
-
-  public RandomAccessSparseVectorWritable(Vector v) {
-    super(v);
+  public RandomAccessSparseVectorWritable(RandomAccessSparseVector vector) {
+    super(vector);
   }
 
   public RandomAccessSparseVectorWritable() {
-    
   }
 
   @Override
   public void write(DataOutput dataOutput) throws IOException {
-    dataOutput.writeUTF(getClass().getName());
-    dataOutput.writeUTF(this.getName() == null ? "" : this.getName());
-    int nde = getNumNondefaultElements();
-    dataOutput.writeInt(size());
-    dataOutput.writeInt(nde);
-    Iterator<Vector.Element> iter = iterateNonZero();
-    int count = 0;
+    RandomAccessSparseVector randomVector = (RandomAccessSparseVector) get();
+    dataOutput.writeInt(randomVector.size());
+    dataOutput.writeInt(randomVector.getNumNondefaultElements());
+    Iterator<Vector.Element> iter = randomVector.iterateNonZero();
     while (iter.hasNext()) {
       Vector.Element element = iter.next();
       dataOutput.writeInt(element.index());
       dataOutput.writeDouble(element.get());
-      count++;
     }
-    assert (nde == count);
   }
 
   @Override
   public void readFields(DataInput dataInput) throws IOException {
-    String className = dataInput.readUTF();
-    if(className.equals(getClass().getName())) {
-      this.setName(dataInput.readUTF());
-    } else {
-      setName(className); // we have already read the class name in VectorWritable
-    }
-    size = dataInput.readInt();
-    int cardinality = dataInput.readInt();
-    OpenIntDoubleHashMap values = new OpenIntDoubleHashMap(cardinality);
-    int i = 0;
-    while (i < cardinality) {
+    int size = dataInput.readInt();
+    int numNonDefaultElements = dataInput.readInt();
+    OpenIntDoubleHashMap values = new OpenIntDoubleHashMap(numNonDefaultElements);
+    for (int i = 0; i < numNonDefaultElements; i++) {
       int index = dataInput.readInt();
       double value = dataInput.readDouble();
       values.put(index, value);
-      i++;
     }
-    assert (i == cardinality);
-    this.values = values;
+    set(new RandomAccessSparseVector(size, values));
   }
 
-  
-
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/SequentialAccessSparseVectorWritable.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/SequentialAccessSparseVectorWritable.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/SequentialAccessSparseVectorWritable.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/SequentialAccessSparseVectorWritable.java Wed Apr 21 07:41:55 2010
@@ -22,52 +22,39 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Iterator;
 
-import org.apache.hadoop.io.Writable;
-
-public class SequentialAccessSparseVectorWritable extends SequentialAccessSparseVector implements Writable {
+public class SequentialAccessSparseVectorWritable extends VectorWritable {
 
   public SequentialAccessSparseVectorWritable(SequentialAccessSparseVector vector) {
     super(vector);
   }
 
   public SequentialAccessSparseVectorWritable() {
-    
   }
 
   @Override
-  public void write(DataOutput dataOutput) throws IOException {
-    dataOutput.writeUTF(getClass().getName());
-    dataOutput.writeUTF(getName() == null ? "" : getName());
-    dataOutput.writeInt(size());
-    int nde = getNumNondefaultElements();
-    dataOutput.writeInt(nde);
-    Iterator<Element> iter = iterateNonZero();
-    int count = 0;
+  public void write(DataOutput out) throws IOException {
+    SequentialAccessSparseVector sequentialVector = (SequentialAccessSparseVector) get();
+    out.writeInt(sequentialVector.size());
+    out.writeInt(sequentialVector.getNumNondefaultElements());
+    Iterator<Vector.Element> iter = sequentialVector.iterateNonZero();
     while (iter.hasNext()) {
-      Element element = iter.next();
-      dataOutput.writeInt(element.index());
-      dataOutput.writeDouble(element.get());
-      count++;
+      Vector.Element element = iter.next();
+      out.writeInt(element.index());
+      out.writeDouble(element.get());
     }
-    assert (nde == count);
   }
 
   @Override
-  public void readFields(DataInput dataInput) throws IOException {
-    String className = dataInput.readUTF();
-    if(className.equals(getClass().getName())) {
-      setName(dataInput.readUTF());
-    } else {
-      setName(className); // we have already read the class name in VectorWritable
-    }
-    size = dataInput.readInt();
-    int nde = dataInput.readInt();
-    OrderedIntDoubleMapping values = new OrderedIntDoubleMapping(nde);
-    for (int i = 0; i < nde; i++) {
-      values.set(dataInput.readInt(), dataInput.readDouble());
+  public void readFields(DataInput in) throws IOException {
+    int size = in.readInt();
+    int numNonDefaultElements = in.readInt();
+    OrderedIntDoubleMapping values = new OrderedIntDoubleMapping(numNonDefaultElements);
+    for (int i = 0; i < numNonDefaultElements; i++) {
+      int index = in.readInt();
+      double value = in.readDouble();
+      values.set(index, value);
     }
-    this.values = values;
+    set(new SequentialAccessSparseVector(size, values));
   }
 
-
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/VectorWritable.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/VectorWritable.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/VectorWritable.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/VectorWritable.java Wed Apr 21 07:41:55 2010
@@ -23,48 +23,60 @@ import org.apache.hadoop.io.Writable;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
 
 public class VectorWritable extends Configured implements Writable {
 
   private Vector vector;
 
-  public Vector get() {
-    return vector;
+  public VectorWritable() {
   }
 
-  public void set(Vector vector) {
+  public VectorWritable(Vector vector) {
     this.vector = vector;
   }
 
-  public VectorWritable() {
+  public Vector get() {
+    return vector;
   }
 
-  public VectorWritable(Vector v) {
-    vector = v;
+  protected void set(Vector vector) {
+    this.vector = vector;
   }
 
   @Override
   public void write(DataOutput out) throws IOException {
-    Writable w;
-    if (vector instanceof Writable) {
-      w = (Writable) vector;
-    } else if(vector instanceof RandomAccessSparseVector) {
-      w = new RandomAccessSparseVectorWritable(vector);
-    } else if(vector instanceof SequentialAccessSparseVector) {
-      w = new SequentialAccessSparseVectorWritable((SequentialAccessSparseVector)vector);
-    } else {
-      w = new DenseVectorWritable(new DenseVector(vector));
+    VectorWritable writable;
+    Class<? extends Vector> vectorClass = vector.getClass();
+    String writableClassName = vectorClass.getName() + "Writable";
+    try {
+      Class<? extends VectorWritable> vectorWritableClass =
+          Class.forName(writableClassName).asSubclass(VectorWritable.class);
+      writable = vectorWritableClass.getConstructor(vectorClass).newInstance(vector);
+    } catch (ClassNotFoundException cnfe) {
+      throw new IOException(cnfe);
+    } catch (NoSuchMethodException nsme) {
+      throw new IOException(nsme);
+    } catch (InvocationTargetException ite) {
+      throw new IOException(ite);
+    } catch (InstantiationException ie) {
+      throw new IOException(ie);
+    } catch (IllegalAccessException iae) {
+      throw new IOException(iae);
     }
-    w.write(out);
+    out.writeUTF(writableClassName);
+    writable.write(out);
   }
 
   @Override
   public void readFields(DataInput in) throws IOException {
+    String writableClassName = in.readUTF();
     try {
-      String vectorClassName = in.readUTF();
-      Class<? extends Vector> vectorClass = Class.forName(vectorClassName).asSubclass(Vector.class);
-      vector = vectorClass.newInstance();
-      ((Writable)vector).readFields(in);
+      Class<? extends VectorWritable> writableClass =
+          Class.forName(writableClassName).asSubclass(VectorWritable.class);
+      VectorWritable writable = writableClass.getConstructor().newInstance();
+      writable.readFields(in);
+      vector = writable.get();
     } catch (ClassNotFoundException cnfe) {
       throw new IOException(cnfe);
     } catch (ClassCastException cce) {
@@ -73,6 +85,10 @@ public class VectorWritable extends Conf
       throw new IOException(ie);
     } catch (IllegalAccessException iae) {
       throw new IOException(iae);
+    } catch (NoSuchMethodException nsme) {
+      throw new IOException(nsme);
+    } catch (InvocationTargetException ite) {
+      throw new IOException(ite);
     }
   }
 

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/MatrixMultiplicationJob.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/MatrixMultiplicationJob.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/MatrixMultiplicationJob.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/MatrixMultiplicationJob.java Wed Apr 21 07:41:55 2010
@@ -120,7 +120,6 @@ public class MatrixMultiplicationJob ext
 
     private int outCardinality;
     private final IntWritable row = new IntWritable();
-    private final VectorWritable outVector = new VectorWritable();
 
     @Override
     public void configure(JobConf conf) {
@@ -140,7 +139,7 @@ public class MatrixMultiplicationJob ext
       while(it.hasNext()) {
         Vector.Element e = it.next();
         row.set(e.index());
-        outVector.set(outFrag.times(e.get()));
+        VectorWritable outVector = new VectorWritable(outFrag.times(e.get()));
         out.collect(row, outVector);
       }
     }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/TimesSquaredJob.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/TimesSquaredJob.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/TimesSquaredJob.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/TimesSquaredJob.java Wed Apr 21 07:41:55 2010
@@ -40,8 +40,6 @@ import org.apache.mahout.math.Sequential
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.math.VectorWritable;
 import org.apache.mahout.math.function.Functions;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.net.URI;
@@ -49,8 +47,6 @@ import java.util.Iterator;
 
 public class TimesSquaredJob {
 
-  private static final Logger log = LoggerFactory.getLogger(TimesSquaredJob.class);
-
   public static final String INPUT_VECTOR = "DistributedMatrix.times.inputVector";
   public static final String IS_SPARSE_OUTPUT = "DistributedMatrix.times.outputVector.sparse";
   public static final String OUTPUT_VECTOR_DIMENSION = "DistributedMatrix.times.output.dimension";
@@ -97,7 +93,7 @@ public class TimesSquaredJob {
                                                   Class<? extends TimesSquaredMapper> mapClass,
                                                   Class<? extends VectorSummingReducer> redClass) throws IOException {
     JobConf conf = new JobConf(TimesSquaredJob.class);
-    conf.setJobName("TimesSquaredJob: " + matrixInputPath + " timesSquared(" + v.getName() + ')');
+    conf.setJobName("TimesSquaredJob: " + matrixInputPath);
     FileSystem fs = FileSystem.get(conf);
     matrixInputPath = fs.makeQualified(matrixInputPath);
     outputVectorPathBase = fs.makeQualified(outputVectorPathBase);

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/decomposer/DistributedLanczosSolver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/decomposer/DistributedLanczosSolver.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/decomposer/DistributedLanczosSolver.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/decomposer/DistributedLanczosSolver.java Wed Apr 21 07:41:55 2010
@@ -102,12 +102,10 @@ public class DistributedLanczosSolver ex
     Configuration conf = getConf();
     FileSystem fs = FileSystem.get(conf);
     SequenceFile.Writer seqWriter = new SequenceFile.Writer(fs, conf, path, IntWritable.class, VectorWritable.class);
-    VectorWritable vw = new VectorWritable();
     IntWritable iw = new IntWritable();
     for(int i=0; i<eigenVectors.numRows() - 1; i++) {
       Vector v = eigenVectors.getRow(i);
-      v.setName("eigenVector" + i + ", eigenvalue = " + eigenValues.get(i));
-      vw.set(v);
+      VectorWritable vw = new VectorWritable(v);
       iw.set(i);
       seqWriter.append(iw, vw);
     }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/decomposer/EigenVector.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/decomposer/EigenVector.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/decomposer/EigenVector.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/decomposer/EigenVector.java Wed Apr 21 07:41:55 2010
@@ -29,9 +29,11 @@ public class EigenVector extends DenseVe
   private static final Pattern EQUAL_PATTERN = Pattern.compile(" = ");
   private static final Pattern PIPE_PATTERN = Pattern.compile("|");
 
+  private final String name;
+
   public EigenVector(DenseVector v, double eigenValue, double cosAngleError, int order) {
     super(v, false);
-    setName("e|" + order +"| = |"+eigenValue+"|, err = "+cosAngleError);
+    name = "e|" + order + "| = |" + eigenValue + "|, err = " + cosAngleError;
   }
 
   public double getEigenValue() {
@@ -48,7 +50,7 @@ public class EigenVector extends DenseVe
 
   protected double[] parseMetaData() {
     double[] m = new double[3];
-    String[] s = EQUAL_PATTERN.split(getName());
+    String[] s = EQUAL_PATTERN.split(name);
     m[0] = Double.parseDouble(PIPE_PATTERN.split(s[0])[1]);
     m[1] = Double.parseDouble(PIPE_PATTERN.split(s[1])[1]);
     m[2] = Double.parseDouble(s[2].substring(1));

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/decomposer/EigenVerificationJob.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/decomposer/EigenVerificationJob.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/decomposer/EigenVerificationJob.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/decomposer/EigenVerificationJob.java Wed Apr 21 07:41:55 2010
@@ -183,7 +183,6 @@ public class EigenVerificationJob extend
     Configuration conf = getConf();
     FileSystem fs = FileSystem.get(conf);
     SequenceFile.Writer seqWriter = new SequenceFile.Writer(fs, conf, path, IntWritable.class, VectorWritable.class);
-    VectorWritable vw = new VectorWritable();
     IntWritable iw = new IntWritable();
     for(Map.Entry<MatrixSlice,EigenStatus> pruneSlice : prunedEigenMeta) {
       MatrixSlice s = pruneSlice.getKey();
@@ -192,8 +191,8 @@ public class EigenVerificationJob extend
                                        meta.getEigenValue(),
                                        Math.abs(1-meta.getCosAngle()),
                                        s.index());
-      log.info("appending " + ev.getName() + " to " + path.toString());
-      vw.set(ev);
+      log.info("appending {} to {}", ev, path);
+      VectorWritable vw = new VectorWritable(ev);
       iw.set(s.index());
       seqWriter.append(iw, vw);
     }

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/canopy/TestCanopyCreation.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/canopy/TestCanopyCreation.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/canopy/TestCanopyCreation.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/canopy/TestCanopyCreation.java Wed Apr 21 07:41:55 2010
@@ -62,22 +62,20 @@ public class TestCanopyCreation extends 
   
   private FileSystem fs;
   
-  private static List<VectorWritable> getPointsWritable(double[][] raw) {
+  private static List<VectorWritable> getPointsWritable() {
     List<VectorWritable> points = new ArrayList<VectorWritable>();
-    int i = 0;
     for (double[] fr : raw) {
-      Vector vec = new RandomAccessSparseVector(String.valueOf(i++), fr.length);
+      Vector vec = new RandomAccessSparseVector(fr.length);
       vec.assign(fr);
       points.add(new VectorWritable(vec));
     }
     return points;
   }
   
-  private static List<Vector> getPoints(double[][] raw) {
+  private static List<Vector> getPoints() {
     List<Vector> points = new ArrayList<Vector>();
-    int i = 0;
     for (double[] fr : raw) {
-      Vector vec = new RandomAccessSparseVector(String.valueOf(i++), fr.length);
+      Vector vec = new RandomAccessSparseVector(fr.length);
       vec.assign(fr);
       points.add(vec);
     }
@@ -145,9 +143,9 @@ public class TestCanopyCreation extends 
     fs = FileSystem.get(conf);
     rmr("output");
     rmr("testdata");
-    referenceManhattan = CanopyClusterer.createCanopies(getPoints(raw), manhattanDistanceMeasure, 3.1, 2.1);
+    referenceManhattan = CanopyClusterer.createCanopies(getPoints(), manhattanDistanceMeasure, 3.1, 2.1);
     manhattanCentroids = CanopyClusterer.calculateCentroids(referenceManhattan);
-    referenceEuclidean = CanopyClusterer.createCanopies(getPoints(raw), euclideanDistanceMeasure, 3.1, 2.1);
+    referenceEuclidean = CanopyClusterer.createCanopies(getPoints(), euclideanDistanceMeasure, 3.1, 2.1);
     euclideanCentroids = CanopyClusterer.calculateCentroids(referenceEuclidean);
   }
   
@@ -193,7 +191,7 @@ public class TestCanopyCreation extends 
   
   /** Story: User can cluster points without instantiating them all in memory at once */
   public void testIterativeManhattan() throws Exception {
-    List<Vector> points = getPoints(raw);
+    List<Vector> points = getPoints();
     List<Canopy> canopies = CanopyClusterer.createCanopies(points, new ManhattanDistanceMeasure(), 3.1, 2.1);
     System.out.println("testIterativeManhattan");
     printCanopies(canopies);
@@ -202,7 +200,7 @@ public class TestCanopyCreation extends 
   
   /** Story: User can cluster points without instantiating them all in memory at once */
   public void testIterativeEuclidean() throws Exception {
-    List<Vector> points = getPoints(raw);
+    List<Vector> points = getPoints();
     List<Canopy> canopies = CanopyClusterer.createCanopies(points, new EuclideanDistanceMeasure(), 3.1, 2.1);
     
     System.out.println("testIterativeEuclidean");
@@ -224,7 +222,7 @@ public class TestCanopyCreation extends 
     mapper.configure(conf);
     
     DummyOutputCollector<Text,VectorWritable> collector = new DummyOutputCollector<Text,VectorWritable>();
-    List<VectorWritable> points = getPointsWritable(raw);
+    List<VectorWritable> points = getPointsWritable();
     // map the data
     for (VectorWritable point : points) {
       mapper.map(new Text(), point, collector, new DummyReporter());
@@ -254,7 +252,7 @@ public class TestCanopyCreation extends 
     mapper.configure(conf);
     
     DummyOutputCollector<Text,VectorWritable> collector = new DummyOutputCollector<Text,VectorWritable>();
-    List<VectorWritable> points = getPointsWritable(raw);
+    List<VectorWritable> points = getPointsWritable();
     // map the data
     for (VectorWritable point : points) {
       mapper.map(new Text(), point, collector, new DummyReporter());
@@ -284,7 +282,7 @@ public class TestCanopyCreation extends 
     reducer.configure(conf);
     
     DummyOutputCollector<Text,Canopy> collector = new DummyOutputCollector<Text,Canopy>();
-    List<VectorWritable> points = getPointsWritable(raw);
+    List<VectorWritable> points = getPointsWritable();
     reducer.reduce(new Text("centroid"), points.iterator(), collector, new DummyReporter());
     reducer.close();
     Set<String> keys = collector.getKeys();
@@ -313,7 +311,7 @@ public class TestCanopyCreation extends 
     reducer.configure(conf);
     
     DummyOutputCollector<Text,Canopy> collector = new DummyOutputCollector<Text,Canopy>();
-    List<VectorWritable> points = getPointsWritable(raw);
+    List<VectorWritable> points = getPointsWritable();
     reducer.reduce(new Text("centroid"), points.iterator(), collector, new DummyReporter());
     reducer.close();
     Set<String> keys = collector.getKeys();
@@ -333,7 +331,7 @@ public class TestCanopyCreation extends 
    * ManhattanDistanceMeasure.
    */
   public void testCanopyGenManhattanMR() throws Exception {
-    List<VectorWritable> points = getPointsWritable(raw);
+    List<VectorWritable> points = getPointsWritable();
     File testData = new File("testdata");
     if (!testData.exists()) {
       testData.mkdir();
@@ -370,7 +368,7 @@ public class TestCanopyCreation extends 
    * EuclideanDistanceMeasure.
    */
   public void testCanopyGenEuclideanMR() throws Exception {
-    List<VectorWritable> points = getPointsWritable(raw);
+    List<VectorWritable> points = getPointsWritable();
     File testData = new File("testdata");
     if (!testData.exists()) {
       testData.mkdir();
@@ -416,7 +414,7 @@ public class TestCanopyCreation extends 
       canopies.add(new Canopy(centroid, nextCanopyId++));
     }
     mapper.config(canopies);
-    List<VectorWritable> points = getPointsWritable(raw);
+    List<VectorWritable> points = getPointsWritable();
     // map the data
     for (VectorWritable point : points) {
       mapper.map(new Text(), point, collector, new DummyReporter());
@@ -459,7 +457,7 @@ public class TestCanopyCreation extends 
       canopies.add(new Canopy(centroid, nextCanopyId++));
     }
     mapper.config(canopies);
-    List<VectorWritable> points = getPointsWritable(raw);
+    List<VectorWritable> points = getPointsWritable();
     // map the data
     for (VectorWritable point : points) {
       mapper.map(new Text(), point, collector, new DummyReporter());
@@ -493,7 +491,7 @@ public class TestCanopyCreation extends 
       canopies.add(new Canopy(centroid, nextCanopyId++));
     }
     mapper.config(canopies);
-    List<VectorWritable> points = getPointsWritable(raw);
+    List<VectorWritable> points = getPointsWritable();
     // map the data
     for (VectorWritable point : points) {
       mapper.map(new Text(), point, collector, new DummyReporter());
@@ -538,7 +536,7 @@ public class TestCanopyCreation extends 
       canopies.add(new Canopy(centroid, nextCanopyId++));
     }
     mapper.config(canopies);
-    List<VectorWritable> points = getPointsWritable(raw);
+    List<VectorWritable> points = getPointsWritable();
     // map the data
     for (VectorWritable point : points) {
       mapper.map(new Text(), point, collector, new DummyReporter());
@@ -571,7 +569,7 @@ public class TestCanopyCreation extends 
    * ManhattanDistanceMeasure.
    */
   public void testClusteringManhattanMR() throws Exception {
-    List<VectorWritable> points = getPointsWritable(raw);
+    List<VectorWritable> points = getPointsWritable();
     File testData = new File("testdata");
     if (!testData.exists()) {
       testData.mkdir();
@@ -605,7 +603,7 @@ public class TestCanopyCreation extends 
    * EuclideanDistanceMeasure.
    */
   public void testClusteringEuclideanMR() throws Exception {
-    List<VectorWritable> points = getPointsWritable(raw);
+    List<VectorWritable> points = getPointsWritable();
     File testData = new File("testdata");
     if (!testData.exists()) {
       testData.mkdir();
@@ -636,7 +634,7 @@ public class TestCanopyCreation extends 
   
   /** Story: Clustering algorithm must support arbitrary user defined distance measure */
   public void testUserDefinedDistanceMeasure() throws Exception {
-    List<VectorWritable> points = getPointsWritable(raw);
+    List<VectorWritable> points = getPointsWritable();
     File testData = new File("testdata");
     if (!testData.exists()) {
       testData.mkdir();

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/fuzzykmeans/TestFuzzyKmeansClustering.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/fuzzykmeans/TestFuzzyKmeansClustering.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/fuzzykmeans/TestFuzzyKmeansClustering.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/fuzzykmeans/TestFuzzyKmeansClustering.java Wed Apr 21 07:41:55 2010
@@ -36,6 +36,7 @@ import org.apache.mahout.common.DummyRep
 import org.apache.mahout.common.MahoutTestCase;
 import org.apache.mahout.common.distance.DistanceMeasure;
 import org.apache.mahout.common.distance.EuclideanDistanceMeasure;
+import org.apache.mahout.math.NamedVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.math.VectorWritable;
 
@@ -85,12 +86,12 @@ public class TestFuzzyKmeansClustering e
     
   }
   
-  private static void computeCluster(List<Vector> points,
+  private static void computeCluster(List<NamedVector> points,
                                      List<SoftCluster> clusterList,
                                      FuzzyKMeansClusterer clusterer,
                                      Map<String,String> pointClusterInfo) {
     
-    for (Vector point : points) {
+    for (NamedVector point : points) {
       StringBuilder outputValue = new StringBuilder("[");
       List<Double> clusterDistanceList = new ArrayList<Double>();
       for (SoftCluster cluster : clusterList) {
@@ -100,14 +101,12 @@ public class TestFuzzyKmeansClustering e
         double probWeight = clusterer.computeProbWeight(clusterDistanceList.get(i), clusterDistanceList);
         outputValue.append(clusterList.get(i).getId()).append(':').append(probWeight).append(' ');
       }
-      String name = point.getName();
-      pointClusterInfo.put(name != null && name.length() != 0 ? name : point.asFormatString().trim(),
-        outputValue.toString().trim() + ']');
+      pointClusterInfo.put(point.getName(), outputValue.toString().trim() + ']');
     }
   }
   
   public void testReferenceImplementation() throws Exception {
-    List<Vector> points = TestKmeansClustering.getPoints(TestKmeansClustering.reference);
+    List<NamedVector> points = TestKmeansClustering.getPoints(TestKmeansClustering.reference);
     for (int k = 0; k < points.size(); k++) {
       System.out.println("test k= " + k);
       
@@ -413,9 +412,10 @@ public class TestFuzzyKmeansClustering e
         Vector vec = tweakValue(points.get(i).get());
         reference.add(new SoftCluster(vec, i));
       }
-      List<Vector> pointsVectors = new ArrayList<Vector>();
-      for(VectorWritable point : points)
-        pointsVectors.add(point.get());
+      List<NamedVector> pointsVectors = new ArrayList<NamedVector>();
+      for (VectorWritable point : points) {
+        pointsVectors.add((NamedVector) point.get());
+      }
       
       DistanceMeasure measure = new EuclideanDistanceMeasure();
       FuzzyKMeansClusterer clusterer = new FuzzyKMeansClusterer(measure, 0.001, 2);
@@ -519,9 +519,10 @@ public class TestFuzzyKmeansClustering e
         reference.add(new SoftCluster(vec, i));
       }
       Map<String,String> pointClusterInfo = new HashMap<String,String>();
-      List<Vector> pointsVectors = new ArrayList<Vector>();
-      for(VectorWritable point : points)
-        pointsVectors.add(point.get());
+      List<NamedVector> pointsVectors = new ArrayList<NamedVector>();
+      for (VectorWritable point : points) {
+        pointsVectors.add((NamedVector) point.get());
+      }
       
       List<List<SoftCluster>> clusters = FuzzyKMeansClusterer.clusterPoints(pointsVectors, reference,
         new EuclideanDistanceMeasure(), 0.001, 2, 1);

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestKmeansClustering.java Wed Apr 21 07:41:55 2010
@@ -38,8 +38,8 @@ import org.apache.mahout.common.MahoutTe
 import org.apache.mahout.common.distance.DistanceMeasure;
 import org.apache.mahout.common.distance.EuclideanDistanceMeasure;
 import org.apache.mahout.common.distance.ManhattanDistanceMeasure;
-import org.apache.mahout.math.AbstractVector;
 import org.apache.mahout.math.DenseVector;
+import org.apache.mahout.math.NamedVector;
 import org.apache.mahout.math.RandomAccessSparseVector;
 import org.apache.mahout.math.SequentialAccessSparseVector;
 import org.apache.mahout.math.Vector;
@@ -80,20 +80,20 @@ public class TestKmeansClustering extend
   
   public static List<VectorWritable> getPointsWritable(double[][] raw) {
     List<VectorWritable> points = new ArrayList<VectorWritable>();
-    int i = 0;
-    for (double[] fr : raw) {
-      Vector vec = new RandomAccessSparseVector(String.valueOf(i++), fr.length);
+    for (int i = 0; i < raw.length; i++) {
+      double[] fr = raw[i];
+      Vector vec = new RandomAccessSparseVector(fr.length);
       vec.assign(fr);
-      points.add(new VectorWritable(vec));
+      points.add(new VectorWritable(new NamedVector(vec, String.valueOf(i))));
     }
     return points;
   }
   
-  public static List<Vector> getPoints(double[][] raw) {
-    List<Vector> points = new ArrayList<Vector>();
+  public static List<NamedVector> getPoints(double[][] raw) {
+    List<NamedVector> points = new ArrayList<NamedVector>();
     for (int i = 0; i < raw.length; i++) {
       double[] fr = raw[i];
-      Vector vec = new SequentialAccessSparseVector(String.valueOf(i), fr.length);
+      NamedVector vec = new NamedVector(new SequentialAccessSparseVector(fr.length), String.valueOf(i));
       vec.assign(fr);
       points.add(vec);
     }
@@ -102,7 +102,7 @@ public class TestKmeansClustering extend
   
   /** Story: Test the reference implementation */
   public void testReferenceImplementation() throws Exception {
-    List<Vector> points = getPoints(reference);
+    List<NamedVector> points = getPoints(reference);
     DistanceMeasure measure = new EuclideanDistanceMeasure();
     // try all possible values of k
     for (int k = 0; k < points.size(); k++) {
@@ -127,7 +127,7 @@ public class TestKmeansClustering extend
   }
   
   public void testStd() {
-    List<Vector> points = getPoints(reference);
+    List<NamedVector> points = getPoints(reference);
     Cluster c = new Cluster(points.get(0));
     for (Vector p : points) {
       c.addPoint(p);
@@ -302,9 +302,10 @@ public class TestKmeansClustering extend
         Vector vec = points.get(i).get();
         reference.add(new Cluster(vec, i));
       }
-      List<Vector> pointsVectors = new ArrayList<Vector>();
-      for(VectorWritable point : points)
-        pointsVectors.add(point.get());
+      List<NamedVector> pointsVectors = new ArrayList<NamedVector>();
+      for (VectorWritable point : points) {
+        pointsVectors.add((NamedVector) point.get());
+      }
       boolean converged = KMeansClusterer.runKMeansIteration(pointsVectors, reference,
         euclideanDistanceMeasure, 0.001);
       if (k == 8) {
@@ -324,13 +325,7 @@ public class TestKmeansClustering extend
         // Since we aren't roundtripping through Writable, we need to compare the reference center with the
         // cluster centroid
         cluster.recomputeCenter();
-        assertTrue(i + " reference center: " + ref.getCenter().asFormatString() + " and cluster center:  "
-                   + cluster.getCenter().asFormatString() + " are not equal", AbstractVector.equivalent(ref
-            .getCenter(), cluster.getCenter()));
-        
-        /*
-         * assertEquals(k + " center[" + key + "][1]", ref.getCenter().get(1), cluster.getCenter().get(1));
-         */
+        assertEquals(ref.getCenter(), cluster.getCenter());
       }
       if (k == 8) {
         assertTrue("not converged? " + k, converged);

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestRandomSeedGenerator.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestRandomSeedGenerator.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestRandomSeedGenerator.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/kmeans/TestRandomSeedGenerator.java Wed Apr 21 07:41:55 2010
@@ -42,11 +42,10 @@ public class TestRandomSeedGenerator ext
   
   private FileSystem fs;
   
-  private static List<VectorWritable> getPoints(double[][] raw) {
+  private static List<VectorWritable> getPoints() {
     List<VectorWritable> points = new ArrayList<VectorWritable>();
-    int i = 0;
     for (double[] fr : raw) {
-      Vector vec = new RandomAccessSparseVector(String.valueOf(i++), fr.length);
+      Vector vec = new RandomAccessSparseVector(fr.length);
       vec.assign(fr);
       points.add(new VectorWritable(vec));
     }
@@ -76,7 +75,7 @@ public class TestRandomSeedGenerator ext
   
   /** Story: test random seed generation generates 4 clusters with proper ids and data */
   public void testRandomSeedGenerator() throws Exception {
-    List<VectorWritable> points = getPoints(raw);
+    List<VectorWritable> points = getPoints();
     File testData = new File("testdata");
     if (!testData.exists()) {
       testData.mkdir();

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/lda/TestMapReduce.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/lda/TestMapReduce.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/lda/TestMapReduce.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/lda/TestMapReduce.java Wed Apr 21 07:41:55 2010
@@ -99,7 +99,6 @@ public class TestMapReduce extends Mahou
     LDAState state = generateRandomState(100,NUM_TOPICS);
     LDAMapper mapper = new LDAMapper();
     mapper.configure(state);
-    VectorWritable vw = new VectorWritable();
     for(int i = 0; i < NUM_TESTS; ++i) {
       RandomAccessSparseVector v = generateRandomDoc(100,0.3);
       int myNumWords = numNonZero(v);
@@ -108,7 +107,7 @@ public class TestMapReduce extends Mahou
       mock.write(isA(IntPairWritable.class),isA(DoubleWritable.class));
       expectLastCall().times(myNumWords * NUM_TOPICS + NUM_TOPICS + 1);
       replay(mock);
-      vw.set(v);
+      VectorWritable vw = new VectorWritable(v);
       mapper.map(new Text("tstMapper"), vw, mock);
       verify(mock);
     }

Added: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/math/VectorTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/math/VectorTest.java?rev=936183&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/math/VectorTest.java (added)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/math/VectorTest.java Wed Apr 21 07:41:55 2010
@@ -0,0 +1,73 @@
+/**
+ * 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.
+ */
+
+package org.apache.mahout.math;
+
+import junit.framework.TestCase;
+import org.apache.hadoop.io.Writable;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+public class VectorTest extends TestCase {
+
+  public void testSequentialAccessSparseVectorWritable() throws Exception {
+    Vector v = new SequentialAccessSparseVector(5);
+    v.set(1, 3.0);
+    v.set(3, 5.0);
+    doTestVectorWritableEquals(v);
+  }
+
+  public void testRandomAccessSparseVectorWritable() throws Exception {
+    Vector v = new RandomAccessSparseVector(5);
+    v.set(1, 3.0);
+    v.set(3, 5.0);
+    doTestVectorWritableEquals(v);
+  }
+
+  public void testDenseVectorWritable() throws Exception {
+    Vector v = new DenseVector(5);
+    v.set(1, 3.0);
+    v.set(3, 5.0);
+    doTestVectorWritableEquals(v);
+  }
+
+  private static void doTestVectorWritableEquals(Vector v) throws IOException {
+    Writable vectorWritable = new VectorWritable(v);
+    VectorWritable vectorWritable2 = new VectorWritable();
+    writeAndRead(vectorWritable, vectorWritable2);
+    Vector v2 = vectorWritable2.get();
+    assertEquals(v, v2);
+  }
+
+  private static void writeAndRead(Writable toWrite, Writable toRead) throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutput dos = new DataOutputStream(baos);
+    toWrite.write(dos);
+
+    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+    DataInput dis = new DataInputStream(bais);
+    toRead.readFields(dis);
+  }
+
+
+}

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/math/hadoop/TestDistributedRowMatrix.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/math/hadoop/TestDistributedRowMatrix.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/math/hadoop/TestDistributedRowMatrix.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/math/hadoop/TestDistributedRowMatrix.java Wed Apr 21 07:41:55 2010
@@ -133,7 +133,7 @@ public class TestDistributedRowMatrix ex
                                                              int numCols,
                                                              int entriesPerRow,
                                                              double entryMean,
-                                                             boolean isSymmetric) throws Exception {
+                                                             boolean isSymmetric) throws IOException {
     return randomDistributedMatrix(numRows, nonNullRows, numCols, entriesPerRow, entryMean, isSymmetric, "");
   }
 
@@ -157,15 +157,13 @@ public class TestDistributedRowMatrix ex
       @Override
       public Iterator<VectorWritable> iterator() {
         final Iterator<MatrixSlice> it = m.iterator();
-        final VectorWritable v = new VectorWritable();
         return new Iterator<VectorWritable>() {
           @Override
           public boolean hasNext() { return it.hasNext(); }
           @Override
           public VectorWritable next() {
             MatrixSlice slice = it.next();
-            v.set(slice.vector());
-            return v;
+            return new VectorWritable(slice.vector());
           }
           @Override
           public void remove() { it.remove(); }

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayDirichlet.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayDirichlet.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayDirichlet.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayDirichlet.java Wed Apr 21 07:41:55 2010
@@ -34,6 +34,7 @@ import org.apache.mahout.clustering.diri
 import org.apache.mahout.clustering.dirichlet.models.ModelDistribution;
 import org.apache.mahout.common.RandomUtils;
 import org.apache.mahout.math.DenseVector;
+import org.apache.mahout.math.NamedVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.math.VectorWritable;
 import org.slf4j.Logger;
@@ -214,8 +215,10 @@ public class DisplayDirichlet extends Fr
     sampleParams.add(new DenseVector(params));
     log.info("Generating {} samples m=[{}, {}] sd={}", new Object[] {num, mx, my, sd});
     for (int i = 0; i < num; i++) {
-      sampleData.add(new VectorWritable(new DenseVector(new double[] {UncommonDistributions.rNorm(mx, sd),
-                                                                      UncommonDistributions.rNorm(my, sd)})));
+      sampleData.add(new VectorWritable(
+          new NamedVector(new DenseVector(new double[] {UncommonDistributions.rNorm(mx, sd),
+                                                        UncommonDistributions.rNorm(my, sd)}),
+                          String.valueOf(i))));
     }
   }
   

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/fuzzykmeans/DisplayFuzzyKMeans.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/fuzzykmeans/DisplayFuzzyKMeans.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/fuzzykmeans/DisplayFuzzyKMeans.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/fuzzykmeans/DisplayFuzzyKMeans.java Wed Apr 21 07:41:55 2010
@@ -28,14 +28,12 @@ import org.apache.mahout.common.RandomUt
 import org.apache.mahout.common.distance.DistanceMeasure;
 import org.apache.mahout.common.distance.ManhattanDistanceMeasure;
 import org.apache.mahout.math.DenseVector;
+import org.apache.mahout.math.NamedVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.math.VectorWritable;
 
 class DisplayFuzzyKMeans extends DisplayDirichlet {
-  private static final double t1 = 3.0;
-  
-  private static final double t2 = 1.5;
-  
+
   private static List<List<SoftCluster>> clusters;
   
   DisplayFuzzyKMeans() {
@@ -64,9 +62,10 @@ class DisplayFuzzyKMeans extends Display
   public static void main(String[] args) {
     RandomUtils.useTestSeed();
     DisplayDirichlet.generateSamples();
-    List<Vector> points = new ArrayList<Vector>();
-    for (VectorWritable sample : sampleData)
-      points.add(sample.get());
+    List<NamedVector> points = new ArrayList<NamedVector>();
+    for (VectorWritable sample : sampleData) {
+      points.add((NamedVector) sample.get());
+    }
     DistanceMeasure measure = new ManhattanDistanceMeasure();
     List<SoftCluster> initialClusters = new ArrayList<SoftCluster>();
     
@@ -75,7 +74,9 @@ class DisplayFuzzyKMeans extends Display
     for (Vector point : points) {
       if (initialClusters.size() < Math.min(k, points.size())) {
         initialClusters.add(new SoftCluster(point, i++));
-      } else break;
+      } else {
+        break;
+      }
     }
     clusters = FuzzyKMeansClusterer.clusterPoints(points, initialClusters, measure, 0.001, 3, 10);
     new DisplayFuzzyKMeans();

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/kmeans/DisplayKMeans.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/kmeans/DisplayKMeans.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/kmeans/DisplayKMeans.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/kmeans/DisplayKMeans.java Wed Apr 21 07:41:55 2010
@@ -28,6 +28,7 @@ import org.apache.mahout.common.RandomUt
 import org.apache.mahout.common.distance.DistanceMeasure;
 import org.apache.mahout.common.distance.ManhattanDistanceMeasure;
 import org.apache.mahout.math.DenseVector;
+import org.apache.mahout.math.NamedVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.math.VectorWritable;
 
@@ -62,9 +63,10 @@ class DisplayKMeans extends DisplayDiric
   public static void main(String[] args) {
     RandomUtils.useTestSeed();
     DisplayDirichlet.generateSamples();
-    List<Vector> points = new ArrayList<Vector>();
-    for (VectorWritable sample : sampleData)
-      points.add(sample.get());
+    List<NamedVector> points = new ArrayList<NamedVector>();
+    for (VectorWritable sample : sampleData) {
+      points.add((NamedVector) sample.get());
+    }
     DistanceMeasure measure = new ManhattanDistanceMeasure();
     List<Cluster> initialClusters = new ArrayList<Cluster>();
     k = 3;
@@ -72,7 +74,9 @@ class DisplayKMeans extends DisplayDiric
     for (Vector point : points) {
       if (initialClusters.size() < Math.min(k, points.size())) {
         initialClusters.add(new Cluster(point, i++));
-      } else break;
+      } else {
+        break;
+      }
     }
     clusters = KMeansClusterer.clusterPoints(points, initialClusters, measure, 10, 0.001);
     System.out.println(clusters.size());

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/InputMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/InputMapper.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/InputMapper.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/InputMapper.java Wed Apr 21 07:41:55 2010
@@ -39,9 +39,7 @@ public class InputMapper extends MapRedu
   private static final Pattern SPACE = Pattern.compile(" ");
   
   private Constructor<?> constructor;
-  
-  private VectorWritable vectorWritable;
-  
+
   @Override
   public void map(LongWritable key,
                   Text values,
@@ -61,7 +59,7 @@ public class InputMapper extends MapRedu
       for (Double d : doubles) {
         result.set(index++, d);
       }
-      vectorWritable.set(result);
+      VectorWritable vectorWritable = new VectorWritable(result);
       output.collect(new Text(String.valueOf(index)), vectorWritable);
       
     } catch (InstantiationException e) {
@@ -75,7 +73,6 @@ public class InputMapper extends MapRedu
   
   @Override
   public void configure(JobConf job) {
-    vectorWritable = new VectorWritable();
     String vectorImplClassName = job.get("vector.implementation.class.name");
     try {
       Class<? extends Vector> outputClass = (Class<? extends Vector>) job.getClassByName(vectorImplClassName);

Modified: lucene/mahout/trunk/examples/src/test/java/org/apache/mahout/ga/watchmaker/cd/CDCrossoverTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/test/java/org/apache/mahout/ga/watchmaker/cd/CDCrossoverTest.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/test/java/org/apache/mahout/ga/watchmaker/cd/CDCrossoverTest.java (original)
+++ lucene/mahout/trunk/examples/src/test/java/org/apache/mahout/ga/watchmaker/cd/CDCrossoverTest.java Wed Apr 21 07:41:55 2010
@@ -91,8 +91,9 @@ public class CDCrossoverTest extends Mah
       int nbattributes = rng.nextInt(maxattributes) + 3;
       int crosspnts = rng.nextInt(maxcrosspnts) + 1;
       // in the case of this test crosspnts should be < nbattributes
-      if (crosspnts >= nbattributes)
+      if (crosspnts >= nbattributes) {
         crosspnts = nbattributes - 1;
+      }
 
       // prepare dataset mock
       EasyMock.reset(dataset);
@@ -127,16 +128,6 @@ public class CDCrossoverTest extends Mah
 
   }
 
-  static String printRule(CDRule rule) {
-    StringBuilder buffer = new StringBuilder();
-
-    for (int index = 0; index < rule.getNbConditions(); index++) {
-      buffer.append(rule.getO(index) ? 1 : 0);
-    }
-
-    return buffer.toString();
-  }
-
   static int countAreas(CDRule rule) {
 
     int nbareas = 1; // we already start in an area

Modified: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/AbstractMatrix.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/AbstractMatrix.java?rev=936183&r1=936182&r2=936183&view=diff
==============================================================================
--- lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/AbstractMatrix.java (original)
+++ lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/AbstractMatrix.java Wed Apr 21 07:41:55 2010
@@ -28,6 +28,7 @@ import java.lang.reflect.Type;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
+import java.util.NoSuchElementException;
 
 /** A few universal implementations of convenience functions */
 public abstract class AbstractMatrix implements Matrix {
@@ -42,7 +43,7 @@ public abstract class AbstractMatrix imp
 
   public Iterator<MatrixSlice> iterateAll() {
     return new Iterator<MatrixSlice>() {
-      int slice = 0;
+      private int slice = 0;
 
       public boolean hasNext() {
         return slice < numSlices();
@@ -203,11 +204,11 @@ public abstract class AbstractMatrix imp
   public Matrix assign(double[][] values) {
     int[] c = size();
     if (c[ROW] != values.length) {
-      throw new CardinalityException();
+      throw new CardinalityException(c[ROW], values.length);
     }
     for (int row = 0; row < c[ROW]; row++) {
       if (c[COL] != values[row].length) {
-        throw new CardinalityException();
+        throw new CardinalityException(c[COL], values[row].length);
       } else {
         for (int col = 0; col < c[COL]; col++) {
           setQuick(row, col, values[row][col]);
@@ -220,8 +221,11 @@ public abstract class AbstractMatrix imp
   public Matrix assign(Matrix other, BinaryFunction function) {
     int[] c = size();
     int[] o = other.size();
-    if (c[ROW] != o[ROW] || c[COL] != o[COL]) {
-      throw new CardinalityException();
+    if (c[ROW] != o[ROW]) {
+      throw new CardinalityException(c[ROW], o[ROW]);
+    }
+    if (c[COL] != o[COL]) {
+      throw new CardinalityException(c[COL], o[COL]);
     }
     for (int row = 0; row < c[ROW]; row++) {
       for (int col = 0; col < c[COL]; col++) {
@@ -235,8 +239,11 @@ public abstract class AbstractMatrix imp
   public Matrix assign(Matrix other) {
     int[] c = size();
     int[] o = other.size();
-    if (c[ROW] != o[ROW] || c[COL] != o[COL]) {
-      throw new CardinalityException();
+    if (c[ROW] != o[ROW]) {
+      throw new CardinalityException(c[ROW], o[ROW]);
+    }
+    if (c[COL] != o[COL]) {
+      throw new CardinalityException(c[COL], o[COL]);
     }
     for (int row = 0; row < c[ROW]; row++) {
       for (int col = 0; col < c[COL]; col++) {
@@ -261,7 +268,7 @@ public abstract class AbstractMatrix imp
     int rowSize = card[ROW];
     int columnSize = card[COL];
     if (rowSize != columnSize) {
-      throw new CardinalityException();
+      throw new CardinalityException(rowSize, columnSize);
     }
 
     if (rowSize == 2) {
@@ -322,8 +329,11 @@ public abstract class AbstractMatrix imp
 
   public double get(int row, int column) {
     int[] c = size();
-    if (row < 0 || column < 0 || row >= c[ROW] || column >= c[COL]) {
-      throw new IndexException();
+    if (row < 0 || row >= c[ROW]) {
+      throw new IndexException(row, c[ROW]);
+    }
+    if (column < 0 || column >= c[COL]) {
+      throw new IndexException(column, c[COL]);
     }
     return getQuick(row, column);
   }
@@ -331,8 +341,11 @@ public abstract class AbstractMatrix imp
   public Matrix minus(Matrix other) {
     int[] c = size();
     int[] o = other.size();
-    if (c[ROW] != o[ROW] || c[COL] != o[COL]) {
-      throw new CardinalityException();
+    if (c[ROW] != o[ROW]) {
+      throw new CardinalityException(c[ROW], o[ROW]);
+    }
+    if (c[COL] != o[COL]) {
+      throw new CardinalityException(c[COL], o[COL]);
     }
     Matrix result = clone();
     for (int row = 0; row < c[ROW]; row++) {
@@ -358,8 +371,11 @@ public abstract class AbstractMatrix imp
   public Matrix plus(Matrix other) {
     int[] c = size();
     int[] o = other.size();
-    if (c[ROW] != o[ROW] || c[COL] != o[COL]) {
-      throw new CardinalityException();
+    if (c[ROW] != o[ROW]) {
+      throw new CardinalityException(c[ROW], o[ROW]);
+    }
+    if (c[COL] != o[COL]) {
+      throw new CardinalityException(c[COL], o[COL]);
     }
     Matrix result = clone();
     for (int row = 0; row < c[ROW]; row++) {
@@ -373,8 +389,11 @@ public abstract class AbstractMatrix imp
 
   public void set(int row, int column, double value) {
     int[] c = size();
-    if (row < 0 || column < 0 || row >= c[ROW] || column >= c[COL]) {
-      throw new IndexException();
+    if (row < 0 || row >= c[ROW]) {
+      throw new IndexException(row, c[ROW]);
+    }
+    if (column < 0 || column >= c[COL]) {
+      throw new IndexException(column, c[COL]);
     }
     setQuick(row, column, value);
   }
@@ -382,10 +401,10 @@ public abstract class AbstractMatrix imp
   public void set(int row, double[] data) {
     int[] c = size();
     if (c[COL] < data.length) {
-      throw new CardinalityException();
+      throw new CardinalityException(c[COL], data.length);
     }
-    if ((c[ROW] < row) || (row < 0)) {
-      throw new IndexException();
+    if (row < 0 || row >= c[ROW]) {
+      throw new IndexException(row, c[ROW]);
     }
 
     for (int i = 0; i < c[COL]; i++) {
@@ -426,7 +445,7 @@ public abstract class AbstractMatrix imp
   public Vector times(Vector v) {
     int[] c = size();
     if (c[COL] != v.size()) {
-      throw new CardinalityException();
+      throw new CardinalityException(c[COL], v.size());
     }
     Vector w = new DenseVector(c[ROW]);
     for (int i = 0; i < c[ROW]; i++) {
@@ -438,14 +457,15 @@ public abstract class AbstractMatrix imp
   public Vector timesSquared(Vector v) {
     int[] c = size();
     if (c[COL] != v.size()) {
-      throw new CardinalityException();
+      throw new CardinalityException(c[COL], v.size());
     }
     Vector w = new DenseVector(c[COL]);
     for (int i = 0; i < c[ROW]; i++) {
       Vector xi = getRow(i);
       double d = xi.dot(v);
-      if(d != 0)
+      if (d != 0.0) {
         w.assign(xi, new PlusMult(d));
+      }
 
     }
     return w;
@@ -485,16 +505,16 @@ public abstract class AbstractMatrix imp
     }
 
     protected TransposeViewVector(Matrix m, int offset, boolean rowToColumn) {
+      super(rowToColumn ? m.numRows() : m.numCols());
       matrix = m;
       this.transposeOffset = offset;
       this.rowToColumn = rowToColumn;
       numCols = rowToColumn ? m.numCols() : m.numRows();
-      size = rowToColumn ? m.numRows() : m.numCols();
     }
 
     @Override
     public Vector clone() {
-      Vector v = new DenseVector(size);
+      Vector v = new DenseVector(size());
       addTo(v);
       return v;
     }
@@ -504,14 +524,17 @@ public abstract class AbstractMatrix imp
       return matrix.like(rows, columns);
     }
 
-    public Iterator<Element> iterateAll() {
+    public Iterator<Element> iterator() {
       return new Iterator<Element>() {
-        int i = 0;
+        private int i = 0;
         public boolean hasNext() {
-          return i < size;
+          return i < size();
         }
 
         public Element next() {
+          if (i >= size()) {
+            throw new NoSuchElementException();
+          }
           return getElement(i++);
         }
 
@@ -522,12 +545,12 @@ public abstract class AbstractMatrix imp
     }
 
     /**
-     * Currently delegates to iterateAll.  TODO: This could be optimized to at least skip empty rows if there are
-     * many of them.
+     * Currently delegates to {@link #iterator()}.
+     * TODO: This could be optimized to at least skip empty rows if there are many of them.
      * @return an iterator (currently dense).
      */
     public Iterator<Element> iterateNonZero() {
-      return iterateAll();
+      return iterator();
     }
 
     public Element getElement(final int i) {
@@ -566,7 +589,7 @@ public abstract class AbstractMatrix imp
     }
 
     public Vector like() {
-      return new DenseVector(size);
+      return new DenseVector(size());
     }
 
     public Vector like(int cardinality) {
@@ -579,7 +602,7 @@ public abstract class AbstractMatrix imp
      * @return the number of nonzero entries
      */
     public int getNumNondefaultElements() {
-      return size;
+      return size();
     }
   }