You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by gs...@apache.org on 2011/05/05 19:55:23 UTC

svn commit: r1099894 - in /mahout/trunk: ./ core/src/main/java/org/apache/mahout/classifier/bayes/mapreduce/common/ core/src/main/java/org/apache/mahout/common/lucene/ core/src/main/java/org/apache/mahout/vectorizer/ core/src/main/java/org/apache/mahou...

Author: gsingers
Date: Thu May  5 17:55:22 2011
New Revision: 1099894

URL: http://svn.apache.org/viewvc?rev=1099894&view=rev
Log:
MAHOUT-686: update to Lucene/Solr 3.1.0

Added:
    mahout/trunk/core/src/main/java/org/apache/mahout/common/lucene/
    mahout/trunk/core/src/main/java/org/apache/mahout/common/lucene/IteratorTokenStream.java
    mahout/trunk/core/src/main/java/org/apache/mahout/common/lucene/TokenStreamIterator.java
      - copied, changed from r1099451, mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/encoders/TokenStreamIterator.java
Removed:
    mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/encoders/TokenStreamIterator.java
    mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/encoders/TokenizationException.java
Modified:
    mahout/trunk/core/src/main/java/org/apache/mahout/classifier/bayes/mapreduce/common/BayesFeatureMapper.java
    mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/DefaultAnalyzer.java
    mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/collocations/llr/CollocMapper.java
    mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/document/SequenceFileTokenizerMapper.java
    mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/encoders/LuceneTextValueEncoder.java
    mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/term/TFPartialVectorReducer.java
    mahout/trunk/examples/pom.xml
    mahout/trunk/examples/src/main/java/org/apache/mahout/analysis/WikipediaAnalyzer.java
    mahout/trunk/examples/src/main/java/org/apache/mahout/classifier/bayes/WikipediaDatasetCreatorMapper.java
    mahout/trunk/examples/src/main/java/org/apache/mahout/classifier/sgd/TrainNewsGroups.java
    mahout/trunk/pom.xml
    mahout/trunk/utils/src/main/java/org/apache/mahout/text/MailArchivesClusteringAnalyzer.java
    mahout/trunk/utils/src/test/java/org/apache/mahout/text/MailArchivesClusteringAnalyzerTest.java

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/classifier/bayes/mapreduce/common/BayesFeatureMapper.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/classifier/bayes/mapreduce/common/BayesFeatureMapper.java?rev=1099894&r1=1099893&r2=1099894&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/classifier/bayes/mapreduce/common/BayesFeatureMapper.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/classifier/bayes/mapreduce/common/BayesFeatureMapper.java Thu May  5 17:55:22 2011
@@ -18,7 +18,6 @@
 package org.apache.mahout.classifier.bayes.mapreduce.common;
 
 import java.io.IOException;
-import java.util.Iterator;
 import java.util.regex.Pattern;
 
 import com.google.common.collect.Iterators;
@@ -30,11 +29,11 @@ import org.apache.hadoop.mapred.MapReduc
 import org.apache.hadoop.mapred.Mapper;
 import org.apache.hadoop.mapred.OutputCollector;
 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.bayes.common.BayesParameters;
 import org.apache.mahout.common.StringTuple;
+import org.apache.mahout.common.lucene.IteratorTokenStream;
 import org.apache.mahout.math.function.ObjectIntProcedure;
 import org.apache.mahout.math.map.OpenObjectIntHashMap;
 import org.slf4j.Logger;
@@ -177,26 +176,5 @@ public class BayesFeatureMapper extends 
       log.warn(ex.toString(), ex);
     }
   }
-  
-  /** Used to emit tokens from an input string array in the style of TokenStream */
-  public static class IteratorTokenStream extends TokenStream {
-    private final TermAttribute termAtt;
-    private final Iterator<String> iterator;
-    
-    public IteratorTokenStream(Iterator<String> iterator) {
-      this.iterator = iterator;
-      this.termAtt = addAttribute(TermAttribute.class);
-    }
-    
-    @Override
-    public boolean incrementToken() throws IOException {
-      if (iterator.hasNext()) {
-        clearAttributes();
-        termAtt.setTermBuffer(iterator.next());
-        return true;
-      } else {
-        return false;
-      }
-    }
-  }
+
 }

