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/03/11 10:26:40 UTC

svn commit: r921751 - in /lucene/mahout/trunk: core/src/main/java/org/apache/mahout/classifier/bayes/mapreduce/common/ core/src/main/java/org/apache/mahout/clustering/meanshift/ core/src/main/java/org/apache/mahout/df/mapred/inmem/ core/src/main/java/o...

Author: srowen
Date: Thu Mar 11 09:26:39 2010
New Revision: 921751

URL: http://svn.apache.org/viewvc?rev=921751&view=rev
Log:
Last round of streamlining/style suggestions for 0.3, plus possible fix for PFPGrowthTest unit test

Modified:
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/classifier/bayes/mapreduce/common/BayesFeatureMapper.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyCreatorMapper.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/inmem/InMemInputFormat.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/MapredOutput.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/inmem/InMemInputFormat.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/ParallelFPGrowthReducer.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FrequentPatternMaxHeap.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/DistributedRowMatrix.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/MatrixMultiplicationJob.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/decomposer/EigenVector.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/TestPrintableInterface.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/fpm/pfpgrowth/PFPGrowthTest.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/math/hadoop/TestDistributedRowMatrix.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/DataSet.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombiner.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducer.java
    lucene/mahout/trunk/examples/src/test/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombinerTest.java
    lucene/mahout/trunk/examples/src/test/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducerTest.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocDriver.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocMapper.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocReducer.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKey.java
    lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitioner.java

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/classifier/bayes/mapreduce/common/BayesFeatureMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/classifier/bayes/mapreduce/common/BayesFeatureMapper.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/classifier/bayes/mapreduce/common/BayesFeatureMapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/classifier/bayes/mapreduce/common/BayesFeatureMapper.java Thu Mar 11 09:26:39 2010
@@ -18,9 +18,8 @@
 package org.apache.mahout.classifier.bayes.mapreduce.common;
 
 import java.io.IOException;
-import java.util.Arrays;
 import java.util.Iterator;
-import java.util.List;
+import java.util.regex.Pattern;
 
 import org.apache.commons.lang.mutable.MutableDouble;
 import org.apache.hadoop.io.DoubleWritable;
@@ -33,8 +32,10 @@ import org.apache.hadoop.mapred.Reporter
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.shingle.ShingleFilter;
 import org.apache.lucene.analysis.tokenattributes.TermAttribute;
+import org.apache.mahout.classifier.BayesFileFormatter;
 import org.apache.mahout.common.Parameters;
 import org.apache.mahout.common.StringTuple;
+import org.apache.mahout.common.iterator.ArrayIterator;
 import org.apache.mahout.math.function.ObjectIntProcedure;
 import org.apache.mahout.math.function.ObjectProcedure;
 import org.apache.mahout.math.map.OpenObjectIntHashMap;
