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

svn commit: r786738 - in /lucene/mahout/trunk: core/src/main/java/org/apache/mahout/clustering/canopy/ core/src/main/java/org/apache/mahout/matrix/ core/src/test/java/org/apache/mahout/clustering/canopy/ examples/src/main/java/org/apache/mahout/cluster...

Author: jeastman
Date: Sat Jun 20 00:48:21 2009
New Revision: 786738

URL: http://svn.apache.org/viewvc?rev=786738&view=rev
Log:
MAHOUT-136
- Modified CanopyMapper and CanopyReducer to produce and consume Canopy centroids as Writable values vs. previous formatStrings
- Modified CanopyMapper to specify SparseVector output from mapper
- Fixed null name hash() bug in SparseVector
- Modified Canopy.emitPointToExistingCanopies to emit only canopy id and not full serialized canopy. 
- This eliminates the need for the OutputDriver and OutputMapper in synthetic control example so they are deleted.
- Updated unit tests; all tests run
- Synthetic control example runs

Removed:
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/OutputDriver.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/OutputMapper.java
Modified:
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/Canopy.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyDriver.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/canopy/CanopyReducer.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/SparseVector.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/canopy/TestCanopyCreation.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/syntheticcontrol/canopy/Job.java

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/Canopy.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/Canopy.java?rev=786738&r1=786737&r2=786738&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 Sat Jun 20 00:48:21 2009
@@ -72,12 +72,11 @@
   /**
    * Create a new Canopy containing the given point
    * 
-   * @param point
-   *            a point in vector space
+   * @param point a point in vector space
    */
   public Canopy(Vector point) {
     this.canopyId = nextCanopyId++;
-    this.center = point;
+    this.center = point.clone();
     this.pointTotal = point.clone();
     this.numPoints = 1;
   }
@@ -85,14 +84,12 @@
   /**
    * Create a new Canopy containing the given point and canopyId
    * 
-   * @param point
-   *            a point in vector space
-   * @param canopyId
-   *            an int identifying the canopy local to this process only
+   * @param point a point in vector space
+   * @param canopyId an int identifying the canopy local to this process only
    */
   public Canopy(Vector point, int canopyId) {
     this.canopyId = canopyId;
-    this.center = point;
+    this.center = point.clone();
     this.pointTotal = point.clone();
     this.numPoints = 1;
   }