Added: mahout/trunk/core/src/main/java/org/apache/mahout/common/lucene/IteratorTokenStream.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/common/lucene/IteratorTokenStream.java?rev=1099894&view=auto
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/common/lucene/IteratorTokenStream.java (added)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/common/lucene/IteratorTokenStream.java Thu May  5 17:55:22 2011
@@ -0,0 +1,47 @@
+package org.apache.mahout.common.lucene;
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+
+
+import java.io.IOException;
+import java.util.Iterator;
+
+/** Used to emit tokens from an input string array in the style of TokenStream */
+public final class IteratorTokenStream extends TokenStream {
+  private final CharTermAttribute termAtt;
+  private final Iterator<String> iterator;
+
+  public IteratorTokenStream(Iterator<String> iterator) {
+    this.iterator = iterator;
+    this.termAtt = addAttribute(CharTermAttribute.class);
+  }
+
+  @Override
+  public boolean incrementToken() throws IOException {
+    if (iterator.hasNext()) {
+      clearAttributes();
+      termAtt.append(iterator.next());
+      return true;
+    } else {
+      return false;
+    }
+  }
+}

Copied: mahout/trunk/core/src/main/java/org/apache/mahout/common/lucene/TokenStreamIterator.java (from r1099451, mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/encoders/TokenStreamIterator.java)
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/common/lucene/TokenStreamIterator.java?p2=mahout/trunk/core/src/main/java/org/apache/mahout/common/lucene/TokenStreamIterator.java&p1=mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/encoders/TokenStreamIterator.java&r1=1099451&r2=1099894&rev=1099894&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/encoders/TokenStreamIterator.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/common/lucene/TokenStreamIterator.java Thu May  5 17:55:22 2011
@@ -15,19 +15,23 @@
  * limitations under the License.
  */
 