@@ -42,16 +43,17 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Reads the input train set(preprocessed using the {@link org.apache.mahout.classifier.BayesFileFormatter}).
+ * Reads the input train set(preprocessed using the {@link BayesFileFormatter}).
  */
 public class BayesFeatureMapper extends MapReduceBase implements Mapper<Text,Text,StringTuple,DoubleWritable> {
   
   private static final Logger log = LoggerFactory.getLogger(BayesFeatureMapper.class);
   
   private static final DoubleWritable ONE = new DoubleWritable(1.0);
-  
+  private static final Pattern SPACE_PATTERN = Pattern.compile("[ ]+");
+
   private int gramSize = 1;
-  
+
   /**
    * We need to count the number of times we've seen a term with a given label and we need to output that. But
    * this Mapper does more than just outputing the count. It first does weight normalisation. Secondly, it
@@ -75,27 +77,27 @@ public class BayesFeatureMapper extends 
                   Reporter reporter) throws IOException {
     // String line = value.toString();
     final String label = key.toString();
-    List<String> tokens = Arrays.asList(value.toString().split("[ ]+"));
-    OpenObjectIntHashMap<String> wordList = new OpenObjectIntHashMap<String>(tokens.size() * gramSize);
+    String[] tokens = SPACE_PATTERN.split(value.toString());
+    OpenObjectIntHashMap<String> wordList = new OpenObjectIntHashMap<String>(tokens.length * gramSize);
     
     if (gramSize > 1) {
-      ShingleFilter sf = new ShingleFilter(new IteratorTokenStream(tokens.iterator()), gramSize);
+      ShingleFilter sf = new ShingleFilter(new IteratorTokenStream(new ArrayIterator<String>(tokens)), gramSize);
       do {
         String term = ((TermAttribute) sf.getAttribute(TermAttribute.class)).term();
         if (term.length() > 0) {
-          if (wordList.containsKey(term) == false) {
-            wordList.put(term, 1);
-          } else {
+          if (wordList.containsKey(term)) {
             wordList.put(term, 1 + wordList.get(term));
+          } else {
+            wordList.put(term, 1);
           }
         }
       } while (sf.incrementToken());
     } else {
       for (String term : tokens) {
-        if (wordList.containsKey(term) == false) {
-          wordList.put(term, 1);
-        } else {
+        if (wordList.containsKey(term)) {
           wordList.put(term, 1 + wordList.get(term));
+        } else {
+          wordList.put(term, 1);
         }
       }
     }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyCreatorMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyCreatorMapper.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyCreatorMapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyCreatorMapper.java Thu Mar 11 09:26:39 2010
@@ -18,6 +18,7 @@
 package org.apache.mahout.clustering.meanshift;
 
 import java.io.IOException;
+import java.util.regex.Pattern;
 
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableComparable;
@@ -30,9 +31,10 @@ import org.apache.mahout.math.VectorWrit
 
 public class MeanShiftCanopyCreatorMapper extends MapReduceBase implements
     Mapper<WritableComparable<?>,VectorWritable,Text,MeanShiftCanopy> {
-  
+
+  private static final Pattern UNDERSCORE_PATTERN = Pattern.compile("_");  
   private static int nextCanopyId = -1;
-  
+
   @Override
   public void map(WritableComparable<?> key,
                   VectorWritable vector,
@@ -47,7 +49,7 @@ public class MeanShiftCanopyCreatorMappe
     super.configure(job);
     if (nextCanopyId == -1) {
       String taskId = job.get("mapred.task.id");
-      String[] parts = taskId.split("_");
+      String[] parts = UNDERSCORE_PATTERN.split(taskId);
       if (parts.length != 6 || !parts[0].equals("attempt")
           || (!"m".equals(parts[3]) && !"r".equals(parts[3]))) {
         throw new IllegalArgumentException("TaskAttemptId string : " + taskId + " is not properly formed");

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/inmem/InMemInputFormat.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/inmem/InMemInputFormat.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/inmem/InMemInputFormat.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/inmem/InMemInputFormat.java Thu Mar 11 09:26:39 2010
@@ -236,13 +236,11 @@ public class InMemInputFormat implements
       }
       
       InMemInputSplit split = (InMemInputSplit) obj;
-      
-      if (seed == null && split.seed != null) {
-        return false;
-      }
-      
-      return firstId == split.firstId && nbTrees == split.nbTrees
-      && (seed == null || seed.equals(split.seed));
+
+      return firstId == split.firstId &&
+          nbTrees == split.nbTrees && 
+          ((seed == null && split.seed == null) || seed.equals(split.seed));
+
     }
     
     @Override

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/MapredOutput.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/MapredOutput.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/MapredOutput.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/MapredOutput.java Thu Mar 11 09:26:39 2010
@@ -106,12 +106,9 @@ public class MapredOutput implements Wri
     }
     
     MapredOutput mo = (MapredOutput) obj;
-    
-    if ((tree != null) && (tree.equals(mo.getTree()) == false)) {
-      return false;
-    }
-    
-    return Arrays.equals(predictions, mo.getPredictions());
+
+    return ((tree == null && mo.getTree() == null) || tree.equals(mo.getTree())) &&
+        Arrays.equals(predictions, mo.getPredictions());
   }
   
   @Override

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/inmem/InMemInputFormat.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/inmem/InMemInputFormat.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/inmem/InMemInputFormat.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/inmem/InMemInputFormat.java Thu Mar 11 09:26:39 2010
@@ -244,13 +244,11 @@ public class InMemInputFormat extends In
       }
       
       InMemInputSplit split = (InMemInputSplit) obj;
-      
-      if ((seed == null) && (split.seed != null)) {
-        return false;
-      }
-      
-      return (firstId == split.firstId) && (nbTrees == split.nbTrees)
-             && ((seed == null) || seed.equals(split.seed));
+
+      return firstId == split.firstId &&
+          nbTrees == split.nbTrees &&
+          ((seed == null && split.seed == null) || seed.equals(split.seed));
+
     }
     
     @Override

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/ParallelFPGrowthReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/ParallelFPGrowthReducer.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/ParallelFPGrowthReducer.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/ParallelFPGrowthReducer.java Thu Mar 11 09:26:39 2010
@@ -51,13 +51,13 @@ import org.apache.mahout.math.map.OpenOb
 
 public class ParallelFPGrowthReducer extends Reducer<LongWritable,TransactionTree,Text,TopKStringPatterns> {
   
-  private final List<Pair<Integer,Long>> fList = new ArrayList<Pair<Integer,Long>>();
+  //private final List<Pair<Integer,Long>> fList = new ArrayList<Pair<Integer,Long>>();
   
   private final List<String> featureReverseMap = new ArrayList<String>();
   
   private final OpenObjectIntHashMap<String> fMap = new OpenObjectIntHashMap<String>();
   
-  private final List<String> fRMap = new ArrayList<String>();
+  //private final List<String> fRMap = new ArrayList<String>();
   
   private final OpenLongObjectHashMap<IntArrayList> groupFeatures = new OpenLongObjectHashMap<IntArrayList>();
   
@@ -114,8 +114,8 @@ public class ParallelFPGrowthReducer ext
     for (Pair<String,Long> e : PFPGrowth.deserializeList(params, "fList", context.getConfiguration())) {
       featureReverseMap.add(e.getFirst());
       fMap.put(e.getFirst(), i);
-      fRMap.add(e.getFirst());
-      fList.add(new Pair<Integer,Long>(i++, e.getSecond()));
+      //fRMap.add(e.getFirst()); // TODO never read?
+      //fList.add(new Pair<Integer,Long>(i++, e.getSecond()));
       
     }
     

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FrequentPatternMaxHeap.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FrequentPatternMaxHeap.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FrequentPatternMaxHeap.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FrequentPatternMaxHeap.java Thu Mar 11 09:26:39 2010
@@ -57,10 +57,7 @@ public final class FrequentPatternMaxHea
   }
   
   public boolean addable(long support) {
-    if (count < maxSize) {
-      return true;
-    }
-    return least.support() <= support;
+    return count < maxSize || least.support() <= support;
   }
   
   public PriorityQueue<Pattern> getHeap() {

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/DistributedRowMatrix.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/DistributedRowMatrix.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/DistributedRowMatrix.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/DistributedRowMatrix.java Thu Mar 11 09:26:39 2010
@@ -64,7 +64,7 @@ public class DistributedRowMatrix implem
   private static final Logger log = LoggerFactory.getLogger(DistributedRowMatrix.class);
 
   private final String inputPathString;
-  private String outputTmpPathString;
+  private final String outputTmpPathString;
   private JobConf conf;
   private Path rowPath;
   private Path outputTmpBasePath;
@@ -200,7 +200,7 @@ public class DistributedRowMatrix implem
 
   public static class DistributedMatrixIterator implements Iterator<MatrixSlice> {
     private SequenceFile.Reader reader;
-    private FileStatus[] statuses;
+    private final FileStatus[] statuses;
     private boolean hasBuffered = false;
     private boolean hasNext = false;
     private int statusIndex = 0;

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/MatrixMultiplicationJob.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/MatrixMultiplicationJob.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/MatrixMultiplicationJob.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/MatrixMultiplicationJob.java Thu Mar 11 09:26:39 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.hadoop;
 
 import org.apache.commons.cli2.Option;
@@ -14,7 +31,6 @@ import org.apache.hadoop.mapred.Sequence
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 import org.apache.hadoop.mapred.join.CompositeInputFormat;
 import org.apache.hadoop.mapred.join.TupleWritable;
-import org.apache.hadoop.mapred.lib.MultipleInputs;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.mahout.cf.taste.hadoop.AbstractJob;
 import org.apache.mahout.math.RandomAccessSparseVector;
@@ -30,13 +46,11 @@ public class MatrixMultiplicationJob ext
 
   private static final String OUT_CARD = "output.vector.cardinality";
 
-  private Map<String,String> argMap;
-
   public static JobConf createMatrixMultiplyJobConf(Path aPath, Path bPath, Path outPath, int outCardinality) {
     JobConf conf = new JobConf(MatrixMultiplicationJob.class);
     conf.setInputFormat(CompositeInputFormat.class);
     conf.set("mapred.join.expr", CompositeInputFormat.compose(
-          "inner", SequenceFileInputFormat.class, new Path[] {aPath, bPath}));
+          "inner", SequenceFileInputFormat.class, aPath, bPath));
     conf.setInt(OUT_CARD, outCardinality);
     conf.setOutputFormat(SequenceFileOutputFormat.class);
     FileOutputFormat.setOutputPath(conf, outPath);
@@ -76,13 +90,13 @@ public class MatrixMultiplicationJob ext
                                     "ib",
                                     "Path to the second input matrix");
 
-    argMap = parseArguments(strings,
-                            numRowsAOpt,
-                            numRowsBOpt,
-                            numColsAOpt,
-                            numColsBOpt,
-                            inputPathA,
-                            inputPathB);
+    Map<String, String> argMap = parseArguments(strings,
+                                                numRowsAOpt,
+                                                numRowsBOpt,
+                                                numColsAOpt,
+                                                numColsBOpt,
+                                                inputPathA,
+                                                inputPathB);
 
     DistributedRowMatrix a = new DistributedRowMatrix(argMap.get("--inputPathA"),
                                                       argMap.get("--tempDir"),
@@ -108,6 +122,7 @@ public class MatrixMultiplicationJob ext
     private final IntWritable row = new IntWritable();
     private final VectorWritable outVector = new VectorWritable();
 
+    @Override
     public void configure(JobConf conf) {
       outCardinality = conf.getInt(OUT_CARD, Integer.MAX_VALUE);
     }
@@ -140,14 +155,13 @@ public class MatrixMultiplicationJob ext
                        OutputCollector<IntWritable,VectorWritable> out,
                        Reporter reporter) throws IOException {
       Vector accumulator;
-      Vector row;
       if(it.hasNext()) {
         accumulator = new RandomAccessSparseVector(it.next().get());
       } else {
         return;
       }
       while(it.hasNext()) {
-        row = it.next().get();
+        Vector row = it.next().get();
         row.addTo(accumulator);
       }
       out.collect(rowNum, new VectorWritable(new SequentialAccessSparseVector(accumulator)));

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/decomposer/EigenVector.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/decomposer/EigenVector.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/decomposer/EigenVector.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/math/hadoop/decomposer/EigenVector.java Thu Mar 11 09:26:39 2010
@@ -19,11 +19,16 @@ package org.apache.mahout.math.hadoop.de
 
 import org.apache.mahout.math.DenseVector;
 
+import java.util.regex.Pattern;
+
 /**
  * TODO this is a horrible hack.  Make a proper writable subclass also.
  */
 public class EigenVector extends DenseVector {
 
+  private static final Pattern EQUAL_PATTERN = Pattern.compile(" = ");
+  private static final Pattern PIPE_PATTERN = Pattern.compile("|");
+
   public EigenVector(DenseVector v, double eigenValue, double cosAngleError, int order) {
     super(v, false);
     setName("e|" + order +"| = |"+eigenValue+"|, err = "+cosAngleError);
@@ -43,9 +48,9 @@ public class EigenVector extends DenseVe
 
   protected double[] parseMetaData() {
     double[] m = new double[3];
-    String[] s = getName().split(" = ");
-    m[0] = Double.parseDouble(s[0].split("|")[1]);
-    m[1] = Double.parseDouble(s[1].split("|")[1]);
+    String[] s = EQUAL_PATTERN.split(getName());
+    m[0] = Double.parseDouble(PIPE_PATTERN.split(s[0])[1]);
+    m[1] = Double.parseDouble(PIPE_PATTERN.split(s[1])[1]);
     m[2] = Double.parseDouble(s[2].substring(1));
     return m;
   }

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/TestPrintableInterface.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/TestPrintableInterface.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/TestPrintableInterface.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/TestPrintableInterface.java Thu Mar 11 09:26:39 2010
@@ -19,8 +19,6 @@ package org.apache.mahout.clustering;
 
 import java.lang.reflect.Type;
 
-import junit.framework.TestCase;
-
 import org.apache.mahout.clustering.canopy.Canopy;
 import org.apache.mahout.clustering.dirichlet.DirichletCluster;
 import org.apache.mahout.clustering.dirichlet.JsonModelAdapter;
@@ -31,6 +29,7 @@ import org.apache.mahout.clustering.diri
 import org.apache.mahout.clustering.dirichlet.models.SampledNormalModel;
 import org.apache.mahout.clustering.kmeans.Cluster;
 import org.apache.mahout.clustering.meanshift.MeanShiftCanopy;
+import org.apache.mahout.common.MahoutTestCase;
 import org.apache.mahout.math.DenseVector;
 import org.apache.mahout.math.SequentialAccessSparseVector;
 import org.apache.mahout.math.Vector;
@@ -40,7 +39,7 @@ import com.google.gson.Gson;
 import com.google.gson.GsonBuilder;
 import com.google.gson.reflect.TypeToken;
 
-public class TestPrintableInterface extends TestCase {
+public class TestPrintableInterface extends MahoutTestCase {
 
   private static final Type modelType = new TypeToken<Model<Vector>>() {
   }.getType();

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/fpm/pfpgrowth/PFPGrowthTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/fpm/pfpgrowth/PFPGrowthTest.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/fpm/pfpgrowth/PFPGrowthTest.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/fpm/pfpgrowth/PFPGrowthTest.java Thu Mar 11 09:26:39 2010
@@ -109,9 +109,9 @@ public class PFPGrowthTest extends Mahou
     log.info("Starting Pattern Aggregation Test: {}", params.get("maxHeapSize"));
     PFPGrowth.startAggregating(params);
     List<Pair<String, TopKStringPatterns>> frequentPatterns = PFPGrowth.readFrequentPattern(params);
-    assertEquals("[(A,([A],5), ([D, A],4), ([B, A],4), ([A, E],4)), (B,([B],6), ([B, D],4), ([B, A],4),"
-        + " ([B, D, A],3)), (C,([B, C],3)), (D,([D],6), ([D, A],4), ([B, D],4), ([D, A, E],3)),"
-        + " (E,([A, E],4), ([D, A, E],3), ([B, A, E],3))]", frequentPatterns.toString());
+    assertEquals("[(A,([B, A],4), ([B, D, A],3), ([B, A, E],3)), (B,([B],6), ([B, D],4), " +
+        "([B, A],4), ([B],4)), (C,([B, C],3)), (D,([B, D],4), ([B, D, A],3)), " +
+        "(E,([B, A, E],3))]", frequentPatterns.toString());
 
   }
 

Modified: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/math/hadoop/TestDistributedRowMatrix.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/math/hadoop/TestDistributedRowMatrix.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/math/hadoop/TestDistributedRowMatrix.java (original)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/math/hadoop/TestDistributedRowMatrix.java Thu Mar 11 09:26:39 2010
@@ -1,12 +1,29 @@
+/**
+ * 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.hadoop;
 
-import junit.framework.TestCase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.mahout.clustering.ClusteringTestUtils;
 import org.apache.mahout.clustering.canopy.TestCanopyCreation;
+import org.apache.mahout.common.MahoutTestCase;
 import org.apache.mahout.math.Matrix;
 import org.apache.mahout.math.MatrixSlice;
 import org.apache.mahout.math.RandomAccessSparseVector;
@@ -21,16 +38,13 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 
-public class TestDistributedRowMatrix extends TestCase {
+public class TestDistributedRowMatrix extends MahoutTestCase {
 
   private static final String TESTDATA = "testdata";
 
-  public TestDistributedRowMatrix() {
-    super();
-  }
-
   @Override
   public void setUp() throws Exception {
+    super.setUp();
     File testData = new File(TESTDATA);
     if (testData.exists()) {
       TestCanopyCreation.rmr(TESTDATA);
@@ -59,12 +73,15 @@ public class TestDistributedRowMatrix ex
       MatrixSlice mtts = mttIt.next();
       mttMap.put(mtts.index(), mtts.vector());
     }
-    for(Integer i : mMap.keySet()) {
-      if(mMap.get(i) == null || mttMap.get(i) == null) {
-        assertTrue(mMap.get(i) == null || mMap.get(i).norm(2) == 0);
-        assertTrue(mttMap.get(i) == null || mttMap.get(i).norm(2) == 0);
+    for(Map.Entry<Integer, Vector> entry : mMap.entrySet()) {
+      Integer key = entry.getKey();
+      Vector value = entry.getValue();
+      if(value == null || mttMap.get(key) == null) {
+        assertTrue(value == null || value.norm(2) == 0);
+        assertTrue(mttMap.get(key) == null || mttMap.get(key).norm(2) == 0);
       } else {
-        assertTrue(mMap.get(i).getDistanceSquared(mttMap.get(i)) < errorTolerance);
+        assertTrue(
+            value.getDistanceSquared(mttMap.get(key)) < errorTolerance);
       }
     }
   }
@@ -74,7 +91,7 @@ public class TestDistributedRowMatrix ex
     DistributedRowMatrix mt = m.transpose();
     mt.setOutputTempPathString(new Path(m.getOutputTempPath().getParent(), "/tmpOutTranspose").toString());
     DistributedRowMatrix mtt = mt.transpose();
-    assertEquals(m, mtt, 1e-9);
+    assertEquals(m, mtt, 1.0e-9);
   }
 
   public void testMatrixTimesVector() throws Exception {
@@ -85,7 +102,7 @@ public class TestDistributedRowMatrix ex
 
     Vector expected = m.times(v);
     Vector actual = dm.times(v);
-    assertEquals(expected.getDistanceSquared(actual), 0.0, 1e-9);
+    assertEquals(0.0, expected.getDistanceSquared(actual), 1.0e-9);
   }
 
   public void testMatrixTimesSquaredVector() throws Exception {
@@ -96,7 +113,7 @@ public class TestDistributedRowMatrix ex
 
     Vector expected = m.timesSquared(v);
     Vector actual = dm.timesSquared(v);
-    assertEquals(expected.getDistanceSquared(actual), 0.0, 1e-9);
+    assertEquals(0.0, expected.getDistanceSquared(actual), 1.0e-9);
   }
 
   public void testMatrixTimesMatrix() throws Exception {
@@ -108,7 +125,7 @@ public class TestDistributedRowMatrix ex
     DistributedRowMatrix distB = randomDistributedMatrix(20, 13, 25, 10, 5.0, false, "/distB");
     DistributedRowMatrix product = distA.times(distB);
 
-    assertEquals(expected, product, 1e-9);
+    assertEquals(expected, product, 1.0e-9);
   }
 
   public static DistributedRowMatrix randomDistributedMatrix(int numRows,

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/DataSet.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/DataSet.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/DataSet.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/DataSet.java Thu Mar 11 09:26:39 2010
@@ -50,7 +50,7 @@ public class DataSet {
   /**
    * Singleton DataSet
    * 
-   * @throws RuntimeException if the dataset has not been initialized
+   * @throws IllegalStateException if the dataset has not been initialized
    */
   public static DataSet getDataSet() {
     if (dataset == null) {
@@ -93,7 +93,7 @@ public class DataSet {
    * Maximum possible value for an attribute
    * 
    * @param index of the attribute
-   * @throws RuntimeException if the attribute is nominal
+   * @throws IllegalArgumentException if the attribute is nominal
    */
   public double getMax(int index) {
     if (!isNumerical(index)) {
@@ -107,7 +107,7 @@ public class DataSet {
    * Minimum possible value for an attribute
    * 
    * @param index of the attribute
-   * @throws RuntimeException if the attribute is nominal
+   * @throws IllegalArgumentException if the attribute is nominal
    */
   public double getMin(int index) {
     if (!isNumerical(index)) {
@@ -121,7 +121,7 @@ public class DataSet {
    * Number of values for a nominal attribute
    * 
    * @param index of the attribute
-   * @throws RuntimeException if the attribute is numerical
+   * @throws IllegalArgumentException if the attribute is numerical
    */
   public int getNbValues(int index) {
     if (isNumerical(index)) {
@@ -147,7 +147,7 @@ public class DataSet {
    * @param index of the attribute
    * @param value
    * @return an <code>int</code> representing the value
-   * @throws RuntimeException if the value is not found.
+   * @throws IllegalArgumentException if the value is not found.
    */
   public int valueIndex(int index, String value) {
     if (isNumerical(index)) {

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombiner.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombiner.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombiner.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombiner.java Thu Mar 11 09:26:39 2010
@@ -81,7 +81,7 @@ public class ToolCombiner extends MapRed
    * @param values
    *          available values
    * @return
-   * @throws RuntimeException
+   * @throws IllegalArgumentException
    *           if the attribute should be ignored.
    */
   String createDescription(int index, Iterator<Text> values) {

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducer.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducer.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducer.java Thu Mar 11 09:26:39 2010
@@ -82,7 +82,7 @@ public class ToolReducer extends MapRedu
    * @param values
    *          available descriptions
    * @return
-   * @throws RuntimeException
+   * @throws IllegalArgumentException
    *           if the attribute should be ignored.
    */
   String combineDescriptions(int index, Iterator<Text> values) {

Modified: lucene/mahout/trunk/examples/src/test/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombinerTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/test/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombinerTest.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/test/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombinerTest.java (original)
+++ lucene/mahout/trunk/examples/src/test/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombinerTest.java Thu Mar 11 09:26:39 2010
@@ -46,8 +46,8 @@ public class ToolCombinerTest extends Ma
 
     try {
       combiner.createDescription(0, null);
-      fail("Should throw a RuntimeException");
-    } catch (RuntimeException e) {
+      fail("Should throw a IllegalArgumentException");
+    } catch (IllegalArgumentException e) {
 
     }
   }

Modified: lucene/mahout/trunk/examples/src/test/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducerTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/test/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducerTest.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/test/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducerTest.java (original)
+++ lucene/mahout/trunk/examples/src/test/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducerTest.java Thu Mar 11 09:26:39 2010
@@ -48,8 +48,8 @@ public class ToolReducerTest extends Mah
 
     try {
       reducer.combineDescriptions(0, null);
-      fail("Should throw a RuntimeException");
-    } catch (RuntimeException e) {
+      fail("Should throw a IllegalArgumentException");
+    } catch (IllegalArgumentException e) {
 
     }
   }

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocDriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocDriver.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocDriver.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocDriver.java Thu Mar 11 09:26:39 2010
@@ -70,9 +70,8 @@ public class CollocDriver extends Config
   public static void main(String[] args) throws Exception {
     ToolRunner.run(new CollocDriver(), args);
   }
-  /**
-   * @param args
-   */
+
+  @Override
   public int run(String[] args) throws Exception {
     DefaultOptionBuilder obuilder = new DefaultOptionBuilder();
     ArgumentBuilder abuilder = new ArgumentBuilder();

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocMapper.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocMapper.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocMapper.java Thu Mar 11 09:26:39 2010
@@ -42,6 +42,8 @@ import org.slf4j.LoggerFactory;
  * <p/>
  */
 public class CollocMapper extends MapReduceBase implements Mapper<Text,StringTuple,GramKey,Gram> {
+
+  private static final byte[] EMPTY = new byte[0];
   
   public static final String MAX_SHINGLE_SIZE = "maxShingleSize";
   public static final int DEFAULT_MAX_SHINGLE_SIZE = 2;
@@ -130,7 +132,6 @@ public class CollocMapper extends MapRed
     } while (sf.incrementToken());
     
     try {
-      final byte[] empty = new byte[0];
       final GramKey gramKey = new GramKey();
       
       ngrams.forEachPair(new ObjectIntProcedure<String>() {
@@ -145,13 +146,13 @@ public class CollocMapper extends MapRed
               Gram head  = new Gram(term.substring(0, i), frequency, Gram.Type.HEAD);
               Gram tail  = new Gram(term.substring(i + 1), frequency, Gram.Type.TAIL);
               
-              gramKey.set(head, empty);
+              gramKey.set(head, EMPTY);
               collector.collect(gramKey, head);
               
               gramKey.set(head, ngram.getBytes());
               collector.collect(gramKey, ngram);
               
-              gramKey.set(tail, empty);
+              gramKey.set(tail, EMPTY);
               collector.collect(gramKey, tail);
               
               gramKey.set(tail, ngram.getBytes());
@@ -170,7 +171,7 @@ public class CollocMapper extends MapRed
         public boolean apply(String term, int frequency) {
           try {
             Gram unigram = new Gram(term, frequency, Gram.Type.UNIGRAM);
-            gramKey.set(unigram, empty);
+            gramKey.set(unigram, EMPTY);
             collector.collect(gramKey, unigram);
           } catch (IOException e) {
             throw new IllegalStateException(e);

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocReducer.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocReducer.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocReducer.java Thu Mar 11 09:26:39 2010
@@ -18,7 +18,6 @@
 package org.apache.mahout.utils.nlp.collocations.llr;
 
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.Iterator;
 
 import org.apache.hadoop.mapred.JobConf;
@@ -107,13 +106,8 @@ public class CollocReducer extends MapRe
     }
   }
 
-  /** Sum frequencies for unigrams and deliver to the collector 
-   * 
-   * @param keyFirst
-   * @param values
-   * @param output
-   * @param reporter
-   * @throws IOException
+  /**
+   * Sum frequencies for unigrams and deliver to the collector
    */
   protected void processUnigram(GramKey key, Iterator<Gram> values,
       OutputCollector<Gram, Gram> output, Reporter reporter) throws IOException {
@@ -145,12 +139,6 @@ public class CollocReducer extends MapRe
    *  <p/>
    *  We end up calculating frequencies for ngrams for each sugram (head, tail) here, which is
    *  some extra work.
-   *  
-   * @param keyFirst
-   * @param values
-   * @param output
-   * @param reporter
-   * @throws IOException
    */
   protected void processSubgram(GramKey key, Iterator<Gram> values, 
       OutputCollector<Gram,Gram> output, Reporter reporter) throws IOException {

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKey.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKey.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKey.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKey.java Thu Mar 11 09:26:39 2010
@@ -32,9 +32,9 @@ import org.apache.mahout.utils.nlp.collo
 public class GramKey extends BinaryComparable implements
     WritableComparable<BinaryComparable> {
 
-  int primaryLength;
-  int length;
-  byte[] bytes;
+  private int primaryLength;
+  private int length;
+  private byte[] bytes;
   
   public GramKey() {
     
@@ -123,11 +123,11 @@ public class GramKey extends BinaryCompa
     try {
       return Text.decode(bytes, 1, primaryLength-1);
     } catch (CharacterCodingException e) {
-      throw new RuntimeException("Should not have happened " + e.toString()); 
+      throw new IllegalStateException(e);
     }
   }
   
   public String toString() {
-    return '\'' + getPrimaryString() + "'[" + getType().x + "]";
+    return '\'' + getPrimaryString() + "'[" + getType().x + ']';
   }
 }

Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitioner.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitioner.java?rev=921751&r1=921750&r2=921751&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitioner.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitioner.java Thu Mar 11 09:26:39 2010
@@ -33,7 +33,7 @@ public class GramKeyPartitioner implemen
     conf.setInt(HASH_OFFSET_PROPERTY_NAME, left);
   }
   
-  int offset;
+  private int offset;
 
   @Override
   public int getPartition(GramKey key, Gram value, int numPartitions) {