@@ -100,8 +97,7 @@
   /**
    * Configure the Canopy and its distance measure
    * 
-   * @param job
-   *            the JobConf for this job
+   * @param job the JobConf for this job
    */
   public static void configure(JobConf job) {
     try {
@@ -139,13 +135,12 @@
    * This is the same algorithm as the reference but inverted to iterate over
    * existing canopies instead of the points. Because of this it does not need
    * to actually store the points, instead storing a total points vector and the
-   * number of points. From this a centroid can be computed. <p/> This method is
-   * used by the CanopyReducer.
+   * number of points. From this a centroid can be computed.
+   * <p/>
+   * This method is used by the CanopyReducer.
    * 
-   * @param point
-   *            the point to be added
-   * @param canopies
-   *            the List<Canopy> to be appended
+   * @param point the point to be added
+   * @param canopies the List<Canopy> to be appended
    */
   public static void addPointToCanopies(Vector point, List<Canopy> canopies) {
     boolean pointStronglyBound = false;
@@ -164,12 +159,9 @@
    * and to emit the point and its covering canopies to the output. The
    * CanopyCombiner will then sum the canopy points and produce the centroids.
    * 
-   * @param point
-   *            the point to be added
-   * @param canopies
-   *            the List<Canopy> to be appended
-   * @param collector
-   *            an OutputCollector in which to emit the point
+   * @param point the point to be added
+   * @param canopies the List<Canopy> to be appended
+   * @param collector an OutputCollector in which to emit the point
    */
   public static void emitPointToNewCanopies(Vector point,
       List<Canopy> canopies, OutputCollector<Text, Text> collector)
@@ -194,15 +186,11 @@
    * point is not covered by any canopies (due to canopy centroid clustering),
    * emit the point to the closest covering canopy.
    * 
-   * @param point
-   *            the point to be added
-   * @param canopies
-   *            the List<Canopy> to be appended
-   * @param writable
-   *            the original Writable from the input, may include arbitrary
-   *            payload information after the point [...]<payload>
-   * @param collector
-   *            an OutputCollector in which to emit the point
+   * @param point the point to be added
+   * @param canopies the List<Canopy> to be appended
+   * @param writable the original Writable from the input, may include arbitrary
+   *        payload information after the point [...]<payload>
+   * @param collector an OutputCollector in which to emit the point
    */
   public static void emitPointToExistingCanopies(Vector point,
       List<Canopy> canopies, Text writable,
@@ -214,7 +202,7 @@
       double dist = measure.distance(canopy.getCenter(), point);
       if (dist < t1) {
         isCovered = true;
-        collector.collect(new Text(formatCanopy(canopy)), writable);
+        collector.collect(new Text(canopy.getIdentifier()), writable);
       } else if (dist < minDist) {
         minDist = dist;
         closest = canopy;
@@ -223,7 +211,7 @@
     // if the point is not contained in any canopies (due to canopy centroid
     // clustering), emit the point to the closest covering canopy.
     if (!isCovered)
-      collector.collect(new Text(formatCanopy(closest)), writable);
+      collector.collect(new Text(closest.getIdentifier()), writable);
   }
 
   /**
@@ -239,8 +227,7 @@
   /**
    * Decodes and returns a Canopy from the formattedString
    * 
-   * @param formattedString
-   *            a String prouced by formatCanopy
+   * @param formattedString a String prouced by formatCanopy
    * @return a new Canopy
    */
   public static Canopy decodeCanopy(String formattedString) {
@@ -248,7 +235,8 @@
     String id = formattedString.substring(0, beginIndex);
     String centroid = formattedString.substring(beginIndex);
     if (id.charAt(0) == 'C') {
-      int canopyId = Integer.parseInt(formattedString.substring(1, beginIndex - 2));
+      int canopyId = Integer.parseInt(formattedString.substring(1,
+          beginIndex - 2));
       Vector canopyCentroid = AbstractVector.decodeVector(centroid);
       return new Canopy(canopyCentroid, canopyId);
     }
@@ -258,8 +246,7 @@
   /**
    * Add a point to the canopy
    * 
-   * @param point
-   *            some point to add
+   * @param point some point to add
    */
   public void addPoint(Vector point) {
     numPoints++;
@@ -271,8 +258,7 @@
    * Emit the point to the collector, keyed by the canopy's formatted
    * representation
    * 
-   * @param point
-   *            a point to emit.
+   * @param point a point to emit.
    */
   public void emitPoint(Vector point, OutputCollector<Text, Text> collector)
       throws IOException {
@@ -314,10 +300,10 @@
   /**
    * Compute the centroid by averaging the pointTotals
    * 
-   * @return a point which is the new centroid
+   * @return a SparseVector (required by Mapper) which is the new centroid
    */
-  public Vector computeCentroid() {
-    Vector result = new SparseVector(pointTotal.size());
+  public SparseVector computeCentroid() {
+    SparseVector result = new SparseVector(pointTotal.size());
     for (int i = 0; i < pointTotal.size(); i++)
       result.set(i, pointTotal.get(i) / numPoints);
     return result;
@@ -326,8 +312,7 @@
   /**
    * Return if the point is covered by this canopy
    * 
-   * @param point
-   *            a point
+   * @param point a point
    * @return if the point is covered
    */
   public boolean covers(Vector point) {

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=786738&r1=786737&r2=786738&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 Sat Jun 20 00:48:21 2009
@@ -17,6 +17,8 @@
 
 package org.apache.mahout.clustering.canopy;
 
+import java.io.IOException;
+
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
@@ -25,8 +27,7 @@
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
-
-import java.io.IOException;
+import org.apache.mahout.matrix.SparseVector;
 
 public class CanopyDriver {
 
@@ -60,6 +61,8 @@
     conf.set(Canopy.T1_KEY, String.valueOf(t1));
     conf.set(Canopy.T2_KEY, String.valueOf(t2));
 
+    conf.setMapOutputKeyClass(Text.class);
+    conf.setMapOutputValueClass(SparseVector.class);
     conf.setOutputKeyClass(Text.class);
     conf.setOutputValueClass(Text.class);
 

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=786738&r1=786737&r2=786738&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 Sat Jun 20 00:48:21 2009
@@ -25,6 +25,7 @@
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.mahout.matrix.AbstractVector;
+import org.apache.mahout.matrix.SparseVector;
 import org.apache.mahout.matrix.Vector;
 
 import java.io.IOException;
@@ -32,15 +33,15 @@
 import java.util.List;
 
 public class CanopyMapper extends MapReduceBase implements
-    Mapper<WritableComparable<?>, Text, Text, Text> {
+    Mapper<WritableComparable<?>, Text, Text, Vector> {
 
   private final List<Canopy> canopies = new ArrayList<Canopy>();
 
-  private OutputCollector<Text, Text> outputCollector;
+  private OutputCollector<Text, Vector> outputCollector;
 
   @Override
   public void map(WritableComparable<?> key, Text values,
-      OutputCollector<Text, Text> output, Reporter reporter) throws IOException {
+      OutputCollector<Text, Vector> output, Reporter reporter) throws IOException {
     outputCollector = output;
     Vector point = AbstractVector.decodeVector(values.toString());
     Canopy.addPointToCanopies(point, canopies);
@@ -52,14 +53,17 @@
     Canopy.configure(job);
   }
 
-  /* (non-Javadoc)
+  /*
+   * (non-Javadoc)
+   * 
    * @see org.apache.hadoop.mapred.MapReduceBase#close()
    */
   @Override
   public void close() throws IOException {
-    for (Canopy canopy : canopies)
-      outputCollector.collect(new Text("centroid"), new Text(canopy
-          .computeCentroid().asFormatString()));
+    for (Canopy canopy : canopies) {
+      SparseVector centroid = canopy.computeCentroid();
+      outputCollector.collect(new Text("centroid"), centroid);
+    }
     super.close();
   }
 

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyReducer.java?rev=786738&r1=786737&r2=786738&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyReducer.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/canopy/CanopyReducer.java Sat Jun 20 00:48:21 2009
@@ -17,31 +17,29 @@
 
 package org.apache.mahout.clustering.canopy;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MapReduceBase;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reporter;
-import org.apache.mahout.matrix.AbstractVector;
 import org.apache.mahout.matrix.Vector;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
 public class CanopyReducer extends MapReduceBase implements
-        Reducer<Text, Text, Text, Text> {
+        Reducer<Text, Vector, Text, Text> {
 
   private final List<Canopy> canopies = new ArrayList<Canopy>();
 
   @Override
-  public void reduce(Text key, Iterator<Text> values,
+  public void reduce(Text key, Iterator<Vector> values,
                      OutputCollector<Text, Text> output, Reporter reporter) throws IOException {
     while (values.hasNext()) {
-      Text value = values.next();
-      Vector point = AbstractVector.decodeVector(value.toString());
+      Vector point = values.next();
       Canopy.addPointToCanopies(point, canopies);
     }
     for (Canopy canopy : canopies)

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/SparseVector.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/SparseVector.java?rev=786738&r1=786737&r2=786738&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/SparseVector.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/matrix/SparseVector.java Sat Jun 20 00:48:21 2009
@@ -44,11 +44,13 @@
   }
 
   public SparseVector(String name, int cardinality) {
-    this(name, cardinality, cardinality / 8); // arbitrary estimate of 'sparseness'
+    this(name, cardinality, cardinality / 8); // arbitrary estimate of
+                                              // 'sparseness'
   }
 
   public SparseVector(int cardinality) {
-    this(null, cardinality, cardinality / 8); // arbitrary estimate of 'sparseness'
+    this(null, cardinality, cardinality / 8); // arbitrary estimate of
+                                              // 'sparseness'
   }
 
   @Override
@@ -117,37 +119,42 @@
   }
 
   /**
-   * Indicate whether the two objects are the same or not.  Two {@link org.apache.mahout.matrix.Vector}s can be equal
-   * even if the underlying implementation is not equal.
-   *  
+   * Indicate whether the two objects are the same or not. Two
+   * {@link org.apache.mahout.matrix.Vector}s can be equal even if the
+   * underlying implementation is not equal.
+   * 
    * @param o The object to compare
-   * @return true if the objects have the same cell values and same name, false otherwise.
-   *
-   * * @see AbstractVector#strictEquivalence(Vector, Vector)
-   * @see AbstractVector#equivalent(Vector, Vector) 
+   * @return true if the objects have the same cell values and same name, false
+   *         otherwise.
+   * 
+   *         * @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;
+    if (this == o)
+      return true;
+    if (!(o instanceof Vector))
+      return false;
 
     Vector that = (Vector) o;
-    if (this.size() != that.size()) return false;
+    if (this.size() != that.size())
+      return false;
 
     if (that instanceof SparseVector) {
-      return (values == null ? ((SparseVector) that).values == null : values.equals(((SparseVector) that).values));
+      return (values == null ? ((SparseVector) that).values == null : values
+          .equals(((SparseVector) that).values));
     } else {
       return equivalent(this, that);
     }
 
   }
 
-
   @Override
   public int hashCode() {
     int result = (values != null ? values.hashCode() : 0);
     result = 31 * result + cardinality;
-    result = 31 * result + name.hashCode();
+    result = 31 * result + (name == null ? 0 : name.hashCode());
     return result;
   }
 
@@ -195,7 +202,7 @@
 
   @Override
   public void write(DataOutput dataOutput) throws IOException {
-    dataOutput.writeUTF(this.name==null? "": this.name);
+    dataOutput.writeUTF(this.name == null ? "" : this.name);
     dataOutput.writeInt(size());
     dataOutput.writeInt(getNumNondefaultElements());
     for (Vector.Element element : this) {

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=786738&r1=786737&r2=786738&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 Sat Jun 20 00:48:21 2009
@@ -17,7 +17,23 @@
 
 package org.apache.mahout.clustering.canopy;
 
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 import junit.framework.TestCase;
+
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.SequenceFile;
@@ -35,21 +51,6 @@
 import org.apache.mahout.utils.ManhattanDistanceMeasure;
 import org.apache.mahout.utils.UserDefinedDistanceMeasure;
 
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.io.FileOutputStream;
-import java.io.InputStreamReader;
-import java.io.FileInputStream;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.nio.charset.Charset;
-
 public class TestCanopyCreation extends TestCase {
   static final double[][] raw = { { 1, 1 }, { 2, 1 }, { 1, 2 }, { 2, 2 },
       { 3, 3 }, { 4, 4 }, { 5, 4 }, { 4, 5 }, { 5, 5 } };
@@ -80,14 +81,6 @@
     return points;
   }
 
-  private static List<Text> getFormattedPoints(List<Vector> points) {
-    List<Text> result = new ArrayList<Text>();
-    for (Vector point : points) {
-      result.add(new Text(point.asFormatString()));
-    }
-    return result;
-  }
-
   /**
    * Verify that the given canopies are equivalent to the referenceManhattan
    * 
@@ -342,7 +335,7 @@
    */
   public void testCanopyMapperManhattan() throws Exception {
     CanopyMapper mapper = new CanopyMapper();
-    DummyOutputCollector<Text, Text> collector = new DummyOutputCollector<Text, Text>();
+    DummyOutputCollector<Text, Vector> collector = new DummyOutputCollector<Text, Vector>();
     Canopy.config(manhattanDistanceMeasure, (3.1), (2.1));
     List<Vector> points = getPoints(raw);
     // map the data
@@ -351,12 +344,12 @@
     mapper.close();
     assertEquals("Number of map results", 1, collector.getData().size());
     // now verify the output
-    List<Text> data = collector.getValue("centroid");
+    List<Vector> data = collector.getValue("centroid");
     assertEquals("Number of centroids", 3, data.size());
     for (int i = 0; i < data.size(); i++)
       assertEquals("Centroid error",
-          manhattanCentroids.get(i).asFormatString(), AbstractVector
-              .decodeVector(data.get(i).toString()).asFormatString());
+          manhattanCentroids.get(i).asFormatString(), data.get(i)
+              .asFormatString());
   }
 
   /**
@@ -368,7 +361,7 @@
    */
   public void testCanopyMapperEuclidean() throws Exception {
     CanopyMapper mapper = new CanopyMapper();
-    DummyOutputCollector<Text, Text> collector = new DummyOutputCollector<Text, Text>();
+    DummyOutputCollector<Text, Vector> collector = new DummyOutputCollector<Text, Vector>();
     Canopy.config(euclideanDistanceMeasure, (3.1), (2.1));
     List<Vector> points = getPoints(raw);
     // map the data
@@ -377,12 +370,12 @@
     mapper.close();
     assertEquals("Number of map results", 1, collector.getData().size());
     // now verify the output
-    List<Text> data = collector.getValue("centroid");
+    List<Vector> data = collector.getValue("centroid");
     assertEquals("Number of centroids", 3, data.size());
     for (int i = 0; i < data.size(); i++)
       assertEquals("Centroid error",
-          euclideanCentroids.get(i).asFormatString(), AbstractVector
-              .decodeVector(data.get(i).toString()).asFormatString());
+          euclideanCentroids.get(i).asFormatString(), data.get(i)
+              .asFormatString());
   }
 
   /**
@@ -397,8 +390,7 @@
     DummyOutputCollector<Text, Text> collector = new DummyOutputCollector<Text, Text>();
     Canopy.config(manhattanDistanceMeasure, (3.1), (2.1));
     List<Vector> points = getPoints(raw);
-    List<Text> texts = getFormattedPoints(points);
-    reducer.reduce(new Text("centroid"), texts.iterator(), collector, null);
+    reducer.reduce(new Text("centroid"), points.iterator(), collector, null);
     reducer.close();
     Set<String> keys = collector.getKeys();
     assertEquals("Number of centroids", 3, keys.size());
@@ -424,8 +416,7 @@
     DummyOutputCollector<Text, Text> collector = new DummyOutputCollector<Text, Text>();
     Canopy.config(euclideanDistanceMeasure, (3.1), (2.1));
     List<Vector> points = getPoints(raw);
-    List<Text> texts = getFormattedPoints(points);
-    reducer.reduce(new Text("centroid"), texts.iterator(), collector, null);
+    reducer.reduce(new Text("centroid"), points.iterator(), collector, null);
     reducer.close();
     Set<String> keys = collector.getKeys();
     assertEquals("Number of centroids", 3, keys.size());
@@ -538,7 +529,8 @@
     Map<String, List<Text>> data = collector.getData();
     assertEquals("Number of map results", canopies.size(), data.size());
     for (Map.Entry<String, List<Text>> stringListEntry : data.entrySet()) {
-      Canopy canopy = Canopy.decodeCanopy(stringListEntry.getKey());
+      String key = stringListEntry.getKey();
+      Canopy canopy = findCanopy(key, canopies);
       List<Text> pts = stringListEntry.getValue();
       for (Writable ptDef : pts)
         assertTrue("Point not in canopy", canopy.covers(AbstractVector
@@ -546,6 +538,13 @@
     }
   }
 
+  private Canopy findCanopy(String key, List<Canopy> canopies) {
+    for (Canopy c : canopies)
+      if (c.getIdentifier().equals(key))
+        return c;
+    return null;
+  }
+
   /**
    * Story: User can cluster a subset of the points using a ClusterMapper and a
    * EuclideanDistanceMeasure.
@@ -567,7 +566,8 @@
     Map<String, List<Text>> data = collector.getData();
     assertEquals("Number of map results", canopies.size(), data.size());
     for (Map.Entry<String, List<Text>> stringListEntry : data.entrySet()) {
-      Canopy canopy = Canopy.decodeCanopy(stringListEntry.getKey());
+      String key = stringListEntry.getKey();
+      Canopy canopy = findCanopy(key, canopies);
       List<Text> pts = stringListEntry.getValue();
       for (Writable ptDef : pts)
         assertTrue("Point not in canopy", canopy.covers(AbstractVector
@@ -606,7 +606,8 @@
     // check the output
     data = collector.getData();
     for (Map.Entry<String, List<Text>> stringListEntry : data.entrySet()) {
-      Canopy canopy = Canopy.decodeCanopy(stringListEntry.getKey());
+      String key = stringListEntry.getKey();
+      Canopy canopy = findCanopy(key, canopies);
       List<Text> pts = stringListEntry.getValue();
       for (Writable ptDef : pts)
         assertTrue("Point not in canopy", canopy.covers(AbstractVector
@@ -645,7 +646,8 @@
     data = collector.getData();
     assertEquals("Number of map results", canopies.size(), data.size());
     for (Map.Entry<String, List<Text>> stringListEntry : data.entrySet()) {
-      Canopy canopy = Canopy.decodeCanopy(stringListEntry.getKey());
+      String key = stringListEntry.getKey();
+      Canopy canopy = findCanopy(key, canopies);
       List<Text> pts = stringListEntry.getValue();
       for (Writable ptDef : pts)
         assertTrue("Point not in canopy", canopy.covers(AbstractVector

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=786738&r1=786737&r2=786738&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 Sat Jun 20 00:48:21 2009
@@ -17,17 +17,16 @@
 
 package org.apache.mahout.clustering.syntheticcontrol.canopy;
 
+import static org.apache.mahout.clustering.syntheticcontrol.Constants.DIRECTORY_CONTAINING_CONVERTED_INPUT;
+
+import java.io.IOException;
+
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.mahout.clustering.canopy.CanopyClusteringJob;
 
-import java.io.IOException;
-
-import static org.apache.mahout.clustering.syntheticcontrol.Constants.CLUSTERED_POINTS_OUTPUT_DIRECTORY;
-import static org.apache.mahout.clustering.syntheticcontrol.Constants.DIRECTORY_CONTAINING_CONVERTED_INPUT;
-
 public class Job {
   private Job() {
   }
@@ -76,8 +75,6 @@
     InputDriver.runJob(input, directoryContainingConvertedInput);
     CanopyClusteringJob.runJob(directoryContainingConvertedInput, output, measureClassName,
         t1, t2);
-    OutputDriver.runJob(output + CanopyClusteringJob.DEFAULT_CLUSTER_OUTPUT_DIRECTORY, output + CLUSTERED_POINTS_OUTPUT_DIRECTORY);
-
   }
 
 }