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) {