You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by ro...@apache.org on 2010/02/07 13:08:27 UTC

svn commit: r907417 - in /lucene/mahout/trunk/core/src: main/java/org/apache/mahout/clustering/meanshift/ test/java/org/apache/mahout/clustering/fuzzykmeans/ test/java/org/apache/mahout/clustering/kmeans/ test/java/org/apache/mahout/clustering/meanshif...

Author: robinanil
Date: Sun Feb  7 12:08:26 2010
New Revision: 907417

URL: http://svn.apache.org/viewvc?rev=907417&view=rev
Log:
Dummy Reporter to aid Map/Reduce unit testing

Added:
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/common/DummyReporter.java
Modified:
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyReducer.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/meanshift/TestMeanShift.java

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyReducer.java?rev=907417&r1=907416&r2=907417&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyReducer.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyReducer.java Sun Feb  7 12:08:26 2010
@@ -51,7 +51,11 @@
     }
 
     for (MeanShiftCanopy canopy : canopies) {
-      allConverged = clusterer.shiftToMean(canopy) && allConverged;
+      boolean converged = clusterer.shiftToMean(canopy);
+      if (converged) {
+        reporter.incrCounter("Clustering", "Converged Clusters", 1);
+      }
+      allConverged = converged && allConverged;
       output.collect(new Text(canopy.getIdentifier()), canopy);
     }
 

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=907417&r1=907416&r2=907417&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 Sun Feb  7 12:08:26 2010
@@ -25,6 +25,7 @@
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.mahout.clustering.ClusteringTestUtils;
 import org.apache.mahout.clustering.kmeans.TestKmeansClustering;
+import org.apache.mahout.common.DummyReporter;
 import org.apache.mahout.common.MahoutTestCase;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.common.distance.DistanceMeasure;
@@ -457,7 +458,7 @@
       for (String key : combinerCollector.getKeys()) {
         List<FuzzyKMeansInfo> values = combinerCollector.getValue(key);
         reducer
-            .reduce(new Text(key), values.iterator(), reducerCollector, null);
+            .reduce(new Text(key), values.iterator(), reducerCollector, new DummyReporter());
       }
 
       // now verify the reducer output

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=907417&r1=907416&r2=907417&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 Sun Feb  7 12:08:26 2010
@@ -17,6 +17,12 @@
 
 package org.apache.mahout.clustering.kmeans;
 
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -25,25 +31,19 @@
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.mahout.clustering.ClusteringTestUtils;
 import org.apache.mahout.clustering.canopy.CanopyDriver;
+import org.apache.mahout.common.DummyOutputCollector;
+import org.apache.mahout.common.DummyReporter;
+import org.apache.mahout.common.HadoopUtil;
 import org.apache.mahout.common.MahoutTestCase;
 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;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
 public class TestKmeansClustering extends MahoutTestCase {
 
   public static final double[][] reference = {{1, 1}, {2, 1}, {1, 2},
@@ -325,7 +325,7 @@
       DummyOutputCollector<Text, Cluster> collector3 = new DummyOutputCollector<Text, Cluster>();
       for (String key : collector2.getKeys()) {
         reducer.reduce(new Text(key), collector2.getValue(key).iterator(),
-            collector3, null);
+            collector3, new DummyReporter());
       }
 
       assertEquals("Number of map results", k + 1, collector3.getData().size());

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=907417&r1=907416&r2=907417&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 Sun Feb  7 12:08:26 2010
@@ -24,6 +24,7 @@
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.mahout.common.DummyReporter;
 import org.apache.mahout.common.MahoutTestCase;
 import org.apache.mahout.math.DenseVector;
 import org.apache.mahout.math.Vector;
@@ -276,7 +277,7 @@
     DummyOutputCollector<Text, MeanShiftCanopy> reduceCollector = new DummyOutputCollector<Text, MeanShiftCanopy>();
     reducer.configure(conf);
     reducer.reduce(new Text("canopy"), mapCollector.getValue("canopy")
-        .iterator(), reduceCollector, null);
+        .iterator(), reduceCollector, new DummyReporter());
     reducer.close();
 
     // now verify the output

Added: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/common/DummyReporter.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/common/DummyReporter.java?rev=907417&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/common/DummyReporter.java (added)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/common/DummyReporter.java Sun Feb  7 12:08:26 2010
@@ -0,0 +1,80 @@
+/**
+ * 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.common;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.commons.lang.mutable.MutableLong;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.Counters.Counter;
+
+public class DummyReporter implements Reporter {
+  
+  private String status = "";
+  private Map<Enum<?>,MutableLong> count1 = new HashMap<Enum<?>,MutableLong>();
+  private Map<String,Map<String,MutableLong>> count2 = new HashMap<String,Map<String,MutableLong>>();
+  
+  @Override
+  public Counter getCounter(Enum<?> name) {
+    throw new NotImplementedException();
+  }
+  
+  @Override
+  public Counter getCounter(String group, String name) {
+    throw new NotImplementedException();
+  }
+  
+  @Override
+  public InputSplit getInputSplit() throws UnsupportedOperationException {
+    throw new UnsupportedOperationException();
+  }
+  
+  @Override
+  public void incrCounter(Enum<?> key, long amount) {
+    if (count1.containsKey(key) == false) {
+      count1.put(key, new MutableLong(0));
+    }
+    count1.get(key).add(amount);
+  }
+  
+  @Override
+  public void incrCounter(String group, String counter, long amount) {
+    if (count2.containsKey(group) == false) {
+      count2.put(group, new HashMap<String,MutableLong>());
+    }
+    if (count2.get(group).containsKey(counter) == false) {
+      count2.get(group).put(counter, new MutableLong(0));
+    }
+    count2.get(group).get(counter).add(amount);
+    
+  }
+  
+  @Override
+  public void setStatus(String status) {
+    this.status = status;
+  }
+  
+  @Override
+  public void progress() {
+
+  }
+  
+}