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/01/14 18:54:32 UTC

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

Author: srowen
Date: Thu Jan 14 17:54:30 2010
New Revision: 899330

URL: http://svn.apache.org/viewvc?rev=899330&view=rev
Log:
MAHOUT-206 for Jake

Added:
    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/test/resources/word-list.txt
    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
Removed:
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/SparseVectorWritable.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/math/
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/SparseVector.java
    lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/math/TestVectorWritable.java
Modified:
    lucene/mahout/trunk/core/pom.xml
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/RecommenderJob.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/RecommenderMapper.java
    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/UserVectorToCooccurrenceMapper.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/ClusterBase.java
    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/canopy/CanopyClusteringJob.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyDriver.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/ClusterDriver.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletDriver.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansDriver.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/SoftCluster.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansDriver.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/VectorWritable.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/ClusteringTestUtils.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/dirichlet/TestMapReduce.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/common/distance/DefaultDistanceMeasureTest.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/InputDriver.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/InputMapper.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/Job.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/dirichlet/Job.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/kmeans/Job.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/PlusWithScaleFunction.java
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/SparseColumnMatrix.java
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/SparseMatrix.java
    lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/SparseRowMatrix.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/test/java/org/apache/mahout/math/TestSparseVector.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/arff/ARFFVectorIterable.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/arff/Driver.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/lucene/Driver.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/lucene/TFDFMapper.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizer.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/PartialVectorGenerator.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/vectors/text/PartialVectorMerger.java
    lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/RandomVectorIterable.java
    lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/SequenceFileVectorIterableTest.java
    lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/arff/ARFFVectorIterableTest.java
    lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/io/VectorWriterTest.java
    lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/lucene/LuceneIterableTest.java
    lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/vectors/text/DictionaryVectorizerTest.java

Modified: lucene/mahout/trunk/core/pom.xml
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/pom.xml?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/core/pom.xml (original)
+++ lucene/mahout/trunk/core/pom.xml Thu Jan 14 17:54:30 2010
@@ -285,7 +285,11 @@
       <version>2.2</version>
       <scope>test</scope>
     </dependency>
-
+    <dependency>
+      <groupId>com.google.collections</groupId>
+      <artifactId>google-collections</artifactId>
+      <version>1.0-rc5</version>
+    </dependency>
   </dependencies>
   
   <repositories>

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/RecommenderJob.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/RecommenderJob.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/RecommenderJob.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/RecommenderJob.java Thu Jan 14 17:54:30 2010
@@ -35,7 +35,6 @@
 import org.apache.mahout.cf.taste.hadoop.ItemPrefWritable;
 import org.apache.mahout.cf.taste.hadoop.RecommendedItemsWritable;
 import org.apache.mahout.cf.taste.hadoop.ToItemPrefsMapper;
-import org.apache.mahout.math.SparseVector;
 import org.apache.mahout.math.VectorWritable;
 
 import java.io.IOException;

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/RecommenderMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/RecommenderMapper.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/RecommenderMapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/RecommenderMapper.java Thu Jan 14 17:54:30 2010
@@ -37,7 +37,7 @@
 import org.apache.mahout.cf.taste.impl.recommender.GenericRecommendedItem;
 import org.apache.mahout.cf.taste.recommender.RecommendedItem;
 import org.apache.mahout.common.FileLineIterable;
-import org.apache.mahout.math.SparseVector;
+import org.apache.mahout.math.RandomAccessSparseVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.math.VectorWritable;
 
@@ -101,7 +101,7 @@
     }
     Vector userVector = vectorWritable.get();
     Iterator<Vector.Element> userVectorIterator = userVector.iterateNonZero();
-    Vector recommendationVector = new SparseVector(Integer.MAX_VALUE, 1000);
+    Vector recommendationVector = new RandomAccessSparseVector(Integer.MAX_VALUE, 1000);
     while (userVectorIterator.hasNext()) {
       Vector.Element element = userVectorIterator.next();
       int index = element.index();
@@ -159,7 +159,7 @@
     private CooccurrenceCache(MapFilesMap<IntWritable,VectorWritable> map) {
       this.map = map;
       columnVector = new VectorWritable();
-      columnVector.set(new SparseVector(Integer.MAX_VALUE, 1000));
+      columnVector.set(new RandomAccessSparseVector(Integer.MAX_VALUE, 1000));
     }
 
     @Override
@@ -173,7 +173,8 @@
       if (value == null) {
         return null;
       }
-      columnVector.set(new SparseVector(Integer.MAX_VALUE, 1000));
+      columnVector = new VectorWritable();
+      columnVector.set(new RandomAccessSparseVector(Integer.MAX_VALUE, 1000));
       return value;
     }
 

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=899330&r1=899329&r2=899330&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 Thu Jan 14 17:54:30 2010
@@ -23,8 +23,9 @@
 import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.mahout.cf.taste.hadoop.ItemPrefWritable;
-import org.apache.mahout.math.SparseVector;
+import org.apache.mahout.math.RandomAccessSparseVector;
 import org.apache.mahout.math.Vector;
+import org.apache.mahout.math.VectorWritable;
 
 import java.io.IOException;
 import java.util.Iterator;
@@ -39,7 +40,7 @@
  *
  * <h1>Output</h1>
  *