-package org.apache.mahout.vectorizer.encoders;
-
-import java.io.IOException;
+package org.apache.mahout.common.lucene;
 
 import com.google.common.collect.AbstractIterator;
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 
-final class TokenStreamIterator extends AbstractIterator<String> {
+import java.io.IOException;
+
+/**
+ * Provide an Iterator for the tokens in a TokenStream.
+ */
+//TODO: consider using the char/byte arrays instead of strings, esp. when we upgrade to Lucene 4.0
+public final class TokenStreamIterator extends AbstractIterator<String> {
 
   private final TokenStream tokenStream;
 
-  TokenStreamIterator(TokenStream tokenStream) {
+  public TokenStreamIterator(TokenStream tokenStream) {
     this.tokenStream = tokenStream;
   }
 
@@ -35,12 +39,12 @@ final class TokenStreamIterator extends 
   protected String computeNext() {
     try {
       if (tokenStream.incrementToken()) {
-        return tokenStream.getAttribute(TermAttribute.class).term();
+        return tokenStream.getAttribute(CharTermAttribute.class).toString();
       } else {
         return endOfData();
       }
     } catch (IOException e) {
-      throw new TokenizationException("IO error while tokenizing", e);
+      throw new RuntimeException("IO error while tokenizing", e);
     }
   }
 

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/DefaultAnalyzer.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/DefaultAnalyzer.java?rev=1099894&r1=1099893&r2=1099894&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/DefaultAnalyzer.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/DefaultAnalyzer.java Thu May  5 17:55:22 2011
@@ -16,18 +16,28 @@
  */
 package org.apache.mahout.vectorizer;
 
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.util.Version;
 
+import java.io.Reader;
+
 /**
  *  A subclass of the Lucene StandardAnalyzer that provides a no-argument constructor. 
  *  Used as the default analyzer in many cases where an analyzer is instantiated by
  *  class name by calling a no-arg constructor.
  */
-public class DefaultAnalyzer extends StandardAnalyzer {
+public final class DefaultAnalyzer extends Analyzer {
+
+  private StandardAnalyzer stdAnalyzer = new StandardAnalyzer(Version.LUCENE_31);
 
   public DefaultAnalyzer() {
-    super(Version.LUCENE_30);
+    super();
   }
 
+  @Override
+  public TokenStream tokenStream(String fieldName, Reader reader) {
+    return stdAnalyzer.tokenStream(fieldName, reader);
+  }
 }

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/collocations/llr/CollocMapper.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/collocations/llr/CollocMapper.java?rev=1099894&r1=1099893&r2=1099894&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/collocations/llr/CollocMapper.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/collocations/llr/CollocMapper.java Thu May  5 17:55:22 2011
@@ -17,24 +17,23 @@
 
 package org.apache.mahout.vectorizer.collocations.llr;
 
-import java.io.IOException;
-import java.util.Iterator;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.shingle.ShingleFilter;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
 import org.apache.mahout.common.StringTuple;
+import org.apache.mahout.common.lucene.IteratorTokenStream;
 import org.apache.mahout.math.function.ObjectIntProcedure;
 import org.apache.mahout.math.map.OpenObjectIntHashMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+
 /**
- * Pass 1 of the Collocation discovery job which generated ngrams and emits ngrams an their component n-1grams. 
+ * 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/>
  */
@@ -62,31 +61,30 @@ public class CollocMapper extends Mapper
    * 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
-   * 
+   * <p/>
    * <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>
-   * 
+   * <p/>
    * 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.
    * <p/>
-   * For example, given 'click and clack' and an ngram length of 3: 
+   * 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>
-   * 
+   * <p/>
    * Also counts the total number of ngrams encountered and adds it to the counter
    * CollocDriver.Count.NGRAM_TOTAL
    * </p>
-   * 
-   * @throws IOException
-   *           if there's a problem with the ShingleFilter reading data or the collector collecting output.
+   *
+   * @throws IOException if there's a problem with the ShingleFilter reading data or the collector collecting output.
    */
   @Override
   protected void map(Text key, StringTuple value, final Context context) throws IOException, InterruptedException {
@@ -95,11 +93,11 @@ public class CollocMapper extends Mapper
     int count = 0; // ngram count
 
     OpenObjectIntHashMap<String> ngrams =
-        new OpenObjectIntHashMap<String>(value.getEntries().size() * (maxShingleSize - 1));
+            new OpenObjectIntHashMap<String>(value.getEntries().size() * (maxShingleSize - 1));
     OpenObjectIntHashMap<String> unigrams = new OpenObjectIntHashMap<String>(value.getEntries().size());
 
     do {
-      String term = sf.getAttribute(TermAttribute.class).term();
+      String term = sf.getAttribute(CharTermAttribute.class).toString();
       String type = sf.getAttribute(TypeAttribute.class).type();
       if ("shingle".equals(type)) {
         count++;
@@ -181,26 +179,4 @@ public class CollocMapper extends Mapper
     }
   }
 
-  /** Used to emit tokens from an input string array in the style of TokenStream */
-  public static class IteratorTokenStream extends TokenStream {
-    private final TermAttribute termAtt;
-
-    private final Iterator<String> iterator;
-
-    public IteratorTokenStream(Iterator<String> iterator) {
-      this.iterator = iterator;
-      this.termAtt = addAttribute(TermAttribute.class);
-    }
-
-    @Override
-    public boolean incrementToken() throws IOException {
-      if (iterator.hasNext()) {
-        clearAttributes();
-        termAtt.setTermBuffer(iterator.next());
-        return true;
-      } else {
-        return false;
-      }
-    }
-  }
 }

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/document/SequenceFileTokenizerMapper.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/document/SequenceFileTokenizerMapper.java?rev=1099894&r1=1099893&r2=1099894&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/document/SequenceFileTokenizerMapper.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/document/SequenceFileTokenizerMapper.java Thu May  5 17:55:22 2011
@@ -24,7 +24,8 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
+
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.mahout.common.StringTuple;
 import org.apache.mahout.vectorizer.DefaultAnalyzer;
 import org.apache.mahout.vectorizer.DocumentProcessor;
@@ -39,11 +40,11 @@ public class SequenceFileTokenizerMapper
   @Override
   protected void map(Text key, Text value, Context context) throws IOException, InterruptedException {
     TokenStream stream = analyzer.tokenStream(key.toString(), new StringReader(value.toString()));
-    TermAttribute termAtt = stream.addAttribute(TermAttribute.class);
+    CharTermAttribute termAtt = stream.addAttribute(CharTermAttribute.class);
     StringTuple document = new StringTuple();
     while (stream.incrementToken()) {
-      if (termAtt.termLength() > 0) {
-        document.add(new String(termAtt.termBuffer(), 0, termAtt.termLength()));
+      if (termAtt.length() > 0) {
+        document.add(new String(termAtt.buffer(), 0, termAtt.length()));
       }
     }
     context.write(key, document);

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/encoders/LuceneTextValueEncoder.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/encoders/LuceneTextValueEncoder.java?rev=1099894&r1=1099893&r2=1099894&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/encoders/LuceneTextValueEncoder.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/encoders/LuceneTextValueEncoder.java Thu May  5 17:55:22 2011
@@ -19,7 +19,9 @@ package org.apache.mahout.vectorizer.enc
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.mahout.common.lucene.TokenStreamIterator;
+
 
 import java.io.IOException;
 import java.io.Reader;
@@ -48,7 +50,7 @@ public class LuceneTextValueEncoder exte
   @Override
   protected Iterable<String> tokenize(CharSequence originalForm) {
     TokenStream ts = analyzer.tokenStream(getName(), new CharSequenceReader(originalForm));
-    ts.addAttribute(TermAttribute.class);
+    ts.addAttribute(CharTermAttribute.class);
     return new LuceneTokenIterable(ts);
   }
 

Modified: mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/term/TFPartialVectorReducer.java
URL: http://svn.apache.org/viewvc/mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/term/TFPartialVectorReducer.java?rev=1099894&r1=1099893&r2=1099894&view=diff
==============================================================================
--- mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/term/TFPartialVectorReducer.java (original)
+++ mahout/trunk/core/src/main/java/org/apache/mahout/vectorizer/term/TFPartialVectorReducer.java Thu May  5 17:55:22 2011
@@ -17,10 +17,6 @@
 
 package org.apache.mahout.vectorizer.term;
 
-import java.io.IOException;
-import java.net.URI;
-import java.util.Iterator;
-
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.filecache.DistributedCache;
@@ -30,19 +26,23 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.lucene.analysis.shingle.ShingleFilter;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.mahout.common.Pair;
 import org.apache.mahout.common.StringTuple;
 import org.apache.mahout.common.iterator.sequencefile.SequenceFileIterable;
+import org.apache.mahout.common.lucene.IteratorTokenStream;
 import org.apache.mahout.math.NamedVector;
 import org.apache.mahout.math.RandomAccessSparseVector;
 import org.apache.mahout.math.SequentialAccessSparseVector;
 import org.apache.mahout.math.Vector;
 import org.apache.mahout.math.VectorWritable;
 import org.apache.mahout.math.map.OpenObjectIntHashMap;
-import org.apache.mahout.vectorizer.collocations.llr.CollocMapper.IteratorTokenStream;
-import org.apache.mahout.vectorizer.common.PartialVectorMerger;
 import org.apache.mahout.vectorizer.DictionaryVectorizer;
+import org.apache.mahout.vectorizer.common.PartialVectorMerger;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Iterator;
 
 /**
  * Converts a document in to a sparse vector
@@ -56,12 +56,12 @@ public class TFPartialVectorReducer exte
   private boolean sequentialAccess;
 
   private boolean namedVector;
-  
+
   private int maxNGramSize = 1;
 
   @Override
   protected void reduce(Text key, Iterable<StringTuple> values, Context context)
-    throws IOException, InterruptedException {
+          throws IOException, InterruptedException {
     Iterator<StringTuple> it = values.iterator();
     if (!it.hasNext()) {
       return;
@@ -74,7 +74,7 @@ public class TFPartialVectorReducer exte
       ShingleFilter sf = new ShingleFilter(new IteratorTokenStream(value.getEntries().iterator()), maxNGramSize);
 
       do {
-        String term = sf.getAttribute(TermAttribute.class).term();
+        String term = (sf.getAttribute(CharTermAttribute.class)).toString();
         if (term.length() > 0 && dictionary.containsKey(term)) { // ngram
           int termId = dictionary.get(term);
           vector.setQuick(termId, vector.getQuick(termId) + 1);
@@ -94,11 +94,11 @@ public class TFPartialVectorReducer exte
     if (sequentialAccess) {
       vector = new SequentialAccessSparseVector(vector);
     }
-    
+
     if (namedVector) {
       vector = new NamedVector(vector, key.toString());
     }
-    
+
     // if the vector has no nonZero entries (nothing in the dictionary), let's not waste space sending it to disk.
     if (vector.getNumNondefaultElements() > 0) {
       VectorWritable vectorWritable = new VectorWritable(vector);
@@ -114,8 +114,8 @@ public class TFPartialVectorReducer exte
     Configuration conf = context.getConfiguration();
     URI[] localFiles = DistributedCache.getCacheFiles(conf);
     Preconditions.checkArgument(localFiles != null && localFiles.length >= 1,
-        "missing paths from the DistributedCache");
-    
+            "missing paths from the DistributedCache");
+
     dimension = conf.getInt(PartialVectorMerger.DIMENSION, Integer.MAX_VALUE);
     sequentialAccess = conf.getBoolean(PartialVectorMerger.SEQUENTIAL_ACCESS, false);
     namedVector = conf.getBoolean(PartialVectorMerger.NAMED_VECTOR, false);
@@ -123,8 +123,8 @@ public class TFPartialVectorReducer exte
 
     Path dictionaryFile = new Path(localFiles[0].getPath());
     // key is word value is id
-    for (Pair<Writable,IntWritable> record
-         : new SequenceFileIterable<Writable,IntWritable>(dictionaryFile, true, conf)) {
+    for (Pair<Writable, IntWritable> record
+            : new SequenceFileIterable<Writable, IntWritable>(dictionaryFile, true, conf)) {
       dictionary.put(record.getFirst().toString(), record.getSecond().get());
     }
   }

Modified: mahout/trunk/examples/pom.xml
URL: http://svn.apache.org/viewvc/mahout/trunk/examples/pom.xml?rev=1099894&r1=1099893&r2=1099894&view=diff
==============================================================================
--- mahout/trunk/examples/pom.xml (original)
+++ mahout/trunk/examples/pom.xml Thu May  5 17:55:22 2011
@@ -157,13 +157,14 @@
       <artifactId>mahout-utils</artifactId>
     </dependency>
 
-
-    <!-- A Lucene wikipedia tokenizer/analyzer -->
     <dependency>
-      <groupId>org.apache.lucene</groupId>
-      <artifactId>lucene-wikipedia</artifactId>
+        <groupId>org.apache.lucene</groupId>
+        <artifactId>lucene-benchmark</artifactId>
     </dependency>
-
+    <dependency>
+        <groupId>org.apache.lucene</groupId>
+        <artifactId>lucene-analyzers</artifactId>
+     </dependency>
 
     <dependency>
       <groupId>org.uncommons.watchmaker</groupId>

Modified: mahout/trunk/examples/src/main/java/org/apache/mahout/analysis/WikipediaAnalyzer.java
URL: http://svn.apache.org/viewvc/mahout/trunk/examples/src/main/java/org/apache/mahout/analysis/WikipediaAnalyzer.java?rev=1099894&r1=1099893&r2=1099894&view=diff
==============================================================================
--- mahout/trunk/examples/src/main/java/org/apache/mahout/analysis/WikipediaAnalyzer.java (original)
+++ mahout/trunk/examples/src/main/java/org/apache/mahout/analysis/WikipediaAnalyzer.java Thu May  5 17:55:22 2011
@@ -26,14 +26,16 @@ import org.apache.lucene.analysis.StopAn
 import org.apache.lucene.analysis.StopFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.standard.StandardFilter;
-import org.apache.lucene.wikipedia.analysis.WikipediaTokenizer;
+import org.apache.lucene.analysis.wikipedia.WikipediaTokenizer;
+import org.apache.lucene.util.Version;
+
 
 public class WikipediaAnalyzer extends Analyzer {
  
   private final CharArraySet stopSet;
   
   public WikipediaAnalyzer() {
-    stopSet = (CharArraySet) StopFilter.makeStopSet(
+    stopSet = (CharArraySet) StopFilter.makeStopSet(Version.LUCENE_31,
         StopAnalyzer.ENGLISH_STOP_WORDS_SET.toArray(new String[StopAnalyzer.ENGLISH_STOP_WORDS_SET.size()]));
   }
   
@@ -44,9 +46,9 @@ public class WikipediaAnalyzer extends A
   @Override
   public TokenStream tokenStream(String fieldName, Reader reader) {
     TokenStream result = new WikipediaTokenizer(reader);
-    result = new StandardFilter(result);
-    result = new LowerCaseFilter(result);
-    result = new StopFilter(true, result, stopSet);
+    result = new StandardFilter(Version.LUCENE_31, result);
+    result = new LowerCaseFilter(Version.LUCENE_31, result);
+    result = new StopFilter(Version.LUCENE_31, result, stopSet);
     return result;
   }
 }

Modified: mahout/trunk/examples/src/main/java/org/apache/mahout/classifier/bayes/WikipediaDatasetCreatorMapper.java
URL: http://svn.apache.org/viewvc/mahout/trunk/examples/src/main/java/org/apache/mahout/classifier/bayes/WikipediaDatasetCreatorMapper.java?rev=1099894&r1=1099893&r2=1099894&view=diff
==============================================================================
--- mahout/trunk/examples/src/main/java/org/apache/mahout/classifier/bayes/WikipediaDatasetCreatorMapper.java (original)
+++ mahout/trunk/examples/src/main/java/org/apache/mahout/classifier/bayes/WikipediaDatasetCreatorMapper.java Thu May  5 17:55:22 2011
@@ -35,7 +35,8 @@ import org.apache.hadoop.mapreduce.Mappe
 import org.apache.hadoop.util.GenericsUtil;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
+
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.mahout.analysis.WikipediaAnalyzer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -67,9 +68,9 @@ public class WikipediaDatasetCreatorMapp
       document = StringEscapeUtils.unescapeHtml(WikipediaDatasetCreatorMapper.CLOSE_TEXT_TAG_PATTERN.matcher(
           WikipediaDatasetCreatorMapper.OPEN_TEXT_TAG_PATTERN.matcher(document).replaceFirst("")).replaceAll(""));
       TokenStream stream = analyzer.tokenStream(catMatch, new StringReader(document));
-      TermAttribute termAtt = stream.addAttribute(TermAttribute.class);
+      CharTermAttribute termAtt = stream.addAttribute(CharTermAttribute.class);
       while (stream.incrementToken()) {
-        contents.append(termAtt.termBuffer(), 0, termAtt.termLength()).append(' ');
+        contents.append(termAtt.buffer(), 0, termAtt.length()).append(' ');
       }
       context.write(
           new Text(WikipediaDatasetCreatorMapper.SPACE_NON_ALPHA_PATTERN.matcher(catMatch).replaceAll("_")),

Modified: mahout/trunk/examples/src/main/java/org/apache/mahout/classifier/sgd/TrainNewsGroups.java
URL: http://svn.apache.org/viewvc/mahout/trunk/examples/src/main/java/org/apache/mahout/classifier/sgd/TrainNewsGroups.java?rev=1099894&r1=1099893&r2=1099894&view=diff
==============================================================================
--- mahout/trunk/examples/src/main/java/org/apache/mahout/classifier/sgd/TrainNewsGroups.java (original)
+++ mahout/trunk/examples/src/main/java/org/apache/mahout/classifier/sgd/TrainNewsGroups.java Thu May  5 17:55:22 2011
@@ -28,7 +28,8 @@ import com.google.common.io.Files;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
+
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.util.Version;
 import org.apache.mahout.common.RandomUtils;
 import org.apache.mahout.ep.State;
@@ -319,9 +320,9 @@ public final class TrainNewsGroups {
 
   private static void countWords(Analyzer analyzer, Collection<String> words, Reader in) throws IOException {
     TokenStream ts = analyzer.tokenStream("text", in);
-    ts.addAttribute(TermAttribute.class);
+    ts.addAttribute(CharTermAttribute.class);
     while (ts.incrementToken()) {
-      String s = ts.getAttribute(TermAttribute.class).term();
+      String s = ts.getAttribute(CharTermAttribute.class).toString();
       words.add(s);
     }
     overallCounts.addAll(words);

Modified: mahout/trunk/pom.xml
URL: http://svn.apache.org/viewvc/mahout/trunk/pom.xml?rev=1099894&r1=1099893&r2=1099894&view=diff
==============================================================================
--- mahout/trunk/pom.xml (original)
+++ mahout/trunk/pom.xml Thu May  5 17:55:22 2011
@@ -157,23 +157,22 @@
       <dependency>
         <groupId>org.apache.lucene</groupId>
         <artifactId>lucene-analyzers</artifactId>
-        <version>3.0.2</version>
+        <version>3.1.0</version>
       </dependency>
       <dependency>
         <groupId>org.apache.lucene</groupId>
         <artifactId>lucene-core</artifactId>
-        <version>3.0.2</version>
+        <version>3.1.0</version>
       </dependency>
       <dependency>
         <groupId>org.apache.lucene</groupId>
-        <artifactId>lucene-wikipedia</artifactId>
-        <version>3.0.2</version>
+        <artifactId>lucene-benchmark</artifactId>
+        <version>3.1.0</version>
       </dependency>
-
       <dependency>
         <groupId>org.apache.solr</groupId>
         <artifactId>solr-commons-csv</artifactId>
-        <version>1.4.1</version>
+        <version>3.1.0</version>
       </dependency>
       
       <dependency>

Modified: mahout/trunk/utils/src/main/java/org/apache/mahout/text/MailArchivesClusteringAnalyzer.java
URL: http://svn.apache.org/viewvc/mahout/trunk/utils/src/main/java/org/apache/mahout/text/MailArchivesClusteringAnalyzer.java?rev=1099894&r1=1099893&r2=1099894&view=diff
==============================================================================
--- mahout/trunk/utils/src/main/java/org/apache/mahout/text/MailArchivesClusteringAnalyzer.java (original)
+++ mahout/trunk/utils/src/main/java/org/apache/mahout/text/MailArchivesClusteringAnalyzer.java Thu May  5 17:55:22 2011
@@ -30,7 +30,8 @@ import org.apache.lucene.analysis.TokenF
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.standard.StandardFilter;
 import org.apache.lucene.analysis.standard.StandardTokenizer;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
+
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.util.Version;
 
 /**
@@ -38,7 +39,7 @@ import org.apache.lucene.util.Version;
  * for clustering the ASF Mail Archives using an extended set of
  * stop words, excluding non-alpha-numeric tokens, and porter stemming.
  */
-public class MailArchivesClusteringAnalyzer extends Analyzer {
+public final class MailArchivesClusteringAnalyzer extends Analyzer {
   
   // extended set of stop words composed of common mail terms like "hi",
   // HTML tags, and Java keywords asmany of the messages in the archives
@@ -103,7 +104,7 @@ public class MailArchivesClusteringAnaly
   private final CharArraySet stopSet;
 
   public MailArchivesClusteringAnalyzer() {
-    stopSet = (CharArraySet)StopFilter.makeStopSet(Arrays.asList(STOP_WORDS));
+    stopSet = (CharArraySet)StopFilter.makeStopSet(Version.LUCENE_31, Arrays.asList(STOP_WORDS));
     /*
     Collection<String> tmp = new java.util.TreeSet<String>();
     for (Object entry : stopSet) {
@@ -118,13 +119,13 @@ public class MailArchivesClusteringAnaly
 
   @Override
   public TokenStream tokenStream(String fieldName, java.io.Reader reader) {
-    @SuppressWarnings("deprecation")
-    TokenStream result = new StandardTokenizer(Version.LUCENE_CURRENT, reader);
-    result = new StandardFilter(result);
-    result = new LowerCaseFilter(result);
+
+    TokenStream result = new StandardTokenizer(Version.LUCENE_31, reader);
+    result = new StandardFilter(Version.LUCENE_31, result);
+    result = new LowerCaseFilter(Version.LUCENE_31, result);
     result = new ASCIIFoldingFilter(result);
     result = new AlphaNumericMaxLengthFilter(result);
-    result = new StopFilter(false, result, stopSet);
+    result = new StopFilter(Version.LUCENE_31, result, stopSet);
     return new PorterStemFilter(result);
   }
 
@@ -132,13 +133,13 @@ public class MailArchivesClusteringAnaly
    * Matches alpha-numeric tokens between 2 and 40 chars long.
    */
   static class AlphaNumericMaxLengthFilter extends TokenFilter {
-    private final TermAttribute termAtt;
+    private final CharTermAttribute termAtt;
     private final char[] output = new char[28];
     private final Matcher matcher;
 
     AlphaNumericMaxLengthFilter(TokenStream in) {
       super(in);
-      termAtt = addAttribute(TermAttribute.class);
+      termAtt = addAttribute(CharTermAttribute.class);
       matcher = alphaNumeric.matcher("foo");
     }
 
@@ -146,9 +147,9 @@ public class MailArchivesClusteringAnaly
     public final boolean incrementToken() throws IOException {
       // return the first alpha-numeric token between 2 and 40 length
       while (input.incrementToken()) {
-        int length = termAtt.termLength();
+        int length = termAtt.length();
         if (length >= 2 && length <= 28) {
-          char[] buf = termAtt.termBuffer();
+          char[] buf = termAtt.buffer();
           int at = 0;
           for (int c=0; c < length; c++) {
             char ch = buf[c];
@@ -159,7 +160,8 @@ public class MailArchivesClusteringAnaly
           String term = new String(output, 0, at);
           matcher.reset(term);
           if (matcher.matches() && !term.startsWith("a0")) {
-            termAtt.setTermBuffer(term);
+            termAtt.setEmpty();
+            termAtt.append(term);
             return true;
           }
         }

Modified: mahout/trunk/utils/src/test/java/org/apache/mahout/text/MailArchivesClusteringAnalyzerTest.java
URL: http://svn.apache.org/viewvc/mahout/trunk/utils/src/test/java/org/apache/mahout/text/MailArchivesClusteringAnalyzerTest.java?rev=1099894&r1=1099893&r2=1099894&view=diff
==============================================================================
--- mahout/trunk/utils/src/test/java/org/apache/mahout/text/MailArchivesClusteringAnalyzerTest.java (original)
+++ mahout/trunk/utils/src/test/java/org/apache/mahout/text/MailArchivesClusteringAnalyzerTest.java Thu May  5 17:55:22 2011
@@ -18,8 +18,9 @@ package org.apache.mahout.text;
 
 import java.io.StringReader;
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
 
+
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.mahout.utils.MahoutTestCase;
 import org.junit.Test;
 
@@ -45,15 +46,16 @@ public class MailArchivesClusteringAnaly
     // order matters too
     String[] expectedTokens = {
         "test", "mahout", "scalabl", "machin", "learn", "librari", "weve", "ad",
-        "stopword", "apach", "hadoop", "provid", "foundat", "scalabl"
+        "stopword", "apache_hadoop","provid", "foundat", "scalabl"
     };
         
     TokenStream tokenStream = analyzer.tokenStream("test", reader);
     assertNotNull(tokenStream);    
-    TermAttribute termAtt = tokenStream.addAttribute(TermAttribute.class);
-    int e = -1;
-    while (tokenStream.incrementToken()) {
-      assertEquals(expectedTokens[++e], termAtt.term());
-    } 
+    CharTermAttribute termAtt = tokenStream.addAttribute(CharTermAttribute.class);
+    int e = 0;
+    while (tokenStream.incrementToken() && e < expectedTokens.length) {
+      assertEquals(expectedTokens[e++], termAtt.toString());
+    }
+    assertEquals(e, expectedTokens.length);
   }
 }