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

svn commit: r652727 - in /lucene/mahout/trunk/core: ./ src/test/java/org/apache/mahout/clustering/canopy/ src/test/java/org/apache/mahout/clustering/kmeans/ src/test/java/org/apache/mahout/clustering/meanshift/ src/test/java/org/apache/mahout/utils/

Author: jeastman
Date: Thu May  1 18:18:21 2008
New Revision: 652727

URL: http://svn.apache.org/viewvc?rev=652727&view=rev
Log:
Replaced clustering unit test DummyOutputCollectors with Grant's genericized DummyOutputCollector in utils. Updated clustering unit tests with type information. All tests run.

- org.apache.mahout.clustering.canopy.DummyOutputCollector.java - removed
- org.apache.mahout.clustering.kmeans.DummyOutputCollector.java - removed
- org.apache.mahout.clustering.meanshift.DummyOutputCollector.java - removed
- org.apache.mahout.clustering.utils.DummyOutputCollector.java - added
- org.apache.mahout.clustering.canopy.TestCanopyCreation.java - updated
- org.apache.mahout.clustering.canopy.TestKmeansClustering.java - updated
- org.apache.mahout.clustering.canopy.TestMeanShift.java - updated





Added:
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/utils/DummyOutputCollector.java
Removed:
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/canopy/DummyOutputCollector.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/meanshift/DummyOutputCollector.java
Modified:
    lucene/mahout/trunk/core/build.xml
    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/kmeans/TestKmeansClustering.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/meanshift/TestMeanShift.java

Modified: lucene/mahout/trunk/core/build.xml
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/build.xml?rev=652727&r1=652726&r2=652727&view=diff
==============================================================================
--- lucene/mahout/trunk/core/build.xml (original)
+++ lucene/mahout/trunk/core/build.xml Thu May  1 18:18:21 2008
@@ -379,4 +379,71 @@
                    value="${java.compat.version}"/>
      </manifest>
   </target>
+
+  <!-- EXAMPLES -->
+
+  <target name="compile-examples"
+          description="Compile the examples source code."
+          depends="compile"
+          >
+    <mahout-javac destdir="${dest}/examples"
+                  classpathref="examples.classpath">
+      <src path="${src}/main/examples/bayes"/>
+      <!-- Add other example sources here -->
+    </mahout-javac>
+  </target>
+
+  <path id="examples.classpath">
+    <path refid="compile.classpath"/>
+    <pathelement location="${dest}/classes"/>
+  </path>
+  <path id="examples.run.classpath">
+    <path refid="examples.classpath"/>
+    <pathelement location="${dest}/examples"/>
+  </path>
+
+  <property name="working.dir" value="work"/>
+  <target name="check-files">
+    <available file="temp/20news-18828.tar.gz" property="20news-18828.exists"/>
+    <available file="${working.dir}/20news-18828" property="20news-18828.expanded"/>
+
+  </target>
+
+  <target name="get-20news-18828" unless="20news-18828.exists">
+    <get src="http://people.csail.mit.edu/jrennie/20Newsgroups/20news-18828.tar.gz"
+         dest="temp/20news-18828.tar.gz"/>
+
+  </target>
+  <target name="expand-20news-18828" unless="20news-18828.expanded">
+    <gunzip src="temp/20news-18828.tar.gz" dest="temp"/>
+    <untar src="temp/20news-18828.tar" dest="${working.dir}"/>
+  </target>
+
+  <target name="extract-20news-18828" depends="check-files, compile-examples" unless="reuters.extracted">
+    <mkdir dir="${working.dir}/20news-18828-collapse"/>
+    <java classname="org.apache.mahout.classifiers.bayes.ExtractTwentyNewsgroups" maxmemory="1024M" fork="true">
+      <classpath refid="examples.run.classpath"/>
+      <!--
+      Input format is:
+      inputDir outputDir label Analyzer character set
+      -->
+      <arg line="-p ${working.dir}/20news-18828/ -o ${working.dir}/20news-18828-collapse -a org.apache.lucene.analysis.standard.StandardAnalyzer -c UTF-8"/>
+    </java>
+  </target>
+
+  <target name="get-files" depends="check-files">
+    <mkdir dir="temp"/>
+    <antcall target="get-20news-18828"/>
+    <antcall target="expand-20news-18828"/>
+    <!--<antcall target="extract-20news-18828"/>-->
+  </target>
+  <target name="examples-job" depends="compile,compile-examples" description="Build example Job jar">
+    <jar jarfile="${dest}/${fullnamever}-ex.jar">
+      <zipfileset dir="${dest}/classes"/>
+      <zipfileset dir="${dest}/examples"/>
+      <zipfileset dir="${lib}" prefix="lib"
+                  includes="**/*.jar" excludes="hadoop-*.jar"/>
+    </jar>
+  </target>
+
 </project>

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=652727&r1=652726&r2=652727&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 May  1 18:18:21 2008
@@ -42,6 +42,7 @@
 import org.apache.mahout.matrix.SparseVector;
 import org.apache.mahout.matrix.Vector;
 import org.apache.mahout.utils.DistanceMeasure;