- * <p>The same user ID mapped to a {@link SparseVector} representation of the
+ * <p>The same user ID mapped to a {@link RandomAccessSparseVector} representation of the
  * same item IDs and preference values. Item IDs are used as vector indexes;
  * they are hashed into ints to work as indexes with
  * {@link ItemIDIndexMapper#idToIndex(long)}. The mapping is remembered for
@@ -51,17 +52,17 @@
  */
 public final class ToUserVectorReducer
     extends MapReduceBase
-    implements Reducer<LongWritable, ItemPrefWritable, LongWritable, SparseVector> {
+    implements Reducer<LongWritable, ItemPrefWritable, LongWritable, VectorWritable> {
 
   public static final int MAX_PREFS_CONSIDERED = 20;
-
+  private VectorWritable vectorWritable = new VectorWritable();
   @Override
   public void reduce(LongWritable userID,
                      Iterator<ItemPrefWritable> itemPrefs,
-                     OutputCollector<LongWritable, SparseVector> output,
+                     OutputCollector<LongWritable, VectorWritable> output,
                      Reporter reporter) throws IOException {
     if (itemPrefs.hasNext()) {
-      SparseVector userVector = new SparseVector(Integer.MAX_VALUE, 100);
+      RandomAccessSparseVector userVector = new RandomAccessSparseVector(Integer.MAX_VALUE, 100);
       while (itemPrefs.hasNext()) {
         ItemPrefWritable itemPref = itemPrefs.next();
         int index = ItemIDIndexMapper.idToIndex(itemPref.getItemID());
@@ -70,7 +71,7 @@
 
       if (userVector.getNumNondefaultElements() > MAX_PREFS_CONSIDERED) {
         double cutoff = findTopNPrefsCutoff(MAX_PREFS_CONSIDERED, userVector);
-        SparseVector filteredVector = new SparseVector(Integer.MAX_VALUE, MAX_PREFS_CONSIDERED);
+        RandomAccessSparseVector filteredVector = new RandomAccessSparseVector(Integer.MAX_VALUE, MAX_PREFS_CONSIDERED);
         Iterator<Vector.Element> it = userVector.iterateNonZero();
         while (it.hasNext()) {
           Vector.Element element = it.next();
@@ -81,7 +82,8 @@
         userVector = filteredVector;
       }
 
-      output.collect(userID, userVector);
+      vectorWritable.set(userVector);
+      output.collect(userID, vectorWritable);
     }
   }
 

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/UserVectorToCooccurrenceMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/UserVectorToCooccurrenceMapper.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/UserVectorToCooccurrenceMapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/UserVectorToCooccurrenceMapper.java Thu Jan 14 17:54:30 2010
@@ -23,26 +23,26 @@
 import org.apache.hadoop.mapred.Mapper;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reporter;
-import org.apache.mahout.math.SparseVector;
 import org.apache.mahout.math.Vector;
+import org.apache.mahout.math.VectorWritable;
 
 import java.io.IOException;
 import java.util.Iterator;
 
 public final class UserVectorToCooccurrenceMapper
     extends MapReduceBase
-    implements Mapper<LongWritable, SparseVector, IntWritable, IntWritable> {
+    implements Mapper<LongWritable, VectorWritable, IntWritable, IntWritable> {
 
   @Override
   public void map(LongWritable userID,
-                  SparseVector userVector,
+                  VectorWritable userVector,
                   OutputCollector<IntWritable, IntWritable> output,
                   Reporter reporter) throws IOException {
-    Iterator<Vector.Element> it = userVector.iterateNonZero();
+    Iterator<Vector.Element> it = userVector.get().iterateNonZero();
     while (it.hasNext()) {
       Vector.Element next1 = it.next();
       int index1 = next1.index();
-      Iterator<Vector.Element> it2 = userVector.iterateNonZero();
+      Iterator<Vector.Element> it2 = userVector.get().iterateNonZero();
       IntWritable itemWritable1 = new IntWritable(index1);
       while (it2.hasNext()) {
         Vector.Element next2 = it2.next();

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=899330&r1=899329&r2=899330&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 Thu Jan 14 17:54:30 2010
@@ -22,23 +22,24 @@
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reporter;
-import org.apache.mahout.math.SparseVector;
+import org.apache.mahout.math.RandomAccessSparseVector;
 import org.apache.mahout.math.Vector;
+import org.apache.mahout.math.VectorWritable;
 
 import java.io.IOException;
 import java.util.Iterator;
 
 public final class UserVectorToCooccurrenceReducer
     extends MapReduceBase
-    implements Reducer<IntWritable, IntWritable, IntWritable, SparseVector> {
-
+    implements Reducer<IntWritable, IntWritable, IntWritable, VectorWritable> {
+  private VectorWritable vectorWritable = new VectorWritable();
   @Override
   public void reduce(IntWritable index1,
                      Iterator<IntWritable> index2s,
-                     OutputCollector<IntWritable, SparseVector> output,
+                     OutputCollector<IntWritable, VectorWritable> output,
                      Reporter reporter) throws IOException {
     if (index2s.hasNext()) {
-      SparseVector cooccurrenceRow = new SparseVector(Integer.MAX_VALUE, 1000);
+      RandomAccessSparseVector cooccurrenceRow = new RandomAccessSparseVector(Integer.MAX_VALUE, 1000);
       while (index2s.hasNext()) {
         int index2 = index2s.next().get();
         cooccurrenceRow.set(index2, cooccurrenceRow.get(index2) + 1.0);
@@ -50,7 +51,8 @@
           element.set(0.0);
         }
       }
-      output.collect(index1, cooccurrenceRow);
+      vectorWritable.set(cooccurrenceRow);
+      output.collect(index1, vectorWritable);
     }
   }
 

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/ClusterBase.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/ClusterBase.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/ClusterBase.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/ClusterBase.java Thu Jan 14 17:54:30 2010
@@ -18,7 +18,7 @@
 package org.apache.mahout.clustering;
 
 import org.apache.hadoop.io.Writable;
-import org.apache.mahout.math.SparseVector;
+import org.apache.mahout.math.RandomAccessSparseVector;
 import org.apache.mahout.math.Vector;
 
 import java.io.DataInput;
@@ -31,7 +31,7 @@
   private int id;
 
   // the current cluster center
-  private Vector center = new SparseVector(0);
+  private Vector center = new RandomAccessSparseVector(0);
 
   // the number of points in the cluster
   private int numPoints = 0;

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=899330&r1=899329&r2=899330&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 Jan 14 17:54:30 2010
@@ -130,7 +130,7 @@
   /**
    * Compute the centroid by averaging the pointTotals
    *
-   * @return a SparseVector (required by Mapper) which is the new centroid
+   * @return a RandomAccessSparseVector (required by Mapper) which is the new centroid
    */
   public Vector computeCentroid() {
     return getPointTotal().divide(getNumPoints());

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyClusteringJob.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyClusteringJob.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyClusteringJob.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyClusteringJob.java Thu Jan 14 17:54:30 2010
@@ -25,7 +25,7 @@
 import org.apache.commons.cli2.builder.DefaultOptionBuilder;
 import org.apache.commons.cli2.builder.GroupBuilder;
 import org.apache.commons.cli2.commandline.Parser;
-import org.apache.mahout.math.SparseVector;
+import org.apache.mahout.math.RandomAccessSparseVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.common.CommandLineUtil;
 import org.apache.mahout.common.distance.SquaredEuclideanDistanceMeasure;
@@ -73,7 +73,7 @@
 
     Option vectorClassOpt = obuilder.withLongName("vectorClass").withRequired(false).withArgument(
         abuilder.withName("vectorClass").withMinimum(1).withMaximum(1).create()).
-        withDescription("The Vector implementation class name.  Default is SparseVector.class").withShortName("v").create();
+        withDescription("The Vector implementation class name.  Default is RandomAccessSparseVector.class").withShortName("v").create();
     Option t1Opt = obuilder.withLongName("t1").withRequired(true).withArgument(
         abuilder.withName("t1").withMinimum(1).withMaximum(1).create()).
         withDescription("t1").withShortName("t1").create();
@@ -109,7 +109,7 @@
       }
 
       Class<? extends Vector> vectorClass = cmdLine.hasOption(vectorClassOpt) == false ?
-          SparseVector.class
+          RandomAccessSparseVector.class
           : (Class<? extends Vector>) Class.forName(cmdLine.getValue(vectorClassOpt).toString());
       double t1 = Double.parseDouble(cmdLine.getValue(t1Opt).toString());
       double t2 = Double.parseDouble(cmdLine.getValue(t2Opt).toString());

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyDriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyDriver.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyDriver.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyDriver.java Thu Jan 14 17:54:30 2010
@@ -35,7 +35,7 @@
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
-import org.apache.mahout.math.SparseVector;
+import org.apache.mahout.math.RandomAccessSparseVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.common.CommandLineUtil;
 import org.apache.mahout.common.distance.SquaredEuclideanDistanceMeasure;
@@ -71,7 +71,7 @@
 
     Option vectorClassOpt = obuilder.withLongName("vectorClass").withRequired(false).withArgument(
         abuilder.withName("vectorClass").withMinimum(1).withMaximum(1).create()).
-        withDescription("The Vector implementation class name.  Default is SparseVector.class").withShortName("v").create();
+        withDescription("The Vector implementation class name.  Default is RandomAccessSparseVector.class").withShortName("v").create();
     Option t1Opt = obuilder.withLongName("t1").withRequired(true).withArgument(
         abuilder.withName("t1").withMinimum(1).withMaximum(1).create()).
         withDescription("t1").withShortName("t1").create();
@@ -106,7 +106,7 @@
       }
 
       Class<? extends Vector> vectorClass = cmdLine.hasOption(vectorClassOpt) == false ?
-          SparseVector.class
+          RandomAccessSparseVector.class
           : (Class<? extends Vector>) Class.forName(cmdLine.getValue(vectorClassOpt).toString());
       double t1 = Double.parseDouble(cmdLine.getValue(t1Opt).toString());
       double t2 = Double.parseDouble(cmdLine.getValue(t2Opt).toString());

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/ClusterDriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/ClusterDriver.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/ClusterDriver.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/ClusterDriver.java Thu Jan 14 17:54:30 2010
@@ -36,7 +36,7 @@
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
-import org.apache.mahout.math.SparseVector;
+import org.apache.mahout.math.RandomAccessSparseVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.common.CommandLineUtil;
 import org.apache.mahout.common.distance.SquaredEuclideanDistanceMeasure;
@@ -63,7 +63,7 @@
 
     Option vectorClassOpt = obuilder.withLongName("vectorClass").withRequired(false).withArgument(
         abuilder.withName("vectorClass").withMinimum(1).withMaximum(1).create()).
-        withDescription("The Vector implementation class name.  Default is SparseVector.class")
+        withDescription("The Vector implementation class name.  Default is RandomAccessSparseVector.class")
         .withShortName("v").create();
     Option t1Opt = obuilder.withLongName("t1").withRequired(true).withArgument(
         abuilder.withName("t1").withMinimum(1).withMaximum(1).create()).
@@ -113,7 +113,7 @@
       String canopies = cmdLine.getValue(canopiesOpt).toString();
       String points = cmdLine.getValue(pointsOpt).toString();
       Class<? extends Vector> vectorClass = cmdLine.hasOption(vectorClassOpt) == false ?
-          SparseVector.class
+          RandomAccessSparseVector.class
           : (Class<? extends Vector>) Class.forName(cmdLine.getValue(vectorClassOpt).toString());
       double t1 = Double.parseDouble(cmdLine.getValue(t1Opt).toString());
       double t2 = Double.parseDouble(cmdLine.getValue(t2Opt).toString());
@@ -151,7 +151,7 @@
     conf.setInputFormat(SequenceFileInputFormat.class);
 
     /*conf.setMapOutputKeyClass(Text.class);
-    conf.setMapOutputValueClass(SparseVector.class);*/
+    conf.setMapOutputValueClass(RandomAccessSparseVector.class);*/
     conf.setOutputKeyClass(Text.class);
     conf.setOutputValueClass(VectorWritable.class);
     conf.setOutputFormat(SequenceFileOutputFormat.class);

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletDriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletDriver.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletDriver.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletDriver.java Thu Jan 14 17:54:30 2010
@@ -38,8 +38,6 @@
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 import org.apache.mahout.clustering.dirichlet.models.ModelDistribution;
 import org.apache.mahout.clustering.kmeans.KMeansDriver;
-import org.apache.mahout.math.SparseVector;
-import org.apache.mahout.math.Vector;
 import org.apache.mahout.common.CommandLineUtil;
 import org.apache.mahout.common.commandline.DefaultOptionCreator;
 import org.apache.mahout.math.VectorWritable;

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansDriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansDriver.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansDriver.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansDriver.java Thu Jan 14 17:54:30 2010
@@ -40,7 +40,7 @@
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 import org.apache.mahout.clustering.kmeans.RandomSeedGenerator;
-import org.apache.mahout.math.SparseVector;
+import org.apache.mahout.math.RandomAccessSparseVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.common.CommandLineUtil;
 import org.apache.mahout.common.HadoopUtil;
@@ -100,7 +100,7 @@
 
     Option vectorClassOpt = obuilder.withLongName("vectorClass").withRequired(false).withArgument(
         abuilder.withName("vectorClass").withMinimum(1).withMaximum(1).create()).
-        withDescription("The Vector implementation class name.  Default is SparseVector.class").withShortName("v").create();
+        withDescription("The Vector implementation class name.  Default is RandomAccessSparseVector.class").withShortName("v").create();
 
     Option helpOpt = obuilder.withLongName("help").
         withDescription("Print out help").withShortName("h").create();
@@ -151,7 +151,7 @@
       float m = Float.parseFloat(cmdLine.getValue(mOpt).toString());
 
       Class<? extends Vector> vectorClass = cmdLine.hasOption(vectorClassOpt) == false ?
-          SparseVector.class
+          RandomAccessSparseVector.class
           : (Class<? extends Vector>) Class.forName(cmdLine.getValue(vectorClassOpt).toString());
 
 

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/SoftCluster.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/SoftCluster.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/SoftCluster.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/SoftCluster.java Thu Jan 14 17:54:30 2010
@@ -18,7 +18,11 @@
 package org.apache.mahout.clustering.fuzzykmeans;
 
 import org.apache.hadoop.io.Writable;
-import org.apache.mahout.math.*;
+import org.apache.mahout.math.AbstractVector;
+import org.apache.mahout.math.RandomAccessSparseVector;
+import org.apache.mahout.math.SquareRootFunction;
+import org.apache.mahout.math.Vector;
+import org.apache.mahout.math.VectorWritable;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -30,7 +34,7 @@
   private int clusterId;
 
   // the current center
-  private Vector center = new SparseVector(0);
+  private Vector center = new RandomAccessSparseVector(0);
 
   // the current centroid is lazy evaluated and may be null
   private Vector centroid = null;

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansDriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansDriver.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansDriver.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansDriver.java Thu Jan 14 17:54:30 2010
@@ -36,7 +36,7 @@
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
-import org.apache.mahout.math.SparseVector;
+import org.apache.mahout.math.RandomAccessSparseVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.common.CommandLineUtil;
 import org.apache.mahout.common.HadoopUtil;
@@ -108,7 +108,7 @@
 
     Option vectorClassOpt = obuilder.withLongName("vectorClass").withRequired(false).withArgument(
         abuilder.withName("vectorClass").withMinimum(1).withMaximum(1).create()).withDescription(
-        "The Vector implementation class name.  Default is SparseVector.class").withShortName("v").create();
+        "The Vector implementation class name.  Default is RandomAccessSparseVector.class").withShortName("v").create();
 
     Option numReduceTasksOpt = obuilder.withLongName("numReduce").withRequired(false).withArgument(
         abuilder.withName("numReduce").withMinimum(1).withMaximum(1).create()).withDescription(
@@ -141,7 +141,7 @@
         convergenceDelta = Double.parseDouble(cmdLine.getValue(convergenceDeltaOpt).toString());
       }
 
-      Class<? extends Vector> vectorClass = cmdLine.hasOption(vectorClassOpt) == false ? SparseVector.class
+      Class<? extends Vector> vectorClass = cmdLine.hasOption(vectorClassOpt) == false ? RandomAccessSparseVector.class
           : (Class<? extends Vector>) Class.forName(cmdLine.getValue(vectorClassOpt).toString());
 
       int maxIterations = 20;

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=899330&r1=899329&r2=899330&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 Thu Jan 14 17:54:30 2010
@@ -1,3 +1,20 @@
+/**
+ * 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 org.apache.hadoop.io.Writable;
@@ -19,6 +36,7 @@
     lengthSquared = v.lengthSquared;
   }
 
+  @Override
   public void write(DataOutput dataOutput) throws IOException {
     dataOutput.writeUTF(getClass().getName());
     dataOutput.writeUTF(this.getName() == null ? "" : this.getName());
@@ -31,8 +49,14 @@
     }
   }
 
+  @Override
   public void readFields(DataInput dataInput) throws IOException {
-    this.setName(dataInput.readUTF());
+    final 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();
     for (int i = 0; i < values.length; i++) {

Added: 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=899330&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/RandomAccessSparseVectorWritable.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/RandomAccessSparseVectorWritable.java Thu Jan 14 17:54:30 2010
@@ -0,0 +1,81 @@
+/**
+ * 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 org.apache.hadoop.io.Writable;
+import org.apache.mahout.math.map.OpenIntDoubleHashMap;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+
+
+public class RandomAccessSparseVectorWritable extends RandomAccessSparseVector implements Writable {
+
+  public RandomAccessSparseVectorWritable(Vector v) {
+    super(v);
+  }
+
+  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;
+    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 {
+    final 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 index = dataInput.readInt();
+      double value = dataInput.readDouble();
+      values.put(index, value);
+      i++;
+    }
+    assert (i == cardinality);
+    this.values = values;
+  }
+
+  
+
+}

Added: 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=899330&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/SequentialAccessSparseVectorWritable.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/SequentialAccessSparseVectorWritable.java Thu Jan 14 17:54:30 2010
@@ -0,0 +1,76 @@
+/**
+ * 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 org.apache.hadoop.io.Writable;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+
+
+public class SequentialAccessSparseVectorWritable extends SequentialAccessSparseVector implements Writable {
+
+  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;
+    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 {
+    final String className = dataInput.readUTF();
+    if(className.equals(getClass().getName())) {
+      setName(dataInput.readUTF());
+    } else {
+      setName(className); // we have already read the class name in VectorWritable
+    }
+    int cardinality = dataInput.readInt();
+    int size = dataInput.readInt();
+    OrderedIntDoubleMapping values = new OrderedIntDoubleMapping(size);
+    int i = 0;
+    for (; i < size; i++) {
+      values.set(dataInput.readInt(), dataInput.readDouble());
+    }
+    assert (i == size);
+    this.values = 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=899330&r1=899329&r2=899330&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 Thu Jan 14 17:54:30 2010
@@ -1,3 +1,20 @@
+/**
+ * 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 org.apache.hadoop.conf.Configured;
@@ -36,8 +53,10 @@
     Writable w = null;
     if(vector instanceof Writable) {
       w = (Writable) vector;
-    } else if(vector instanceof SparseVector) {
-      w = new SparseVectorWritable((SparseVector)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));
     }

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/ClusteringTestUtils.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/ClusteringTestUtils.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/ClusteringTestUtils.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/ClusteringTestUtils.java Thu Jan 14 17:54:30 2010
@@ -22,8 +22,6 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.SequenceFile;
-import org.apache.mahout.math.SparseVector;
-import org.apache.mahout.math.Vector;
 import org.apache.mahout.math.VectorWritable;
 
 import java.io.File;

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=899330&r1=899329&r2=899330&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 Thu Jan 14 17:54:30 2010
@@ -27,7 +27,7 @@
 import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
 import org.apache.mahout.clustering.ClusteringTestUtils;
-import org.apache.mahout.math.SparseVector;
+import org.apache.mahout.math.RandomAccessSparseVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.common.distance.DistanceMeasure;
 import org.apache.mahout.common.DummyOutputCollector;
@@ -69,7 +69,7 @@
     List<VectorWritable> points = new ArrayList<VectorWritable>();
     int i = 0;
     for (double[] fr : raw) {
-      Vector vec = new SparseVector(String.valueOf(i++), fr.length);
+      Vector vec = new RandomAccessSparseVector(String.valueOf(i++), fr.length);
       vec.assign(fr);
       points.add(new VectorWritable(vec));
     }
@@ -400,6 +400,7 @@
       testData.mkdir();
     }
     JobConf job = new JobConf(CanopyDriver.class);
+    job.setMapOutputValueClass(points.get(0).getClass());
     ClusteringTestUtils.writePointsToFile(points, "testdata/file1", fs, job);
     ClusteringTestUtils.writePointsToFile(points, "testdata/file2", fs, job);
     // now run the Canopy Driver
@@ -626,7 +627,8 @@
     if (!testData.exists()) {
       testData.mkdir();
     }
-    Configuration conf = new Configuration();
+    JobConf conf = new JobConf();
+    conf.setMapOutputValueClass(points.get(0).getClass());
     ClusteringTestUtils.writePointsToFile(points, "testdata/file1", fs, conf);
     ClusteringTestUtils.writePointsToFile(points, "testdata/file2", fs, conf);
     // now run the Job

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestMapReduce.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestMapReduce.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestMapReduce.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestMapReduce.java Thu Jan 14 17:54:30 2010
@@ -33,9 +33,13 @@
 import org.apache.mahout.clustering.dirichlet.models.SampledNormalDistribution;
 import org.apache.mahout.clustering.dirichlet.models.SampledNormalModel;
 import org.apache.mahout.clustering.kmeans.KMeansDriver;
-import org.apache.mahout.math.*;
 import org.apache.mahout.common.DummyOutputCollector;
 import org.apache.mahout.common.RandomUtils;
+import org.apache.mahout.math.DenseVector;
+import org.apache.mahout.math.JsonVectorAdapter;
+import org.apache.mahout.math.RandomAccessSparseVector;
+import org.apache.mahout.math.Vector;
+import org.apache.mahout.math.VectorWritable;
 
 import java.io.File;
 import java.io.IOException;
@@ -72,7 +76,7 @@
   }
 
   private void addSample(double[] values) {
-    Vector v = new SparseVector(2);
+    Vector v = new RandomAccessSparseVector(2);
     for (int j = 0; j < values.length; j++) {
       v.setQuick(j, values[j]);
     }

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=899330&r1=899329&r2=899330&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 Thu Jan 14 17:54:30 2010
@@ -26,7 +26,6 @@
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.mahout.clustering.ClusteringTestUtils;
 import org.apache.mahout.clustering.kmeans.TestKmeansClustering;
-import org.apache.mahout.math.SparseVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.common.distance.DistanceMeasure;
 import org.apache.mahout.common.DummyOutputCollector;

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=899330&r1=899329&r2=899330&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 Thu Jan 14 17:54:30 2010
@@ -26,13 +26,17 @@
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.mahout.clustering.ClusteringTestUtils;
 import org.apache.mahout.clustering.canopy.CanopyDriver;
-import org.apache.mahout.math.*;
 import org.apache.mahout.common.distance.DistanceMeasure;
 import org.apache.mahout.common.DummyOutputCollector;
 import org.apache.mahout.common.distance.EuclideanDistanceMeasure;
 import org.apache.mahout.common.HadoopUtil;
 import org.apache.mahout.common.distance.ManhattanDistanceMeasure;
 import org.apache.mahout.common.FileLineIterable;
+import org.apache.mahout.math.AbstractVector;
+import org.apache.mahout.math.DenseVector;
+import org.apache.mahout.math.RandomAccessSparseVector;
+import org.apache.mahout.math.Vector;
+import org.apache.mahout.math.VectorWritable;
 
 import java.io.File;
 import java.util.ArrayList;
@@ -138,7 +142,7 @@
     List<VectorWritable> points = new ArrayList<VectorWritable>();
     for (int i = 0; i < raw.length; i++) {
       double[] fr = raw[i];
-      Vector vec = new SparseVector(String.valueOf(i), fr.length);
+      Vector vec = new RandomAccessSparseVector(String.valueOf(i), fr.length);
       vec.assign(fr);
       points.add(new VectorWritable(vec));
     }

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=899330&r1=899329&r2=899330&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 Thu Jan 14 17:54:30 2010
@@ -33,7 +33,7 @@
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.mahout.clustering.ClusteringTestUtils;
-import org.apache.mahout.math.SparseVector;
+import org.apache.mahout.math.RandomAccessSparseVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.math.VectorWritable;
 
@@ -48,7 +48,7 @@
     List<VectorWritable> points = new ArrayList<VectorWritable>();
     int i = 0;
     for (double[] fr : raw) {
-      Vector vec = new SparseVector(String.valueOf(i++), fr.length);
+      Vector vec = new RandomAccessSparseVector(String.valueOf(i++), fr.length);
       vec.assign(fr);
       points.add(new VectorWritable(vec));
     }
@@ -90,6 +90,7 @@
     }
     
     JobConf job = new JobConf(RandomSeedGenerator.class);
+    job.setMapOutputValueClass(VectorWritable.class);
     ClusteringTestUtils.writePointsToFile(points, "testdata/random-input", fs, job);
     
     RandomSeedGenerator.buildRandom("testdata/random-input", "testdata/random-output", 4);

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=899330&r1=899329&r2=899330&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 Thu Jan 14 17:54:30 2010
@@ -30,7 +30,7 @@
 import org.apache.hadoop.io.Text;
 import org.apache.mahout.math.DenseMatrix;
 import org.apache.mahout.math.Matrix;
-import org.apache.mahout.math.SparseVector;
+import org.apache.mahout.math.RandomAccessSparseVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.common.RandomUtils;
 
@@ -48,8 +48,8 @@
    * @param numWords int number of words in the vocabulary
    * @param numWords E[count] for each word
    */
-  private SparseVector generateRandomDoc(int numWords, double sparsity) throws MathException {
-    SparseVector v = new SparseVector(numWords,(int)(numWords * sparsity));
+  private RandomAccessSparseVector generateRandomDoc(int numWords, double sparsity) throws MathException {
+    RandomAccessSparseVector v = new RandomAccessSparseVector(numWords,(int)(numWords * sparsity));
     PoissonDistribution dist = new PoissonDistributionImpl(sparsity);
     for (int i = 0; i < numWords; i++) {
       // random integer
@@ -98,7 +98,7 @@
     mapper.configure(state);
 
     for(int i = 0; i < NUM_TESTS; ++i) {
-      SparseVector v = generateRandomDoc(100,0.3);
+      RandomAccessSparseVector v = generateRandomDoc(100,0.3);
       int myNumWords = numNonZero(v);
       LDAMapper.Context mock = createMock(LDAMapper.Context.class);
 

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/common/distance/DefaultDistanceMeasureTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/common/distance/DefaultDistanceMeasureTest.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/common/distance/DefaultDistanceMeasureTest.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/common/distance/DefaultDistanceMeasureTest.java Thu Jan 14 17:54:30 2010
@@ -20,7 +20,7 @@
 import junit.framework.TestCase;
 
 import org.apache.mahout.math.DenseVector;
-import org.apache.mahout.math.SparseVector;
+import org.apache.mahout.math.RandomAccessSparseVector;
 import org.apache.mahout.math.Vector;
 
 public abstract class DefaultDistanceMeasureTest extends TestCase {
@@ -42,22 +42,22 @@
 
     vectors = new Vector[4];
     
-    vectors[0] = new SparseVector(5);
+    vectors[0] = new RandomAccessSparseVector(5);
     vectors[0].setQuick(0, 1);
     vectors[0].setQuick(3, 1);
     vectors[0].setQuick(4, 1);
 
-    vectors[1] = new SparseVector(5);
+    vectors[1] = new RandomAccessSparseVector(5);
     vectors[1].setQuick(0, 2);
     vectors[1].setQuick(3, 2);
     vectors[1].setQuick(4, 2);
 
-    vectors[2] = new SparseVector(5);
+    vectors[2] = new RandomAccessSparseVector(5);
     vectors[2].setQuick(0, 6);
     vectors[2].setQuick(3, 6);
     vectors[2].setQuick(4, 6);
     
-    vectors[3] = new SparseVector(5);
+    vectors[3] = new RandomAccessSparseVector(5);
 
     compare(distanceMeasure, vectors);
   }

Added: lucene/mahout/trunk/core/src/test/resources/word-list.txt
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/resources/word-list.txt?rev=899330&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/resources/word-list.txt (added)
+++ lucene/mahout/trunk/core/src/test/resources/word-list.txt Thu Jan 14 17:54:30 2010
@@ -0,0 +1,512 @@
+Algol
+Apollo
+Athens
+Atlantic
+Aztec
+Babylon
+Belfast
+Bradbury
+Brazilian
+Burbank
+Burlington
+Camelot
+Capricorn
+Cherokee
+Chicago
+Christmas
+Dakota
+December
+Dupont
+Eskimo
+Galveston
+Geiger
+Hamilton
+Istanbul
+Jamaica
+Jupiter
+Medusa
+Mohawk
+Montana
+Neptune
+Norwegian
+Oakland
+October
+Ohio
+Orlando
+Pacific
+Pandora
+Pegasus
+Pluto
+Saturday
+Scotland
+Trojan
+Virginia
+Vulcan
+Waterloo
+Wichita
+Wilmington
+Wyoming
+Yucatan
+Zulu
+aardvark
+absurd
+accrue
+acme
+adrift
+adroitness
+adult
+adviser
+afflict
+aftermath
+aggregate
+ahead
+aimless
+alkali
+allow
+almighty
+alone
+ammo
+amulet
+amusement
+ancient
+antenna
+apple
+applicant
+armistice
+article
+artist
+assume
+asteroid
+atlas
+atmosphere
+autopsy
+baboon
+backfield
+backward
+backwater
+banjo
+barbecue
+beaming
+bedlamp
+beehive
+beeswax
+befriend
+belowground
+berserk
+bifocals
+billiard
+bison
+blackjack
+blockade
+blowtorch
+bluebird
+bodyguard
+bombast
+bookseller
+bookshelf
+borderline
+bottomless
+brackish
+bravado
+breadline
+breakaway
+breakup
+brickyard
+briefcase
+businessman
+butterfat
+button
+buzzard
+candidate
+cannonball
+caravan
+caretaker
+celebrate
+cellulose
+cement
+certify
+chairlift
+chambermaid
+chatter
+checkup
+chisel
+choking
+chopper
+clamshell
+classic
+classroom
+cleanup
+clergyman
+clockwork
+cobra
+coherence
+combustion
+commando
+commence
+company
+component
+concert
+concurrent
+confidence
+conformist
+congregate
+consensus
+consulting
+corporate
+corrosion
+councilman
+cowbell
+crackdown
+cranky
+crossover
+crowfoot
+crucial
+crucifix
+crumpled
+crusade
+cubic
+cumbersome
+customer
+dashboard
+deadbolt
+decadence
+decimal
+deckhand
+designing
+detector
+detergent
+determine
+dictator
+dinosaur
+direction
+disable
+disbelief
+disruptive
+distortion
+document
+dogsled
+dragnet
+drainage
+dreadful
+drifter
+dropper
+drumbeat
+drunken
+dwelling
+eating
+edict
+egghead
+eightball
+embezzle
+enchanting
+endorse
+endow
+enlist
+enrollment
+enterprise
+equation
+equipment
+erase
+escapade
+escape
+everyday
+examine
+exceed
+existence
+exodus
+eyeglass
+eyetooth
+facial
+fallout
+fascinate
+filament
+finicky
+flagpole
+flatfoot
+flytrap
+forever
+fortitude
+fracture
+framework
+freedom
+frequency
+frighten
+gadgetry
+gazelle
+getaway
+glitter
+glossary
+glucose
+goggles
+goldfish
+gossamer
+graduate
+gravity
+gremlin
+guidance
+guitarist
+hamburger
+hamlet
+handiwork
+hazardous
+headwaters
+hemisphere
+hesitate
+hideaway
+highchair
+hockey
+holiness
+hurricane
+hydraulic
+impartial
+impetus
+inception
+indigo
+indoors
+indulge
+inertia
+infancy
+inferno
+informant
+insincere
+insurgent
+integrate
+intention
+inventive
+inverse
+involve
+island
+jawbone
+keyboard
+kickoff
+kiwi
+klaxon
+leprosy
+letterhead
+liberty
+locale
+lockup
+maritime
+matchmaker
+maverick
+megaton
+merit
+microscope
+microwave
+midsummer
+millionaire
+minnow
+miracle
+miser
+misnomer
+molasses
+molecule
+monument
+mosquito
+mural
+music
+narrative
+nebula
+necklace
+newborn
+newsletter
+nightbird
+obtuse
+offload
+onlooker
+optic
+opulent
+orca
+outfielder
+pandemic
+paperweight
+paragon
+paragraph
+paramount
+passenger
+payday
+peachy
+pedigree
+penetrate
+perceptive
+performance
+pharmacy
+pheasant
+phonetic
+photograph
+physique
+pioneer
+playhouse
+pocketful
+politeness
+positive
+potato
+preclude
+prefer
+preshrunk
+printer
+processor
+provincial
+prowler
+proximate
+puberty
+publisher
+pupil
+puppy
+pyramid
+python
+quadrant
+quantity
+quiver
+quota
+racketeer
+ragtime
+ratchet
+rebellion
+rebirth
+recipe
+recover
+reform
+regain
+reindeer
+rematch
+repay
+repellent
+replica
+reproduce
+resistor
+responsive
+retouch
+retraction
+retrieval
+retrospect
+revenge
+revenue
+revival
+revolver
+reward
+rhythm
+ribcage
+ringbolt
+robust
+rocker
+ruffled
+sailboat
+sandalwood
+sardonic
+savagery
+sawdust
+scallion
+scavenger
+scenic
+scorecard
+seabird
+select
+sensation
+sentence
+shadow
+shamrock
+showgirl
+skullcap
+skydive
+slingshot
+slowdown
+snapline
+snapshot
+snowcap
+snowslide
+sociable
+solo
+southward
+souvenir
+soybean
+spaniel
+spearhead
+specialist
+speculate
+spellbind
+spheroid
+spigot
+spindle
+spyglass
+stagehand
+stagnate
+stairway
+standard
+stapler
+steamship
+sterling
+stethoscope
+stockman
+stopwatch
+stormy
+stupendous
+sugar
+supportive
+surmount
+surrender
+suspense
+suspicious
+sweatband
+swelter
+sympathy
+tactics
+talon
+tambourine
+tapeworm
+telephone
+tempest
+therapist
+tiger
+tissue
+tobacco
+tolerance
+tomorrow
+tonic
+topmost
+torpedo
+tracker
+tradition
+transit
+trauma
+travesty
+treadmill
+trombonist
+trouble
+truncated
+tumor
+tunnel
+tycoon
+typewriter
+ultimate
+uncut
+undaunted
+underfoot
+unearth
+unicorn
+unify
+universe
+unravel
+unwind
+upcoming
+uproot
+upset
+upshot
+vacancy
+vagabond
+vapor
+vertigo
+village
+virus
+visitor
+vocalist
+voyager
+waffle
+wallet
+warranty
+watchword
+wayside
+whimsical
+willow
+woodlark
+yesteryear

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/InputDriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/InputDriver.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/InputDriver.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/InputDriver.java Thu Jan 14 17:54:30 2010
@@ -75,21 +75,22 @@
 
       String input = cmdLine.getValue(inputOpt, "testdata").toString();
       String output = cmdLine.getValue(outputOpt, "output").toString();
-      String vectorClassName = cmdLine.getValue(vectorOpt, "org.apache.mahout.math.SparseVector").toString();
-      runJob(input, output);
+      String vectorClassName = cmdLine.getValue(vectorOpt, "org.apache.mahout.math.RandomAccessSparseVector").toString();
+      runJob(input, output, vectorClassName);
     } catch (OptionException e) {
       LOG.error("Exception parsing command line: ", e);
       CommandLineUtil.printHelp(group);
     }
   }
 
-  public static void runJob(String input, String output) throws IOException {
+  public static void runJob(String input, String output, String vectorClassName) throws IOException {
     JobClient client = new JobClient();
     JobConf conf = new JobConf(InputDriver.class);
 
     conf.setOutputKeyClass(Text.class);
     conf.setOutputValueClass(VectorWritable.class);
     conf.setOutputFormat(SequenceFileOutputFormat.class);
+    conf.set("vector.implementation.class.name", vectorClassName);
     FileInputFormat.setInputPaths(conf, new Path(input));
     FileOutputFormat.setOutputPath(conf, new Path(output));
 

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=899330&r1=899329&r2=899330&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 Thu Jan 14 17:54:30 2010
@@ -25,6 +25,7 @@
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.mahout.math.Vector;
+import org.apache.mahout.math.VectorWritable;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -34,15 +35,17 @@
 import java.lang.reflect.InvocationTargetException;
 
 public class InputMapper extends MapReduceBase implements
-    Mapper<LongWritable, Text, Text, Vector> {
+    Mapper<LongWritable, Text, Text, VectorWritable> {
 
   private static final Pattern SPACE = Pattern.compile(" ");
 
   private Constructor<?> constructor;
 
+  private VectorWritable vectorWritable;
+
   @Override
   public void map(LongWritable key, Text values,
-      OutputCollector<Text, Vector> output, Reporter reporter) throws IOException {
+      OutputCollector<Text, VectorWritable> output, Reporter reporter) throws IOException {
     String[] numbers = SPACE.split(values.toString());
     // sometimes there are multiple separator spaces
     List<Double> doubles = new ArrayList<Double>();
@@ -51,11 +54,13 @@
         doubles.add(Double.valueOf(value));
     }
     try {
-      Vector result = (Vector) constructor.newInstance(doubles.size());//new DenseVector(doubles.size());
+      Vector result = (Vector) constructor.newInstance(doubles.size());
       int index = 0;
-      for (Double d : doubles)
+      for (Double d : doubles) {
         result.set(index++, d);
-      output.collect(new Text(String.valueOf(index)), result);
+      }
+      vectorWritable.set(result);
+      output.collect(new Text(String.valueOf(index)), vectorWritable);
 
     } catch (InstantiationException e) {
       throw new IllegalStateException(e);
@@ -69,11 +74,15 @@
 
   @Override
   public void configure(JobConf job) {
-    Class<? extends Vector> outputClass = (Class<? extends Vector>) job.getOutputValueClass();
+    vectorWritable = new VectorWritable();
+    String vectorImplClassName = job.get("vector.implementation.class.name");
     try {
+      Class<? extends Vector> outputClass = (Class<? extends Vector>) job.getClassByName(vectorImplClassName);
       constructor = outputClass.getConstructor(int.class);
     } catch (NoSuchMethodException e) {
       throw new IllegalStateException(e);
+    } catch (ClassNotFoundException e) {
+      throw new IllegalStateException(e);
     }
 
   }

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/Job.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/Job.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/Job.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/Job.java Thu Jan 14 17:54:30 2010
@@ -62,7 +62,7 @@
           withDescription("The Distance Measure to use.  Default is SquaredEuclidean").withShortName("m").create();
       Option vectorClassOpt = obuilder.withLongName("vectorClass").withRequired(false).withArgument(
           abuilder.withName("vectorClass").withMinimum(1).withMaximum(1).create()).
-          withDescription("The Vector implementation class name.  Default is SparseVector.class").withShortName("v").create();
+          withDescription("The Vector implementation class name.  Default is RandomAccessSparseVector.class").withShortName("v").create();
 
       Option t1Opt = obuilder.withLongName("t1").withRequired(false).withArgument(
           abuilder.withName("t1").withMinimum(1).withMaximum(1).create()).
@@ -96,7 +96,7 @@
         String measureClass = cmdLine.getValue(
             measureClassOpt, "org.apache.mahout.common.distance.EuclideanDistanceMeasure").toString();
 
-        String className =  cmdLine.getValue(vectorClassOpt, "org.apache.mahout.math.SparseVector").toString();
+        String className =  cmdLine.getValue(vectorClassOpt, "org.apache.mahout.math.RandomAccessSparseVector").toString();
         Class<? extends Vector> vectorClass = Class.forName(className).asSubclass(Vector.class);
         double t1 = Double.parseDouble(cmdLine.getValue(t1Opt, "80").toString());
         double t2 = Double.parseDouble(cmdLine.getValue(t2Opt, "55").toString());
@@ -142,7 +142,7 @@
       dfs.delete(outPath, true);
     String directoryContainingConvertedInput = output
         + Constants.DIRECTORY_CONTAINING_CONVERTED_INPUT;
-    InputDriver.runJob(input, directoryContainingConvertedInput);
+    InputDriver.runJob(input, directoryContainingConvertedInput, "org.apache.mahout.math.RandomAccessSparseVector");
     CanopyClusteringJob.runJob(directoryContainingConvertedInput, output,
         measureClassName, t1, t2);
   }

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/dirichlet/Job.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/dirichlet/Job.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/dirichlet/Job.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/dirichlet/Job.java Thu Jan 14 17:54:30 2010
@@ -103,7 +103,7 @@
       int maxIterations = Integer.parseInt(cmdLine.getValue(maxIterOpt, "5").toString());
       double alpha_0 = Double.parseDouble(cmdLine.getValue(mOpt, "1.0").toString());
       int numReducers = Integer.parseInt(cmdLine.getValue(redOpt, "1").toString());
-      String vectorClassName = cmdLine.getValue(vectorOpt, "org.apache.mahout.math.SparseVector").toString();
+      String vectorClassName = cmdLine.getValue(vectorOpt, "org.apache.mahout.math.RandomAccessSparseVector").toString();
       Class<? extends Vector> vectorClass = (Class<? extends Vector>) Class.forName(vectorClassName);
       runJob(input, output, modelFactory, numModels, maxIterations, alpha_0, numReducers, vectorClass);
     } catch (OptionException e) {
@@ -140,7 +140,7 @@
     }
     fs.mkdirs(outPath);
     final String directoryContainingConvertedInput = output + DIRECTORY_CONTAINING_CONVERTED_INPUT;
-    InputDriver.runJob(input, directoryContainingConvertedInput);
+    InputDriver.runJob(input, directoryContainingConvertedInput, "org.apache.mahout.math.RandomAccessSparseVector");
     DirichletDriver.runJob(directoryContainingConvertedInput, output + "/state", modelFactory,
         numModels, maxIterations, alpha_0, numReducers);
     printResults(output + "/state", modelFactory, maxIterations, numModels,

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/kmeans/Job.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/kmeans/Job.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/kmeans/Job.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/kmeans/Job.java Thu Jan 14 17:54:30 2010
@@ -74,7 +74,7 @@
         "The t2 value to use.").withShortName("m").create();
     Option vectorClassOpt = obuilder.withLongName("vectorClass").withRequired(false).withArgument(
         abuilder.withName("vectorClass").withMinimum(1).withMaximum(1).create()).withDescription(
-        "The Vector implementation class name.  Default is SparseVector.class").withShortName("v").create();
+        "The Vector implementation class name.  Default is RandomAccessSparseVector.class").withShortName("v").create();
 
     Option helpOpt = DefaultOptionCreator.helpOption();
 
@@ -97,7 +97,7 @@
       double t2 = Double.parseDouble(cmdLine.getValue(t2Opt, "55").toString());
       double convergenceDelta = Double.parseDouble(cmdLine.getValue(convergenceDeltaOpt, "0.5").toString());
       int maxIterations = Integer.parseInt(cmdLine.getValue(maxIterationsOpt, 10).toString());
-      String className = cmdLine.getValue(vectorClassOpt, "org.apache.mahout.math.SparseVector").toString();
+      String className = cmdLine.getValue(vectorClassOpt, "org.apache.mahout.math.RandomAccessSparseVector").toString();
       Class<? extends Vector> vectorClass = Class.forName(className).asSubclass(Vector.class);
 
       runJob(input, output, measureClass, t1, t2, convergenceDelta, maxIterations);
@@ -139,7 +139,7 @@
     final String directoryContainingConvertedInput = output
         + DIRECTORY_CONTAINING_CONVERTED_INPUT;
     System.out.println("Preparing Input");
-    InputDriver.runJob(input, directoryContainingConvertedInput);
+    InputDriver.runJob(input, directoryContainingConvertedInput, "org.apache.mahout.math.RandomAccessSparseVector");
     System.out.println("Running Canopy to get initial clusters");
     CanopyDriver.runJob(directoryContainingConvertedInput, output
         + CanopyClusteringJob.DEFAULT_CANOPIES_OUTPUT_DIRECTORY, measureClass,

Modified: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/AbstractVector.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/AbstractVector.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/AbstractVector.java (original)
+++ lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/AbstractVector.java Thu Jan 14 17:54:30 2010
@@ -36,11 +36,21 @@
 
   private String name;
 
+  protected int size;
+
+  protected double lengthSquared = -1;
+
   protected AbstractVector() {
+    this(null, 0);
   }
 
   protected AbstractVector(String name) {
+    this(name, 0);
+  }
+
+  protected AbstractVector(String name, int size) {
     this.name = name;
+    this.size = size;
   }
 
   /**
@@ -52,6 +62,16 @@
    */
   protected abstract Matrix matrixLike(int rows, int columns);
 
+  @Override
+  public Vector viewPart(int offset, int length) {
+    if (length > size) {
+      throw new CardinalityException();
+    }
+    if (offset < 0 || offset + length > size) {
+      throw new IndexException();
+    }
+    return new VectorView(this, offset, length);
+  }
 
   @Override
   public Vector clone() {
@@ -161,6 +181,26 @@
   }
 
   @Override
+  public double getLengthSquared() {
+    if (lengthSquared >= 0.0) {
+      return lengthSquared;
+    }
+    return lengthSquared = dot(this);
+  }
+
+  @Override
+  public double getDistanceSquared(Vector v) {
+    double d = 0;
+    Iterator<Element> it = iterateNonZero();
+    Element e;
+    while(it.hasNext() && (e = it.next()) != null) {
+      final double diff = e.get() - v.getQuick(e.index());
+      d += (diff * diff);
+    }
+    return d;
+  }
+
+  @Override
   public double maxValue() {
     double result = Double.MIN_VALUE;
     for (int i = 0; i < size(); i++) {
@@ -211,6 +251,16 @@
   }
 
   @Override
+  public void addTo(Vector v) {
+    Iterator<Element> it = iterateNonZero();
+    Element e;
+    while(it.hasNext() && (e = it.next()) != null) {
+      int i = e.index();
+      v.setQuick(i, v.getQuick(i) + e.get());
+    }
+  }
+
+  @Override
   public void set(int index, double value) {
     if (index >= 0 && index < size()) {
       setQuick(index, value);
@@ -353,6 +403,11 @@
   }
 
   @Override
+  public int size() {
+    return size;  
+  }
+
+  @Override
   public String asFormatString() {
     Type vectorType = new TypeToken<Vector>() {
     }.getType();
@@ -431,19 +486,17 @@
 
   @Override
   public int hashCode() {
-    int prime = 31;
+    final int prime = 31;
     int result = prime + ((name == null) ? 0 : name.hashCode());
     result = prime * result + size();
-    Iterator<Element> iter = iterateNonZero(true);
+    Iterator<Element> iter = iterateNonZero();
     while (iter.hasNext()) {
       Element ele = iter.next();
-      result = prime * result + ele.index();
       long v = Double.doubleToLongBits(ele.get());
-      result = prime * result + (int) (v ^ (v >> 32));
+      result += (ele.index() * (int)(v^(v>>32)));
     }
-
     return result;
-  }
+   }
 
 
   @Override

Modified: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/DenseVector.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/DenseVector.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/DenseVector.java (original)
+++ lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/DenseVector.java Thu Jan 14 17:54:30 2010
@@ -17,9 +17,6 @@
 
 package org.apache.mahout.math;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
 import java.util.Arrays;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
@@ -59,7 +56,7 @@
   }
 
   /**
-   * Copy-constructor (for use in turning a SparseVector into a dense one, for example)
+   * Copy-constructor (for use in turning a sparse vector into a dense one, for example)
    * @param vector
    */
   public DenseVector(Vector vector) {
@@ -142,11 +139,6 @@
   }
 
   @Override
-  public Iterator<Vector.Element> iterateNonZero(boolean sorted) {
-    return new NonZeroIterator();
-  }
-
-  @Override
   public Iterator<Vector.Element> iterateAll() {
     return new AllIterator();
   }
@@ -312,8 +304,11 @@
 
   @Override
   public void addTo(Vector v) {
-    for (int i = 0; i < size(); i++) {
-      v.setQuick(i, get(i) + v.get(i));
+    if (v.size() != size()) {
+      throw new CardinalityException();
+    }
+    for (int i = 0; i < values.length; i++) {
+      v.setQuick(i, values[i] + v.getQuick(i));
     }
   }
 }

Modified: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/OrderedIntDoubleMapping.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/OrderedIntDoubleMapping.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/OrderedIntDoubleMapping.java (original)
+++ lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/OrderedIntDoubleMapping.java Thu Jan 14 17:54:30 2010
@@ -134,9 +134,9 @@
   public boolean equals(Object o) {
     if (o instanceof OrderedIntDoubleMapping) {
       OrderedIntDoubleMapping other = (OrderedIntDoubleMapping) o;
-      if (numMappings == other.getNumMappings()) {
+      if (numMappings == other.numMappings) {
         for (int i = 0; i < numMappings; i++) {
-          if (indices[i] != other.getIndices()[i] || values[i] != other.getValues()[i]) {
+          if (indices[i] != other.indices[i] || values[i] != other.values[i]) {
             return false;
           }
         }

Modified: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/PlusWithScaleFunction.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/PlusWithScaleFunction.java?rev=899330&r1=899329&r2=899330&view=diff
==============================================================================
--- lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/PlusWithScaleFunction.java (original)
+++ lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/PlusWithScaleFunction.java Thu Jan 14 17:54:30 2010
@@ -19,7 +19,9 @@
 
 public class PlusWithScaleFunction implements BinaryFunction {
 
-  private final double scale;
+  private double scale;
+
+  public PlusWithScaleFunction() {}
 
   public PlusWithScaleFunction(double scale) {
     this.scale = scale;
@@ -29,4 +31,8 @@
   public double apply(double arg1, double arg2) {
     return arg1 + scale * arg2;
   }
+
+  public void setScale(double scale) {
+    this.scale = scale;
+  }
 }

Added: lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/RandomAccessSparseVector.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/RandomAccessSparseVector.java?rev=899330&view=auto
==============================================================================
--- lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/RandomAccessSparseVector.java (added)
+++ lucene/mahout/trunk/math/src/main/java/org/apache/mahout/math/RandomAccessSparseVector.java Thu Jan 14 17:54:30 2010
@@ -0,0 +1,297 @@
+/**
+ * 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.util.Iterator;
+import java.util.NoSuchElementException;
+
+import org.apache.mahout.math.function.IntDoubleProcedure;
+import org.apache.mahout.math.list.IntArrayList;
+import org.apache.mahout.math.map.OpenIntDoubleHashMap;
+
+
+/** Implements vector that only stores non-zero doubles */
+public class RandomAccessSparseVector extends AbstractVector {
+
+  protected OpenIntDoubleHashMap values;
+
+  /** For serialization purposes only. */
+  public RandomAccessSparseVector() {
+  }
+
+  public RandomAccessSparseVector(int cardinality) {
+    this(null, cardinality, cardinality / 8); // arbitrary estimate of
+    // 'sparseness'
+  }
+
+  public RandomAccessSparseVector(int cardinality, int size) {
+    this(null, cardinality, size);
+  }
+
+  public RandomAccessSparseVector(String name, int cardinality) {
+    this(name, cardinality, cardinality / 8); // arbitrary estimate of
+    // 'sparseness'
+  }
+
+  public RandomAccessSparseVector(String name, int cardinality, int size) {
+    super(name, cardinality);
+    values = new OpenIntDoubleHashMap(size);
+  }
+
+  public RandomAccessSparseVector(Vector other) {
+    this(other.getName(), other.size(), other.getNumNondefaultElements());
+    Iterator<Vector.Element> it = other.iterateNonZero();
+    Vector.Element e;
+    while(it.hasNext() && (e = it.next()) != null) {
+      values.put(e.index(), e.get());
+    }
+  }
+
+  @Override
+  protected Matrix matrixLike(int rows, int columns) {
+    int[] cardinality = {rows, columns};
+    return new SparseRowMatrix(cardinality);
+  }
+
+  @Override
+  public RandomAccessSparseVector clone() {
+    RandomAccessSparseVector clone = (RandomAccessSparseVector) super.clone();
+    clone.values = (OpenIntDoubleHashMap)values.clone();
+    return clone;
+  }
+
+  @Override
+  public double getQuick(int index) {
+    return values.get(index);
+  }
+
+  @Override
+  public void setQuick(int index, double value) {
+    values.put(index, value);
+  }
+
+  @Override
+  public int getNumNondefaultElements() {
+    return values.size();
+  }
+
+  @Override
+  public RandomAccessSparseVector like() {
+    int numValues = 256;
+    if (values != null) {
+      numValues = values.size();
+    }
+    return new RandomAccessSparseVector(size(), numValues);
+  }
+
+  @Override
+  public Vector like(int newCardinality) {
+    int numValues = 256;
+    if (values != null) {
+      numValues = values.size();
+    }
+    return new RandomAccessSparseVector(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 NonZeroIterator} over the Elements.
+   * @see #getElement(int)
+   */
+  @Override
+  public java.util.Iterator<Vector.Element> iterateNonZero() {
+    return new NonZeroIterator(false);
+  }
+  
+  @Override
+  public Iterator<Vector.Element> iterateAll() {
+    return new AllIterator();
+  }
+
+  /**
+   * Indicate whether the two objects are the same or not. Two {@link org.apache.mahout.math.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)
+   */
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof Vector)) {
+      return false;
+    }
+
+    Vector that = (Vector) o;
+    String thisName = getName();
+    String thatName = that.getName();
+    if (this.size() != that.size()) {
+      return false;
+    }
+    if (thisName != null && thatName != null && !thisName.equals(thatName)) {
+      return false;
+    } else if ((thisName != null && thatName == null)
+        || (thatName != null && thisName == null)) {
+      return false;
+    }
+
+    return equivalent(this, that);
+  }
+
+  private class AllIterator implements java.util.Iterator<Vector.Element> {
+    private int offset = 0;
+    private final Element element = new Element(0);
+
+    @Override
+    public boolean hasNext() {
+      return offset < size();
+    }
+
+    @Override
+    public Vector.Element next() {
+      if (offset >= size()) {
+        throw new NoSuchElementException();
+      }
+      element.ind = offset++;
+      return element;
+    }
+
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+
+  private class NonZeroIterator implements java.util.Iterator<Vector.Element> {
+    private int offset = 0;
+    private final Element element = new Element(0);
+
+    private IntArrayList intArrList =  values.keys();
+    
+    public NonZeroIterator(boolean sorted) {
+      if (sorted) {
+        intArrList.sort();
+      }      
+    }
+
+    @Override
+    public boolean hasNext() {
+      return offset < intArrList.size();
+    }
+
+    @Override
+    public Element next() {
+      if (offset < intArrList.size()) {
+        element.ind = intArrList.get(offset++);
+        return element;
+      }
+      throw new NoSuchElementException();
+    }
+
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  @Override
+  public Vector.Element getElement(int index) {
+    return new Element(index);
+  }
+
+  public class Element implements Vector.Element {
+    private int ind;
+
+    public Element(int ind) {
+      this.ind = ind;
+    }
+
+    @Override
+    public double get() {
+      return values.get(ind);
+    }
+
+    @Override
+    public int index() {
+      return ind;
+    }
+
+    @Override
+    public void set(double value) {
+      values.put(ind, value);
+    }
+  }
+
+  private class DistanceSquared implements IntDoubleProcedure {
+    Vector v;
+    public double result = 0.0;
+
+    public DistanceSquared(Vector v) {
+      this.v = v;
+    }
+
+    @Override
+    public boolean apply(int key, double value) {
+      double centroidValue = v.get(key);
+      double delta = value - centroidValue;
+      result += (delta * delta) - (centroidValue * centroidValue);
+      return true;
+    }
+  }
+
+  @Override
+  public double getDistanceSquared(Vector v) {
+    //TODO: Check sizes?
+
+    DistanceSquared distanceSquared = new DistanceSquared(v);
+    values.forEachPair(distanceSquared);
+    double result = distanceSquared.result;    
+    return result;
+  }
+
+  private class AddToVector implements IntDoubleProcedure {
+    Vector v;
+
+    public AddToVector(Vector v) {
+      this.v = v;
+    }
+
+    @Override
+    public boolean apply(int key, double value) {
+      v.set(key, value + v.get(key));
+      return true;
+    }
+  }
+
+  @Override
+  public void addTo(Vector v) {
+    if (v.size() != size()) {
+      throw new CardinalityException();
+    }
+    AddToVector addToVector = new AddToVector(v);
+    values.forEachPair(addToVector);
+  }
+
+}