You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by dr...@apache.org on 2010/03/06 17:38:03 UTC
svn commit: r919798 - in /lucene/mahout/trunk/utils/src:
main/java/org/apache/mahout/utils/nlp/collocations/llr/
test/java/org/apache/mahout/utils/nlp/collocations/llr/
Author: drew
Date: Sat Mar 6 16:38:03 2010
New Revision: 919798
URL: http://svn.apache.org/viewvc?rev=919798&view=rev
Log:
MAHOUT-317: llr collocation improvements
Added:
lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKey.java (with props)
lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparator.java (with props)
lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitioner.java (with props)
lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparatorTest.java (with props)
lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitionerTest.java (with props)
lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyTest.java (with props)
Modified:
lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocCombiner.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/Gram.java
lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/CollocMapperTest.java
lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/CollocReducerTest.java
lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramTest.java
Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocCombiner.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocCombiner.java?rev=919798&r1=919797&r2=919798&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocCombiner.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/CollocCombiner.java Sat Mar 6 16:38:03 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.MapReduceBase;
@@ -26,52 +25,27 @@
import org.apache.hadoop.mapred.Reducer;
import org.apache.hadoop.mapred.Reporter;
-/** Combiner for pass1 of the CollocationDriver */
-public class CollocCombiner extends MapReduceBase implements Reducer<Gram,Gram,Gram,Gram> {
+/** Combiner for pass1 of the CollocationDriver. Combines frequencies for values for the same key */
+public class CollocCombiner extends MapReduceBase implements Reducer<GramKey,Gram,GramKey,Gram> {
- /**
- * collocation finder: pass 1 colloc phase:
- *
- * given input from the mapper, k:h_subgram:1 v:ngram:1 k:t_subgram:1 v:ngram:1
- *
- * count ngrams and subgrams.
- *
- * output is:
- *
- * k:h_subgram:subgramfreq v:ngram:ngramfreq k:t_subgram:subgramfreq v:ngram:ngramfreq
- *
- * Each ngram's frequency is essentially counted twice, frequency should be the same for the head and tail.
- * Fix this to count only for the head and move the count into the value?
- */
@Override
- public void reduce(Gram subgramKey,
- Iterator<Gram> ngramValues,
- OutputCollector<Gram,Gram> output,
+ public void reduce(GramKey key,
+ Iterator<Gram> values,
+ OutputCollector<GramKey,Gram> output,
Reporter reporter) throws IOException {
+
+ int freq = 0;
+ Gram value = null;
- HashMap<Gram,Gram> ngramSet = new HashMap<Gram,Gram>();
- int subgramFrequency = 0;
-
- while (ngramValues.hasNext()) {
- Gram ngram = ngramValues.next();
- subgramFrequency += ngram.getFrequency();
-
- Gram ngramCanon = ngramSet.get(ngram);
- if (ngramCanon == null) {
- // t is potentially reused, so create a new object to populate the HashMap
- Gram ngramEntry = new Gram(ngram);
- ngramSet.put(ngramEntry, ngramEntry);
- } else {
- ngramCanon.incrementFrequency(ngram.getFrequency());
- }
+ // accumulate frequencies from values.
+ while (values.hasNext()) {
+ value = values.next();
+ freq += value.getFrequency();
}
+
+ value.setFrequency(freq);
- // emit subgram:subgramFreq ngram:ngramFreq pairs
- subgramKey.setFrequency(subgramFrequency);
-
- for (Gram ngram : ngramSet.keySet()) {
- output.collect(subgramKey, ngram);
- }
+ output.collect(key, value);
}
}
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=919798&r1=919797&r2=919798&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 Sat Mar 6 16:38:03 2010
@@ -27,6 +27,7 @@
import org.apache.commons.cli2.builder.DefaultOptionBuilder;
import org.apache.commons.cli2.builder.GroupBuilder;
import org.apache.commons.cli2.commandline.Parser;
+import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.DoubleWritable;
import org.apache.hadoop.io.Text;
@@ -38,6 +39,8 @@
import org.apache.hadoop.mapred.SequenceFileInputFormat;
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
import org.apache.hadoop.mapred.lib.IdentityMapper;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.standard.StandardAnalyzer;
import org.apache.mahout.common.CommandLineUtil;
@@ -47,8 +50,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-/** Driver for LLR collocation discovery mapreduce job */
-public class CollocDriver {
+/** Driver for LLR Collocation discovery mapreduce job */
+public class CollocDriver extends Configured implements Tool {
public static final String DEFAULT_OUTPUT_DIRECTORY = "output";
public static final String SUBGRAM_OUTPUT_DIRECTORY = "subgrams";
public static final String NGRAM_OUTPUT_DIRECTORY = "ngrams";
@@ -64,10 +67,13 @@
private CollocDriver() {
}
+ public static void main(String[] args) throws Exception {
+ ToolRunner.run(new CollocDriver(), args);
+ }
/**
* @param args
*/
- public static void main(String[] args) throws Exception {
+ public int run(String[] args) throws Exception {
DefaultOptionBuilder obuilder = new DefaultOptionBuilder();
ArgumentBuilder abuilder = new ArgumentBuilder();
GroupBuilder gbuilder = new GroupBuilder();
@@ -131,7 +137,7 @@
if (cmdLine.hasOption(helpOpt)) {
CommandLineUtil.printHelp(group);
- return;
+ return 1;
}
String input = cmdLine.getValue(inputOpt).toString();
@@ -202,8 +208,10 @@
} catch (OptionException e) {
log.error("Exception", e);
CommandLineUtil.printHelp(group);
+ return 1;
}
+ return 0;
}
/**
@@ -247,9 +255,12 @@
int reduceTasks,
int minSupport) throws IOException {
JobConf conf = new JobConf(CollocDriver.class);
+ conf.setJobName(CollocDriver.class.getSimpleName() + ".generateCollocations:" + input);
- conf.setMapOutputKeyClass(Gram.class);
+ conf.setMapOutputKeyClass(GramKey.class);
conf.setMapOutputValueClass(Gram.class);
+ conf.setPartitionerClass(GramKeyPartitioner.class);
+ conf.setOutputValueGroupingComparator(GramKeyGroupComparator.class);
conf.setOutputKeyClass(Gram.class);
conf.setOutputValueClass(Gram.class);
@@ -284,6 +295,8 @@
float minLLRValue,
int reduceTasks) throws IOException {
JobConf conf = new JobConf(CollocDriver.class);
+ conf.setJobName(CollocDriver.class.getSimpleName() + ".computeNGrams: " + output);
+
conf.setLong(LLRReducer.NGRAM_TOTAL, nGramTotal);
conf.setBoolean(EMIT_UNIGRAMS, emitUnigrams);
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=919798&r1=919797&r2=919798&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 Sat Mar 6 16:38:03 2010
@@ -37,11 +37,11 @@
import org.slf4j.LoggerFactory;
/**
- * Runs pass 1 of the Collocation discovery job on input of SequeceFile<Text,Text>, where the key is a
- * document id and the value is the document contents. . Delegates to NGramCollector to perform tokenization,
- * ngram-creation and output collection.
+ * Pass 1 of the Collocation discovery job which generated ngrams and emits ngrams an their component n-1grams.
+ * Input is a SequeceFile<Text,StringTuple>, where the key is a document id and the value is the tokenized documents.
+ * <p/>
*/
-public class CollocMapper extends MapReduceBase implements Mapper<Text,StringTuple,Gram,Gram> {
+public class CollocMapper extends MapReduceBase implements Mapper<Text,StringTuple,GramKey,Gram> {
public static final String MAX_SHINGLE_SIZE = "maxShingleSize";
public static final int DEFAULT_MAX_SHINGLE_SIZE = 2;
@@ -71,21 +71,32 @@
/**
* Collocation finder: pass 1 map phase.
- *
- * Receives a token stream which gets passed through the ShingleFilter. The ShingleFilter delivers ngrams of
+ * <p/>
+ * Receives a token stream which gets passed through a Lucene ShingleFilter. The ShingleFilter delivers ngrams of
* the appropriate size which are then decomposed into head and tail subgrams which are collected in the
* following manner
*
- * k:h_subgram v:ngram k:t_subgram v:ngram
+ * <pre>
+ * k:head_key, v:head_subgram
+ * k:head_key,ngram_key, v:ngram
+ * k:tail_key, v:tail_subgram
+ * k:tail_key,ngram_key, v:ngram
+ * </pre>
*
- * The 'h_' or 't_' prefix is used to specify whether the subgram in question is the head or tail of the
+ * The 'head' or 'tail' prefix is used to specify whether the subgram in question is the head or tail of the
* ngram. In this implementation the head of the ngram is a (n-1)gram, and the tail is a (1)gram.
- *
- * For example, given 'click and clack' and an ngram length of 3: k:'h_click and' v:'click and clack'
- * k;'t_clack' v:'click and clack'
+ * <p/>
+ * For example, given 'click and clack' and an ngram length of 3:
+ * <pre>
+ * k: head_'click and' v:head_'click and'
+ * k: head_'click and',ngram_'click and clack' v:ngram_'click and clack'
+ * k: tail_'clack', v:tail_'clack'
+ * k: tail_'clack',ngram_'click and clack' v:ngram_'click and clack'
+ * </pre>
*
* Also counts the total number of ngrams encountered and adds it to the counter
* CollocDriver.Count.NGRAM_TOTAL
+ * </p>
*
* @param collector
* The collector to send output to
@@ -98,7 +109,7 @@
*/
@Override
public void map(Text key, StringTuple value,
- final OutputCollector<Gram,Gram> collector, Reporter reporter) throws IOException {
+ final OutputCollector<GramKey,Gram> collector, Reporter reporter) throws IOException {
ShingleFilter sf = new ShingleFilter(new IteratorTokenStream(value.getEntries().iterator()), maxShingleSize);
int count = 0; // ngram count
@@ -119,16 +130,33 @@
} while (sf.incrementToken());
try {
+ final byte[] empty = new byte[0];
+ final GramKey gramKey = new GramKey();
+
ngrams.forEachPair(new ObjectIntProcedure<String>() {
@Override
public boolean apply(String term, int frequency) {
// obtain components, the leading (n-1)gram and the trailing unigram.
int i = term.lastIndexOf(' '); // TODO: fix for non-whitespace delimited languages.
if (i != -1) { // bigram, trigram etc
- Gram ngram = new Gram(term, frequency, Gram.Type.NGRAM);
+
try {
- collector.collect(new Gram(term.substring(0, i), frequency, Gram.Type.HEAD), ngram);
- collector.collect(new Gram(term.substring(i + 1), frequency, Gram.Type.TAIL), ngram);
+ Gram ngram = new Gram(term, frequency, Gram.Type.NGRAM);
+ 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);
+ collector.collect(gramKey, head);
+
+ gramKey.set(head, ngram.getBytes());
+ collector.collect(gramKey, ngram);
+
+ gramKey.set(tail, empty);
+ collector.collect(gramKey, tail);
+
+ gramKey.set(tail, ngram.getBytes());
+ collector.collect(gramKey, ngram);
+
} catch (IOException e) {
throw new IllegalStateException(e);
}
@@ -142,7 +170,8 @@
public boolean apply(String term, int frequency) {
try {
Gram unigram = new Gram(term, frequency, Gram.Type.UNIGRAM);
- collector.collect(unigram, unigram);
+ 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=919798&r1=919797&r2=919798&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 Sat Mar 6 16:38:03 2010
@@ -32,7 +32,7 @@
/**
* Reducer for Pass 1 of the collocation identification job. Generates counts for ngrams and subgrams.
*/
-public class CollocReducer extends MapReduceBase implements Reducer<Gram,Gram,Gram,Gram> {
+public class CollocReducer extends MapReduceBase implements Reducer<GramKey,Gram,Gram,Gram> {
private static final Logger log = LoggerFactory.getLogger(CollocReducer.class);
@@ -40,7 +40,11 @@
public static final int DEFAULT_MIN_SUPPORT = 2;
public enum Skipped {
- LESS_THAN_MIN_SUPPORT
+ LESS_THAN_MIN_SUPPORT,
+ MALFORMED_KEY_TUPLE,
+ MALFORMED_TUPLE,
+ MALFORMED_TYPES,
+ MALFORMED_UNIGRAM
}
private int minSupport;
@@ -60,51 +64,140 @@
/**
* collocation finder: pass 1 reduce phase:
+ * <p/>
+ * given input from the mapper,
*
- * given input from the mapper, k:h_subgram v:ngram k:t_subgram v:ngram
- *
- * count ngrams and subgrams.
- *
+ * <pre>
+ * k:head_subgram,ngram, v:ngram:partial freq
+ * k:head_subgram v:head_subgram:partial freq
+ * k:tail_subgram,ngram, v:ngram:partial freq
+ * k:tail_subgram v:tail_subgram:partial freq
+ * k:unigram v:unigram:partial freq
+ * </pre>
+ * sum gram frequencies and output for llr calculation
+ * <p/>
* output is:
- *
- * k:ngram:ngramfreq v:h_subgram:h_subgramfreq k:ngram:ngramfreq v:t_subgram:t_subgramfreq
- *
- * Each ngram's frequency is essentially counted twice, frequency should be the same for the head and tail.
- * Fix this to count only for the head and move the count into the value?
+ * <pre>
+ * k:ngram:ngramfreq v:head_subgram:head_subgramfreq
+ * k:ngram:ngramfreq v:tail_subgram:tail_subgramfreq
+ * k:unigram:unigramfreq v:unigram:unigramfreq
+ * </pre>
+ * Each ngram's frequency is essentially counted twice, once for head, once for tail.
+ * frequency should be the same for the head and tail. Fix this to count only for the
+ * head and move the count into the value?
*/
@Override
- public void reduce(Gram subgramKey,
- Iterator<Gram> ngramValues,
+ public void reduce(GramKey key,
+ Iterator<Gram> values,
OutputCollector<Gram,Gram> output,
Reporter reporter) throws IOException {
- HashMap<Gram,Gram> ngramSet = new HashMap<Gram,Gram>();
- int subgramFrequency = 0;
+ Gram.Type keyType = key.getType();
+
+ if (keyType == Gram.Type.UNIGRAM) {
+ // sum frequencies for unigrams.
+ processUnigram(key, values, output, reporter);
+ }
+ else if (keyType == Gram.Type.HEAD || keyType == Gram.Type.TAIL) {
+ // sum frequencies for subgrams, ngram and collect for each ngram.
+ processSubgram(key, values, output, reporter);
+ }
+ else {
+ reporter.incrCounter(Skipped.MALFORMED_TYPES, 1);
+ }
+ }
+
+ /** Sum frequencies for unigrams and deliver to the collector
+ *
+ * @param keyFirst
+ * @param values
+ * @param output
+ * @param reporter
+ * @throws IOException
+ */
+ protected void processUnigram(GramKey key, Iterator<Gram> values,
+ OutputCollector<Gram, Gram> output, Reporter reporter) throws IOException {
+
+ int freq = 0;
+ Gram value = null;
- while (ngramValues.hasNext()) {
- Gram ngram = ngramValues.next();
- subgramFrequency += ngram.getFrequency();
+ // accumulate frequencies from values.
+ while (values.hasNext()) {
+ value = values.next();
+ freq += value.getFrequency();
+ }
+
+ if (freq < minSupport) {
+ reporter.incrCounter(Skipped.LESS_THAN_MIN_SUPPORT, 1);
+ return;
+ }
+
+ value.setFrequency(freq);
+ output.collect(value, value);
+
+ }
- Gram ngramCanon = ngramSet.get(ngram);
- if (ngramCanon == null) {
- // t is potentially reused, so create a new object to populate the
- // HashMap
- Gram ngramEntry = new Gram(ngram);
- ngramSet.put(ngramEntry, ngramEntry);
- } else {
- ngramCanon.incrementFrequency(ngram.getFrequency());
+ /** Sum frequencies for subgram, ngrams and deliver ngram, subgram pairs to the collector.
+ * <p/>
+ * Sort order guarantees that the subgram/subgram pairs will be seen first and then
+ * subgram/ngram1 pairs, subgram/ngram2 pairs ... subgram/ngramN pairs, so frequencies for
+ * ngrams can be calcualted here as well.
+ * <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 {
+
+ Gram subgram = null;
+ Gram currentNgram = null;
+
+ while (values.hasNext()) {
+ Gram value = values.next();
+
+ if (value.getType() == Gram.Type.HEAD || value.getType() == Gram.Type.TAIL) {
+ // collect frequency for subgrams.
+ if (subgram == null) {
+ subgram = new Gram(value);
+ }
+ else {
+ subgram.incrementFrequency(value.getFrequency());
+ }
+ }
+ else if (!value.equals(currentNgram)) {
+ // we've collected frequency for all subgrams and we've encountered a new ngram.
+ // collect the old ngram if there was one and we have sufficient support and
+ // create the new ngram.
+ if (currentNgram != null) {
+ if (currentNgram.getFrequency() < minSupport) {
+ reporter.incrCounter(Skipped.LESS_THAN_MIN_SUPPORT, 1);
+ }
+ else {
+ output.collect(currentNgram, subgram);
+ }
+ }
+
+ currentNgram = new Gram(value);
+ }
+ else {
+ currentNgram.incrementFrequency(value.getFrequency());
}
}
- // emit ngram:ngramFreq, subgram:subgramFreq pairs.
- subgramKey.setFrequency(subgramFrequency);
-
- for (Gram ngram : ngramSet.keySet()) {
- if (ngram.getFrequency() < minSupport) {
+ // collect last ngram.
+ if (currentNgram != null) {
+ if (currentNgram.getFrequency() < minSupport) {
reporter.incrCounter(Skipped.LESS_THAN_MIN_SUPPORT, 1);
- continue;
+ return;
}
- output.collect(ngram, subgramKey);
+
+ output.collect(currentNgram, subgram);
}
}
}
Modified: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/Gram.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/Gram.java?rev=919798&r1=919797&r2=919798&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/Gram.java (original)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/Gram.java Sat Mar 6 16:38:03 2010
@@ -42,7 +42,8 @@
UNIGRAM('u'),
NGRAM('n');
- private final char x;
+ final char x;
+
Type(char c) {
this.x = c;
}
@@ -101,14 +102,14 @@
try {
// extra character is used for storing type which is part
// of the sort key.
- ByteBuffer bb = Text.encode(ngram + '\0', true);
+ ByteBuffer bb = Text.encode('\0' + ngram, true);
bytes = bb.array();
length = bb.limit();
} catch (CharacterCodingException e) {
throw new IllegalStateException("Should not have happened ",e);
}
- encodeType(type, bytes, length-1);
+ encodeType(type, bytes, 0);
this.frequency = frequency;
}
@@ -127,7 +128,7 @@
* @return the gram is at the head of its text unit or tail or unigram.
*/
public Type getType() {
- return decodeType(bytes, length-1);
+ return decodeType(bytes, 0);
}
/**
@@ -135,7 +136,7 @@
*/
public String getString() {
try {
- return Text.decode(bytes, 0, length-1);
+ return Text.decode(bytes, 1, length-1);
} catch (CharacterCodingException e) {
throw new IllegalStateException("Should not have happened " + e.toString());
}
Added: 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=919798&view=auto
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKey.java (added)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKey.java Sat Mar 6 16:38:03 2010
@@ -0,0 +1,133 @@
+/**
+ * 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.utils.nlp.collocations.llr;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.charset.CharacterCodingException;
+
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.mahout.utils.nlp.collocations.llr.Gram.Type;
+
+/** A GramKey, based on the identity fields of Gram (type, string) plus a byte[] used for secondary ordering */
+public class GramKey extends BinaryComparable implements
+ WritableComparable<BinaryComparable> {
+
+ int primaryLength;
+ int length;
+ byte[] bytes;
+
+ public GramKey() {
+
+ }
+
+ /** create a GramKey based on the specified Gram and order
+ *
+ * @param gram
+ * @param order
+ */
+ public GramKey(Gram gram, byte[] order) {
+ set(gram, order);
+ }
+
+ /** set the gram held by this key */
+ public void set(Gram gram, byte[] order) {
+ primaryLength = gram.getLength();
+ length = primaryLength + order.length;
+ setCapacity(length, false);
+ System.arraycopy(gram.getBytes(), 0, bytes, 0, primaryLength);
+ if (order.length > 0) {
+ System.arraycopy(order, 0, bytes, primaryLength, order.length);
+ }
+ }
+
+ @Override
+ public byte[] getBytes() {
+ return bytes;
+ }
+
+ @Override
+ public int getLength() {
+ return length;
+ }
+
+ public int getPrimaryLength() {
+ return primaryLength;
+ }
+
+ @Override
+ public void readFields(DataInput in) throws IOException {
+ int newLength = WritableUtils.readVInt(in);
+ int newPrimaryLength = WritableUtils.readVInt(in);
+ setCapacity(newLength, false);
+ in.readFully(bytes, 0, newLength);
+ length = newLength;
+ primaryLength = newPrimaryLength;
+
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ WritableUtils.writeVInt(out, length);
+ WritableUtils.writeVInt(out, primaryLength);
+ out.write(bytes, 0, length);
+ }
+
+ /* Cribbed from o.a.hadoop.io.Text:
+ * Sets the capacity of this object to <em>at least</em>
+ * <code>len</code> bytes. If the current buffer is longer,
+ * then the capacity and existing content of the buffer are
+ * unchanged. If <code>len</code> is larger
+ * than the current capacity, this object's capacity is
+ * increased to match.
+ * @param len the number of bytes we need
+ * @param keepData should the old data be kept
+ */
+ private void setCapacity(int len, boolean keepData) {
+ if (bytes == null || bytes.length < len) {
+ byte[] newBytes = new byte[len];
+ if (bytes != null && keepData) {
+ System.arraycopy(bytes, 0, newBytes, 0, length);
+ }
+ bytes = newBytes;
+ }
+ }
+
+ /**
+ * @return the gram is at the head of its text unit or tail or unigram.
+ */
+ public Type getType() {
+ return Gram.decodeType(bytes, 0);
+ }
+
+ public String getPrimaryString() {
+ try {
+ return Text.decode(bytes, 1, primaryLength-1);
+ } catch (CharacterCodingException e) {
+ throw new RuntimeException("Should not have happened " + e.toString());
+ }
+ }
+
+ public String toString() {
+ return '\'' + getPrimaryString() + "'[" + getType().x + "]";
+ }
+}
Propchange: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKey.java
------------------------------------------------------------------------------
svn:eol-style = native
Added: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparator.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparator.java?rev=919798&view=auto
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparator.java (added)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparator.java Sat Mar 6 16:38:03 2010
@@ -0,0 +1,41 @@
+/**
+ * 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.utils.nlp.collocations.llr;
+
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableComparator;
+
+/** Group GramKeys based on their Gram, ignoring the secondary sort key, so that all keys with the same Gram are sent
+ * to the same call of the reduce method, sorted in natural order (for GramKeys).
+ */
+public class GramKeyGroupComparator extends WritableComparator {
+
+ protected GramKeyGroupComparator() {
+ super(GramKey.class, true);
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public int compare(WritableComparable a, WritableComparable b) {
+ GramKey gka = (GramKey) a;
+ GramKey gkb = (GramKey) b;
+
+ return WritableComparator.compareBytes(gka.getBytes(), 0, gka.getPrimaryLength(), gkb.getBytes(), 0, gkb.getPrimaryLength());
+ }
+
+}
Propchange: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparator.java
------------------------------------------------------------------------------
svn:eol-style = native
Added: 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=919798&view=auto
==============================================================================
--- lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitioner.java (added)
+++ lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitioner.java Sat Mar 6 16:38:03 2010
@@ -0,0 +1,51 @@
+/**
+ * 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.utils.nlp.collocations.llr;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Partitioner;
+
+/** Partition GramKeys based on their Gram, ignoring the secondary sort key so that all GramKeys with the same
+ * gram are sent to the same partition.
+ */
+public class GramKeyPartitioner implements Partitioner<GramKey, Gram> {
+
+ private static final String HASH_OFFSET_PROPERTY_NAME = "grampartitioner.hash.offset";
+
+ public static void setOffsets(Configuration conf, int left, int right) {
+ conf.setInt(HASH_OFFSET_PROPERTY_NAME, left);
+ }
+
+ int offset;
+
+ @Override
+ public int getPartition(GramKey key, Gram value, int numPartitions) {
+ // see: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapreduce/lib/partition/BinaryPartitioner.java?revision=816664&view=markup
+ int length = key.getLength()-1;
+ int right = (offset + length) % length;
+ int hash = WritableComparator.hashBytes(key.getBytes(), right);
+ return (hash & Integer.MAX_VALUE) % numPartitions;
+ }
+
+ @Override
+ public void configure(JobConf conf) {
+ offset = conf.getInt(HASH_OFFSET_PROPERTY_NAME, -1);
+ }
+}
Propchange: lucene/mahout/trunk/utils/src/main/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitioner.java
------------------------------------------------------------------------------
svn:eol-style = native
Modified: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/CollocMapperTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/CollocMapperTest.java?rev=919798&r1=919797&r2=919798&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/CollocMapperTest.java (original)
+++ lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/CollocMapperTest.java Sat Mar 6 16:38:03 2010
@@ -35,14 +35,14 @@
import org.junit.Test;
/**
- * Test for CollocMapper FIXME: Add negative test cases
+ * Test for CollocMapper
*/
@SuppressWarnings("deprecation")
public class CollocMapperTest {
- private OutputCollector<Gram,Gram> collector;
+ private OutputCollector<GramKey,Gram> collector;
private Reporter reporter;
-
+
@Before
@SuppressWarnings("unchecked")
public void setUp() {
@@ -82,9 +82,14 @@
if (v[1].equals("of times")) {
frequency = 2;
}
+
Gram subgram = new Gram(v[0].substring(2), frequency, p);
- Gram ngram = new Gram(v[1], frequency, Type.NGRAM);
- collector.collect(subgram, ngram);
+ Gram ngram = new Gram(v[1], frequency, Gram.Type.NGRAM);
+
+ GramKey subgramKey = new GramKey(subgram, new byte[0]);
+ GramKey subgramNgramKey = new GramKey(subgram, ngram.getBytes());
+ collector.collect(subgramKey, subgram);
+ collector.collect(subgramNgramKey, ngram);
}
reporter.incrCounter(CollocMapper.Count.NGRAM_TOTAL, 7);
@@ -129,6 +134,7 @@
{"u_worst", "worst"}, {"u_of", "of"},
{"u_the", "the"}, {"u_best", "best"},
{"u_times", "times"},};
+
// set up expectations for mocks. ngram max size = 2
for (String[] v : values) {
Type p = v[0].startsWith("h") ? Gram.Type.HEAD : Gram.Type.TAIL;
@@ -138,9 +144,23 @@
|| v[1].equals("the")) {
frequency = 2;
}
- Gram subgram = new Gram(v[0].substring(2), frequency, p);
- Gram ngram = new Gram(v[1], frequency, p == Gram.Type.UNIGRAM ? Gram.Type.UNIGRAM : Gram.Type.NGRAM);
- collector.collect(subgram, ngram);
+
+
+
+ if (p == Gram.Type.UNIGRAM) {
+ Gram unigram = new Gram(v[1], frequency, Gram.Type.UNIGRAM);
+ GramKey unigramKey = new GramKey(unigram, new byte[0]);
+ collector.collect(unigramKey, unigram);
+ }
+ else {
+ Gram subgram = new Gram(v[0].substring(2), frequency, p);
+ Gram ngram = new Gram(v[1], frequency, Gram.Type.NGRAM);
+
+ GramKey subgramKey = new GramKey(subgram, new byte[0]);
+ GramKey subgramNgramKey = new GramKey(subgram, ngram.getBytes());
+ collector.collect(subgramKey, subgram);
+ collector.collect(subgramNgramKey, ngram);
+ }
}
reporter.incrCounter(CollocMapper.Count.NGRAM_TOTAL, 7);
Modified: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/CollocReducerTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/CollocReducerTest.java?rev=919798&r1=919797&r2=919798&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/CollocReducerTest.java (original)
+++ lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/CollocReducerTest.java Sat Mar 6 16:38:03 2010
@@ -33,7 +33,7 @@
import org.junit.Test;
/**
- * Test the CollocReducer FIXME: add negative test cases.
+ * Test the CollocReducer
*/
public class CollocReducerTest {
@@ -52,10 +52,11 @@
// test input, input[*][0] is the key,
// input[*][1..n] are the values passed in via
// the iterator.
- Gram[][] input = {{new Gram("the", UNIGRAM), new Gram("the", UNIGRAM), new Gram("the", UNIGRAM)},
- {new Gram("the", HEAD), new Gram("the best", NGRAM), new Gram("the worst", NGRAM)},
- {new Gram("of", HEAD), new Gram("of times", NGRAM), new Gram("of times", NGRAM)},
- {new Gram("times", TAIL), new Gram("of times", NGRAM), new Gram("of times", NGRAM)}};
+ Gram[][] input = {
+ {new Gram("the", UNIGRAM), new Gram("the", UNIGRAM), new Gram("the", UNIGRAM)},
+ {new Gram("the", HEAD), new Gram("the best", NGRAM), new Gram("the worst", NGRAM)},
+ {new Gram("of", HEAD), new Gram("of times", NGRAM), new Gram("of times", NGRAM)},
+ {new Gram("times", TAIL), new Gram("of times", NGRAM), new Gram("of times", NGRAM)}};
// expected results.
Gram[][] values = {{new Gram("the", 2, UNIGRAM), new Gram("the", 2, UNIGRAM)},
@@ -64,6 +65,8 @@
{new Gram("of times", 2, NGRAM), new Gram("of", 2, HEAD)},
{new Gram("of times", 2, NGRAM), new Gram("times", 2, TAIL)}};
+ byte[] empty = new byte[0];
+
// set up expectations
for (Gram[] v : values) {
output.collect(v[0], v[1]);
@@ -73,10 +76,14 @@
// play back the input data.
CollocReducer c = new CollocReducer();
+ GramKey key = new GramKey();
+
for (Gram[] ii : input) {
+ key.set(ii[0], empty);
+
List<Gram> vv = new LinkedList<Gram>();
- vv.addAll(Arrays.asList(ii).subList(1, ii.length));
- c.reduce(ii[0], vv.iterator(), output, reporter);
+ vv.addAll(Arrays.asList(ii));
+ c.reduce(key, vv.iterator(), output, reporter);
}
EasyMock.verify(reporter, output);
Added: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparatorTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparatorTest.java?rev=919798&view=auto
==============================================================================
--- lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparatorTest.java (added)
+++ lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparatorTest.java Sat Mar 6 16:38:03 2010
@@ -0,0 +1,46 @@
+/**
+ * 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.utils.nlp.collocations.llr;
+
+import junit.framework.Assert;
+
+import org.junit.Test;
+
+
+public class GramKeyGroupComparatorTest {
+ @Test
+ public void testComparator() {
+ byte[] empty = new byte[0];
+ byte[] foo = new byte[1];
+ foo[0] = (byte) 1;
+
+ GramKey a = new GramKey(new Gram("foo", 1, Gram.Type.HEAD), empty); // base
+ GramKey b = new GramKey(new Gram("foo", 1, Gram.Type.HEAD), foo); // vary byte
+ GramKey c = new GramKey(new Gram("foo", 2, Gram.Type.HEAD), empty); // vary freq
+ GramKey d = new GramKey(new Gram("foo", 1, Gram.Type.TAIL), empty); // vary type
+ GramKey e = new GramKey(new Gram("bar", 5, Gram.Type.HEAD), empty); // vary string
+
+ GramKeyGroupComparator cmp = new GramKeyGroupComparator();
+
+ Assert.assertTrue(0 == cmp.compare(a, b));
+ Assert.assertTrue(0 == cmp.compare(a, c));
+ Assert.assertTrue(0 > cmp.compare(a, d));
+ Assert.assertTrue(0 < cmp.compare(a, e));
+ Assert.assertTrue(0 < cmp.compare(d, e));
+ }
+}
Propchange: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyGroupComparatorTest.java
------------------------------------------------------------------------------
svn:eol-style = native
Added: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitionerTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitionerTest.java?rev=919798&view=auto
==============================================================================
--- lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitionerTest.java (added)
+++ lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitionerTest.java Sat Mar 6 16:38:03 2010
@@ -0,0 +1,54 @@
+/**
+ * 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.utils.nlp.collocations.llr;
+
+import junit.framework.Assert;
+
+import org.junit.Test;
+
+
+public class GramKeyPartitionerTest {
+ @Test
+ public void testPartition() {
+ byte[] empty = new byte[0];
+ byte[] foo = new byte[1];
+ foo[0] = 1;
+
+ byte[] bar = new byte[1];
+ foo[0] = 2;
+
+ GramKey a = new GramKey(new Gram("foo", 1, Gram.Type.HEAD), empty);
+ GramKey b = new GramKey(new Gram("foo", 1, Gram.Type.HEAD), foo);
+ GramKey c = new GramKey(new Gram("foo", 2, Gram.Type.HEAD), bar);
+ GramKey d = new GramKey(new Gram("foo", 1, Gram.Type.TAIL), empty);
+ GramKey e = new GramKey(new Gram("foo", 2, Gram.Type.TAIL), foo);
+
+ GramKeyPartitioner p = new GramKeyPartitioner();
+ int numPartitions = 5;
+
+ int ap = p.getPartition(a, null, numPartitions);
+ int bp = p.getPartition(b, null, numPartitions);
+ int cp = p.getPartition(c, null, numPartitions);
+ int dp = p.getPartition(d, null, numPartitions);
+ int ep = p.getPartition(e, null, numPartitions);
+
+ Assert.assertEquals(ap, bp);
+ Assert.assertEquals(ap, cp);
+ Assert.assertEquals(dp, ep);
+ }
+}
Propchange: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyPartitionerTest.java
------------------------------------------------------------------------------
svn:eol-style = native
Added: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyTest.java?rev=919798&view=auto
==============================================================================
--- lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyTest.java (added)
+++ lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyTest.java Sat Mar 6 16:38:03 2010
@@ -0,0 +1,101 @@
+/**
+ * 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.utils.nlp.collocations.llr;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+
+import junit.framework.Assert;
+import junit.framework.TestCase;
+
+import org.junit.Test;
+
+
+public class GramKeyTest {
+ @Test
+ public void testGramKeySort() {
+ byte[] empty = new byte[0];
+ byte[] foo = new byte[1]; foo[0] = 1;
+ byte[] bar = new byte[1]; bar[0] = 2;
+
+
+ // byte argument in GramKey breaks tie between equal grams
+ GramKey[] input =
+ {
+ new GramKey(new Gram("bar", 1, Gram.Type.UNIGRAM), empty),
+ new GramKey(new Gram("bar", 1, Gram.Type.UNIGRAM), empty),
+ new GramKey(new Gram("bar", 1, Gram.Type.UNIGRAM), foo),
+ new GramKey(new Gram("bar", 8, Gram.Type.NGRAM), foo),
+ new GramKey(new Gram("bar", 8, Gram.Type.NGRAM), empty),
+ new GramKey(new Gram("foo", 2, Gram.Type.HEAD), foo),
+ new GramKey(new Gram("foo", 3, Gram.Type.HEAD), empty),
+ new GramKey(new Gram("foo", 4, Gram.Type.TAIL), foo),
+ new GramKey(new Gram("foo", 5, Gram.Type.TAIL), foo),
+ new GramKey(new Gram("bar", 6, Gram.Type.HEAD), foo),
+ new GramKey(new Gram("bar", 7, Gram.Type.TAIL), empty),
+ };
+
+ int[] expect = {
+ 9, 6, 5, 10, 7, 8, 0, 1, 2, 4, 3
+ };
+
+ GramKey[] sorted = new GramKey[input.length];
+
+ System.arraycopy(input, 0, sorted, 0, input.length);
+
+ Arrays.sort(sorted);
+
+ for (int i=0; i < input.length; i++) {
+ TestCase.assertSame(input[expect[i]], sorted[i]);
+ }
+ }
+
+ @Test
+ public void testWritable() throws IOException {
+ byte[] foo = new byte[0];
+ byte[] bar = new byte[1]; bar[0] = 2;
+
+ GramKey one = new GramKey(new Gram("foo", 2, Gram.Type.HEAD), foo);
+ GramKey two = new GramKey(new Gram("foobar", 3, Gram.Type.UNIGRAM), bar);
+
+ Assert.assertEquals("foo", one.getPrimaryString());
+ Assert.assertEquals("foobar", two.getPrimaryString());
+
+ Assert.assertEquals(Gram.Type.UNIGRAM, two.getType());
+
+ ByteArrayOutputStream bout = new ByteArrayOutputStream();
+ DataOutputStream out = new DataOutputStream(bout);
+
+ two.write(out);
+
+ byte[] b = bout.toByteArray();
+
+ ByteArrayInputStream bin = new ByteArrayInputStream(b);
+ DataInputStream din = new DataInputStream(bin);
+
+ one.readFields(din);
+
+ Assert.assertTrue(Arrays.equals(two.getBytes(), one.getBytes()));
+ Assert.assertEquals(Gram.Type.UNIGRAM, one.getType());
+
+ }
+}
Propchange: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramKeyTest.java
------------------------------------------------------------------------------
svn:eol-style = native
Modified: lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramTest.java?rev=919798&r1=919797&r2=919798&view=diff
==============================================================================
--- lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramTest.java (original)
+++ lucene/mahout/trunk/utils/src/test/java/org/apache/mahout/utils/nlp/collocations/llr/GramTest.java Sat Mar 6 16:38:03 2010
@@ -22,6 +22,7 @@
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
+import java.util.Arrays;
import java.util.HashMap;
import junit.framework.Assert;
@@ -153,7 +154,6 @@
};
for (int i = 0; i < input.length; i++) {
- System.err.println(i);
Assert.assertEquals(freq[i], input[i].getFrequency());
Assert.assertEquals(memb[i], input[i] == map.get(input[i]));
}
@@ -189,4 +189,34 @@
Assert.assertEquals(Gram.Type.UNIGRAM, one.getType());
}
+
+ @Test
+ public void testSorting() throws IOException {
+ Gram[] input =
+ {
+ new Gram("foo", 2, Gram.Type.HEAD),
+ new Gram("foo", 3, Gram.Type.HEAD),
+ new Gram("foo", 4, Gram.Type.TAIL),
+ new Gram("foo", 5, Gram.Type.TAIL),
+ new Gram("bar", 6, Gram.Type.HEAD),
+ new Gram("bar", 7, Gram.Type.TAIL),
+ new Gram("bar", 8, Gram.Type.NGRAM),
+ new Gram("bar", Gram.Type.UNIGRAM)
+ };
+
+ Gram[] sorted = new Gram[input.length];
+
+ int[] expectations = {
+ 4, 0, 1, 5, 2, 3, 7, 6
+ };
+
+
+ System.arraycopy(input, 0, sorted, 0, input.length);
+
+ Arrays.sort(sorted);
+
+ for (int i=0; i < sorted.length; i++) {
+ Assert.assertSame(input[expectations[i]], sorted[i]);
+ }
+ }
}