+import org.apache.mahout.utils.DummyOutputCollector;
 import org.apache.mahout.utils.EuclideanDistanceMeasure;
 import org.apache.mahout.utils.ManhattanDistanceMeasure;
 import org.apache.mahout.utils.UserDefinedDistanceMeasure;
@@ -345,7 +346,7 @@
   public void testCanopyMapperManhattan() throws Exception {
     CanopyMapper mapper = new CanopyMapper();
     CanopyCombiner combiner = new CanopyCombiner();
-    DummyOutputCollector collector = new DummyOutputCollector();
+    DummyOutputCollector<Text,Text> collector = new DummyOutputCollector<Text,Text>();
     Canopy.config(manhattanDistanceMeasure, (3.1), (2.1));
     List<Vector> points = getPoints(raw);
     // map the data
@@ -355,7 +356,7 @@
     // now combine the mapper output
     Canopy.config(manhattanDistanceMeasure, (3.1), (2.1));
     Map<String, List<Text>> mapData = collector.getData();
-    collector = new DummyOutputCollector();
+    collector = new DummyOutputCollector<Text,Text>();
     for (String key : mapData.keySet())
       combiner.reduce(new Text(key), mapData.get(key).iterator(), collector,
           null);
@@ -378,7 +379,7 @@
   public void testCanopyMapperEuclidean() throws Exception {
     CanopyMapper mapper = new CanopyMapper();
     CanopyCombiner combiner = new CanopyCombiner();
-    DummyOutputCollector collector = new DummyOutputCollector();
+    DummyOutputCollector<Text,Text> collector = new DummyOutputCollector<Text,Text>();
     Canopy.config(euclideanDistanceMeasure, (3.1), (2.1));
     List<Vector> points = getPoints(raw);
     // map the data
@@ -388,7 +389,7 @@
     // now combine the mapper output
     Canopy.config(euclideanDistanceMeasure, (3.1), (2.1));
     Map<String, List<Text>> mapData = collector.getData();
-    collector = new DummyOutputCollector();
+    collector = new DummyOutputCollector<Text,Text>();
     for (String key : mapData.keySet())
       combiner.reduce(new Text(key), mapData.get(key).iterator(), collector,
           null);
@@ -410,7 +411,7 @@
    */
   public void testCanopyReducerManhattan() throws Exception {
     CanopyReducer reducer = new CanopyReducer();
-    DummyOutputCollector collector = new DummyOutputCollector();
+    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);
@@ -437,7 +438,7 @@
    */
   public void testCanopyReducerEuclidean() throws Exception {
     CanopyReducer reducer = new CanopyReducer();
-    DummyOutputCollector collector = new DummyOutputCollector();
+    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);
@@ -537,7 +538,7 @@
     Canopy.config(manhattanDistanceMeasure, (3.1), (2.1));
     ClusterMapper mapper = new ClusterMapper();
     List<Canopy> canopies = new ArrayList<Canopy>();
-    DummyOutputCollector collector = new DummyOutputCollector();
+    DummyOutputCollector<Text,Text> collector = new DummyOutputCollector<Text,Text>();
     for (Vector centroid : manhattanCentroids)
       canopies.add(new Canopy(centroid));
     mapper.config(canopies);
@@ -566,7 +567,7 @@
     Canopy.config(euclideanDistanceMeasure, (3.1), (2.1));
     ClusterMapper mapper = new ClusterMapper();
     List<Canopy> canopies = new ArrayList<Canopy>();
-    DummyOutputCollector collector = new DummyOutputCollector();
+    DummyOutputCollector<Text,Text> collector = new DummyOutputCollector<Text,Text>();
     for (Vector centroid : euclideanCentroids)
       canopies.add(new Canopy(centroid));
     mapper.config(canopies);
@@ -595,7 +596,7 @@
     Canopy.config(manhattanDistanceMeasure, (3.1), (2.1));
     ClusterMapper mapper = new ClusterMapper();
     List<Canopy> canopies = new ArrayList<Canopy>();
-    DummyOutputCollector collector = new DummyOutputCollector();
+    DummyOutputCollector<Text,Text> collector = new DummyOutputCollector<Text,Text>();
     for (Vector centroid : manhattanCentroids)
       canopies.add(new Canopy(centroid));
     mapper.config(canopies);
@@ -607,8 +608,8 @@
     assertEquals("Number of map results", canopies.size(), data.size());
 
     // reduce the data
-    Reducer reducer = new IdentityReducer();
-    collector = new DummyOutputCollector();
+    Reducer<Text, Text, Text, Text> reducer = new IdentityReducer<Text, Text>();
+    collector = new DummyOutputCollector<Text,Text>();
     for (String key : data.keySet())
       reducer.reduce(new Text(key), data.get(key).iterator(), collector, null);
 
@@ -633,7 +634,7 @@
     Canopy.config(euclideanDistanceMeasure, (3.1), (2.1));
     ClusterMapper mapper = new ClusterMapper();
     List<Canopy> canopies = new ArrayList<Canopy>();
-    DummyOutputCollector collector = new DummyOutputCollector();
+    DummyOutputCollector<Text,Text> collector = new DummyOutputCollector<Text,Text>();
     for (Vector centroid : euclideanCentroids)
       canopies.add(new Canopy(centroid));
     mapper.config(canopies);
@@ -644,8 +645,8 @@
     Map<String, List<Text>> data = collector.getData();
 
     // reduce the data
-    Reducer reducer = new IdentityReducer();
-    collector = new DummyOutputCollector();
+    Reducer<Text, Text, Text, Text> reducer = new IdentityReducer<Text, Text>();
+    collector = new DummyOutputCollector<Text,Text>();
     for (String key : data.keySet())
       reducer.reduce(new Text(key), data.get(key).iterator(), collector, null);
 

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=652727&r1=652726&r2=652727&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 May  1 18:18:21 2008
@@ -35,12 +35,12 @@
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.mahout.clustering.canopy.CanopyDriver;
-import org.apache.mahout.clustering.canopy.DummyOutputCollector;
 import org.apache.mahout.matrix.AbstractVector;
 import org.apache.mahout.matrix.DenseVector;
 import org.apache.mahout.matrix.SparseVector;
 import org.apache.mahout.matrix.Vector;
 import org.apache.mahout.utils.DistanceMeasure;
+import org.apache.mahout.utils.DummyOutputCollector;
 import org.apache.mahout.utils.EuclideanDistanceMeasure;
 import org.apache.mahout.utils.ManhattanDistanceMeasure;
 
@@ -77,7 +77,6 @@
     super.tearDown();
   }
 
-  
   /**
    * This is the reference k-means implementation. Given its inputs it iterates
    * over the points and clusters until their centers converge or until the
@@ -195,7 +194,7 @@
     List<Vector> points = getPoints(reference);
     for (int k = 0; k < points.size(); k++) {
       // pick k initial cluster centers at random
-      DummyOutputCollector collector = new DummyOutputCollector();
+      DummyOutputCollector<Text, Text> collector = new DummyOutputCollector<Text, Text>();
       List<Cluster> clusters = new ArrayList<Cluster>();
       for (int i = 0; i < k + 1; i++) {
         Cluster cluster = new Cluster(points.get(i));
@@ -239,7 +238,7 @@
     List<Vector> points = getPoints(reference);
     for (int k = 0; k < points.size(); k++) {
       // pick k initial cluster centers at random
-      DummyOutputCollector collector = new DummyOutputCollector();
+      DummyOutputCollector<Text, Text> collector = new DummyOutputCollector<Text, Text>();
       List<Cluster> clusters = new ArrayList<Cluster>();
       for (int i = 0; i < k + 1; i++) {
         Vector vec = points.get(i);
@@ -257,7 +256,7 @@
       }
       // now combine the data
       KMeansCombiner combiner = new KMeansCombiner();
-      DummyOutputCollector collector2 = new DummyOutputCollector();
+      DummyOutputCollector<Text, Text> collector2 = new DummyOutputCollector<Text, Text>();
       for (String key : collector.getKeys())
         combiner.reduce(new Text(key), collector.getValue(key).iterator(),
             collector2, null);
@@ -295,7 +294,7 @@
     for (int k = 0; k < points.size(); k++) {
       System.out.println("K = " + k);
       // pick k initial cluster centers at random
-      DummyOutputCollector collector = new DummyOutputCollector();
+      DummyOutputCollector<Text, Text> collector = new DummyOutputCollector<Text, Text>();
       List<Cluster> clusters = new ArrayList<Cluster>();
       for (int i = 0; i < k + 1; i++) {
         Vector vec = points.get(i);
@@ -312,14 +311,14 @@
       }
       // now combine the data
       KMeansCombiner combiner = new KMeansCombiner();
-      DummyOutputCollector collector2 = new DummyOutputCollector();
+      DummyOutputCollector<Text, Text> collector2 = new DummyOutputCollector<Text, Text>();
       for (String key : collector.getKeys())
         combiner.reduce(new Text(key), collector.getValue(key).iterator(),
             collector2, null);
 
       // now reduce the data
       KMeansReducer reducer = new KMeansReducer();
-      DummyOutputCollector collector3 = new DummyOutputCollector();
+      DummyOutputCollector<Text, Text> collector3 = new DummyOutputCollector<Text, Text>();
       for (String key : collector2.getKeys())
         reducer.reduce(new Text(key), collector2.getValue(key).iterator(),
             collector3, null);
@@ -409,7 +408,7 @@
       BufferedReader reader = new BufferedReader(new FileReader(
           "output/points/part-00000"));
       int[] expect = expectedNumPoints[k];
-      DummyOutputCollector collector = new DummyOutputCollector();
+      DummyOutputCollector<Text, Text> collector = new DummyOutputCollector<Text, Text>();
       while (reader.ready()) {
         String line = reader.readLine();
         String[] lineParts = line.split("\t");
@@ -460,7 +459,7 @@
     assertEquals("output dir files?", 4, outFiles.length);
     BufferedReader reader = new BufferedReader(new FileReader(
         "output/points/part-00000"));
-    DummyOutputCollector collector = new DummyOutputCollector();
+    DummyOutputCollector<Text, Text> collector = new DummyOutputCollector<Text, Text>();
     while (reader.ready()) {
       String line = reader.readLine();
       String[] lineParts = line.split("\t");
@@ -488,7 +487,7 @@
   /**
    * Split pattern for {@link #decodePoint(String)}.
    */
-  
+
   public static void writePointsToFile(List<Vector> points, String fileName)
       throws IOException {
     writePointsToFileWithPayload(points, fileName, "");

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/meanshift/TestMeanShift.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/meanshift/TestMeanShift.java?rev=652727&r1=652726&r2=652727&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/meanshift/TestMeanShift.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/meanshift/TestMeanShift.java Thu May  1 18:18:21 2008
@@ -37,6 +37,7 @@
 import org.apache.mahout.matrix.DenseVector;
 import org.apache.mahout.matrix.Vector;
 import org.apache.mahout.utils.DistanceMeasure;
+import org.apache.mahout.utils.DummyOutputCollector;
 import org.apache.mahout.utils.EuclideanDistanceMeasure;
 import org.apache.mahout.utils.ManhattanDistanceMeasure;
 

Added: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/utils/DummyOutputCollector.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/utils/DummyOutputCollector.java?rev=652727&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/utils/DummyOutputCollector.java (added)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/utils/DummyOutputCollector.java Thu May  1 18:18:21 2008
@@ -0,0 +1,56 @@
+package org.apache.mahout.utils;
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.OutputCollector;
+
+public class DummyOutputCollector<K extends WritableComparable, V extends Writable> implements OutputCollector<K, V> {
+
+  Map<String, List<V>> data = new TreeMap<String, List<V>>();
+
+  public void collect(K key, V values)
+          throws IOException {
+    List<V> points = data.get(key.toString());
+    if (points == null) {
+      points = new ArrayList<V>();
+      data.put(key.toString(), points);
+    }
+    points.add(values);
+  }
+
+  public Map<String, List<V>> getData() {
+    return data;
+  }
+
+  public List<V> getValue(String key) {
+    return data.get(key);
+  }
+
+  public Set<String> getKeys() {
+    return data.keySet();
+  }
+
+}