You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by us...@apache.org on 2009/07/04 22:08:56 UTC

svn commit: r791173 [1/4] - in /lucene/java/branches/lucene_2_4_back_compat_tests: ./ contrib/highlighter/src/test/org/apache/lucene/search/highlight/ src/java/org/apache/lucene/analysis/ src/java/org/apache/lucene/analysis/standard/ src/java/org/apach...

Author: uschindler
Date: Sat Jul  4 20:08:54 2009
New Revision: 791173

URL: http://svn.apache.org/viewvc?rev=791173&view=rev
Log:
Reverted some of the early patches in the backwards-compatibility branch (which was branched from trunk in November, already containing things like the new TokenStream API):
I was going through the changelog between revisions 696710 (status of trunk, when Lucene 2.4 was branched) and 721664 (status of trunk when backwards branch was created). I then used the merge tool to revert certain "big" patches:
- new TokenStream API
- MultiTermQuery extensions for Range, Prefix,*
- FieldCacheRangeQuery
The backwards branch is now more close to what is real the API of 2.4. For more info, see discussion on java-dev

Added:
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/CharStream.java
      - copied unchanged from r790886, lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/search/CharStream.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/search/TestConstantScoreRangeQuery.java
      - copied unchanged from r712889, lucene/java/trunk/src/test/org/apache/lucene/search/TestConstantScoreRangeQuery.java
Removed:
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/tokenattributes/
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/search/CharStream.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/search/PrefixTermEnum.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/search/RangeTermEnum.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/util/Attribute.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/util/AttributeSource.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/search/TestFieldCacheRangeFilter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/search/TestMultiTermConstantScore.java
Modified:
    lucene/java/branches/lucene_2_4_back_compat_tests/CHANGES.txt
    lucene/java/branches/lucene_2_4_back_compat_tests/contrib/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/CachingTokenFilter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/CharTokenizer.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/ISOLatin1AccentFilter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/KeywordTokenizer.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/LengthFilter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/LowerCaseFilter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/PorterStemFilter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/SinkTokenizer.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/StopFilter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/TeeTokenFilter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/Token.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/TokenFilter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/TokenStream.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/Tokenizer.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/package.html
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/standard/StandardFilter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/standard/StandardTokenizer.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/standard/StandardTokenizerImpl.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/standard/StandardTokenizerImpl.jflex
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/index/DocInverter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/index/DocInverterPerField.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/index/DocInverterPerThread.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/index/FieldInvertState.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/index/InvertedDocConsumerPerField.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/index/Payload.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/index/TermsHashConsumerPerField.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/index/TermsHashPerField.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/queryParser/CharStream.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/queryParser/ParseException.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/queryParser/QueryParser.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/queryParser/QueryParser.jj
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/queryParser/QueryParserTokenManager.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/queryParser/Token.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/queryParser/TokenMgrError.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/search/ConstantScoreRangeQuery.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/search/FieldCache.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/search/FuzzyQuery.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/search/MultiTermQuery.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/search/PrefixFilter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/search/PrefixQuery.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/search/QueryTermVector.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/search/RangeFilter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/search/RangeQuery.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/search/WildcardQuery.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/search/WildcardTermEnum.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/analysis/TeeSinkTokenTest.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/analysis/TestAnalyzers.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/analysis/TestCachingTokenFilter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/analysis/TestISOLatin1AccentFilter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/analysis/TestKeywordAnalyzer.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/analysis/TestLengthFilter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/analysis/TestPerFieldAnalzyerWrapper.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/analysis/TestStandardAnalyzer.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/analysis/TestStopAnalyzer.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/analysis/TestStopFilter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/analysis/TestToken.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/index/TestDocumentWriter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/index/TestIndexWriter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/index/TestPayloads.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/index/TestTermVectorsReader.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/index/TestTermdocPerf.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/queryParser/TestMultiAnalyzer.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/queryParser/TestMultiFieldQueryParser.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/queryParser/TestQueryParser.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/search/TestPositionIncrement.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/search/TestRangeFilter.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/search/TestRangeQuery.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/search/payloads/PayloadHelper.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/search/payloads/TestBoostingTermQuery.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/search/spans/TestPayloadSpans.java
    lucene/java/branches/lucene_2_4_back_compat_tests/src/test/org/apache/lucene/util/LuceneTestCase.java

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/CHANGES.txt?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/CHANGES.txt (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/CHANGES.txt Sat Jul  4 20:08:54 2009
@@ -5,12 +5,6 @@
 
 Changes in runtime behavior
 
- 1. LUCENE-1424: QueryParser now by default uses constant score query
-    rewriting when it generates a WildcardQuery and PrefixQuery (it
-    already does so for RangeQuery, as well).  Call
-    setConstantScoreRewrite(false) to revert to BooleanQuery rewriting
-    method.  (Mark Miller via Mike McCandless)
-
 API Changes
 
 1. LUCENE-1419: Add expert API to set custom indexing chain. This API is 
@@ -25,15 +19,6 @@
    and deprecate FSDirectory.getDirectory().  FSDirectory instances
    are not required to be singletons per path. (yonik)
 
-4. LUCENE-1422: New TokenStream API that uses a new class called 
-   AttributeSource instead of the now deprecated Token class. All attributes
-   that the Token class had have been moved into separate classes:
-   TermAttribute, OffsetAttribute, PositionIncrementAttribute, 
-   PayloadAttribute, TypeAttribute and FlagsAttribute. The new API
-   is much more flexible; it allows to combine the Attributes arbitrarily 
-   and also to define custom Attributes. The new API has the same performance
-   as the old next(Token) approach. (Michael Busch)
-
 5. LUCENE-1467: Add nextDoc() and next(int) methods to OpenBitSetIterator.
    These methods can be used to avoid additional calls to doc(). 
    (Michael Busch)
@@ -89,22 +74,6 @@
  5. Added web-based demo of functionality in contrib's XML Query Parser
     packaged as War file (Mark Harwood)    
 
- 6. LUCENE-1424: Moved constant score query rewrite capability into
-    MultiTermQuery, allowing RangeQuery, PrefixQuery and WildcardQuery
-    to switch betwen constant-score rewriting or BooleanQuery
-    expansion rewriting via a new setConstantScoreRewrite method.
-    Deprecated ConstantScoreRangeQuery (Mark Miller via Mike
-    McCandless)
-
- 7. LUCENE-1461: Added FieldCacheRangeFilter, a RangeFilter for
-    single-term fields that uses FieldCache to compute the filter.  If
-    your field has a single term per document, and you need to create
-    many RangeFilters with varying lower/upper bounds, then this is
-    likely a much faster way to create the filters than RangeFilter.
-    However, it comes at the expense of added RAM consumption and
-    slower first-time usage due to populating the FieldCache.  (Tim
-    Sturge via Mike McCandless)
-
 Optimizations
 
  1. LUCENE-1427: Fixed QueryWrapperFilter to not waste time computing

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/contrib/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/contrib/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/contrib/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/contrib/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java Sat Jul  4 20:08:54 2009
@@ -44,6 +44,7 @@
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Field.Index;
 import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.Term;
@@ -51,7 +52,6 @@
 import org.apache.lucene.queryParser.ParseException;
 import org.apache.lucene.queryParser.QueryParser;
 import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.ConstantScoreRangeQuery;
 import org.apache.lucene.search.FilteredQuery;
 import org.apache.lucene.search.Hits;
 import org.apache.lucene.search.IndexSearcher;
@@ -502,8 +502,14 @@
   public void testGetConstantScoreRangeFragments() throws Exception {
 
     numHighlights = 0;
+    String queryString = FIELD_NAME + ":[kannedy TO kznnedy]";
 
-    query = new ConstantScoreRangeQuery(FIELD_NAME, "kannedy", "kznnedy", true, true);
+    // Need to explicitly set the QueryParser property to use RangeQuery
+    // rather
+    // than RangeFilters
+    QueryParser parser = new QueryParser(FIELD_NAME, new StandardAnalyzer());
+    // parser.setUseOldRangeQuery(true);
+    query = parser.parse(queryString);
 
     searcher = new IndexSearcher(ramDir);
     // can't rewrite ConstantScoreRangeQuery if you want to highlight it -
@@ -1077,7 +1083,6 @@
     searchers[1] = new IndexSearcher(ramDir2);
     MultiSearcher multiSearcher = new MultiSearcher(searchers);
     QueryParser parser = new QueryParser(FIELD_NAME, new StandardAnalyzer());
-    parser.setConstantScoreRewrite(false);
     query = parser.parse("multi*");
     System.out.println("Searching for: " + query.toString(FIELD_NAME));
     // at this point the multisearcher calls combine(query[])
@@ -1378,7 +1383,6 @@
 
   public void doSearching(String queryString) throws Exception {
     QueryParser parser = new QueryParser(FIELD_NAME, new StandardAnalyzer());
-    parser.setConstantScoreRewrite(false);
     query = parser.parse(queryString);
     doSearching(query);
   }

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/CachingTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/CachingTokenFilter.java?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/CachingTokenFilter.java (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/CachingTokenFilter.java Sat Jul  4 20:08:54 2009
@@ -22,8 +22,6 @@
 import java.util.LinkedList;
 import java.util.List;
 
-import org.apache.lucene.util.AttributeSource;
-
 /**
  * This class can be used if the Tokens of a TokenStream
  * are intended to be consumed more than once. It caches
@@ -36,31 +34,12 @@
  */
 public class CachingTokenFilter extends TokenFilter {
   private List cache;
-  private Iterator iterator; 
+  private Iterator iterator;
   
   public CachingTokenFilter(TokenStream input) {
     super(input);
   }
   
-  public boolean incrementToken() throws IOException {
-    if (cache == null) {
-      // fill cache lazily
-      cache = new LinkedList();
-      fillCache();
-      iterator = cache.iterator();
-    }
-    
-    if (!iterator.hasNext()) {
-      // the cache is exhausted, return null
-      return false;
-    }
-    // Since the TokenFilter can be reset, the tokens need to be preserved as immutable.
-    AttributeSource state = (AttributeSource) iterator.next();
-    state.restoreState(this);
-    return true;
-  }
-  
-  /** @deprecated */
   public Token next(final Token reusableToken) throws IOException {
     assert reusableToken != null;
     if (cache == null) {
@@ -81,17 +60,10 @@
   
   public void reset() throws IOException {
     if(cache != null) {
-      iterator = cache.iterator();
-    }
-  }
-  
-  private void fillCache() throws IOException {
-    while(input.incrementToken()) {
-      cache.add(captureState());
+    	iterator = cache.iterator();
     }
   }
   
-  /** @deprecated */
   private void fillCache(final Token reusableToken) throws IOException {
     for (Token nextToken = input.next(reusableToken); nextToken != null; nextToken = input.next(reusableToken)) {
       cache.add(nextToken.clone());

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/CharTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/CharTokenizer.java?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/CharTokenizer.java (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/CharTokenizer.java Sat Jul  4 20:08:54 2009
@@ -20,24 +20,16 @@
 import java.io.IOException;
 import java.io.Reader;
 
-import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
-
 /** An abstract base class for simple, character-oriented tokenizers.*/
 public abstract class CharTokenizer extends Tokenizer {
   public CharTokenizer(Reader input) {
     super(input);
-    offsetAtt = (OffsetAttribute) addAttribute(OffsetAttribute.class);
-    termAtt = (TermAttribute) addAttribute(TermAttribute.class);
   }
 
   private int offset = 0, bufferIndex = 0, dataLen = 0;
   private static final int MAX_WORD_LEN = 255;
   private static final int IO_BUFFER_SIZE = 4096;
   private final char[] ioBuffer = new char[IO_BUFFER_SIZE];
-  
-  private TermAttribute termAtt;
-  private OffsetAttribute offsetAtt;
 
   /** Returns true iff a character should be included in a token.  This
    * tokenizer generates as tokens adjacent sequences of characters which
@@ -52,50 +44,6 @@
     return c;
   }
 
-  public final boolean incrementToken() throws IOException {
-    clearAttributes();
-    int length = 0;
-    int start = bufferIndex;
-    char[] buffer = termAtt.termBuffer();
-    while (true) {
-
-      if (bufferIndex >= dataLen) {
-        offset += dataLen;
-        dataLen = input.read(ioBuffer);
-        if (dataLen == -1) {
-          if (length > 0)
-            break;
-          else
-            return false;
-        }
-        bufferIndex = 0;
-      }
-
-      final char c = ioBuffer[bufferIndex++];
-
-      if (isTokenChar(c)) {               // if it's a token char
-
-        if (length == 0)                 // start of token
-          start = offset + bufferIndex - 1;
-        else if (length == buffer.length)
-          buffer = termAtt.resizeTermBuffer(1+length);
-
-        buffer[length++] = normalize(c); // buffer it, normalized
-
-        if (length == MAX_WORD_LEN)      // buffer overflow!
-          break;
-
-      } else if (length > 0)             // at non-Letter w/ chars
-        break;                           // return 'em
-    }
-
-    termAtt.setTermLength(length);
-    offsetAtt.setStartOffset(start);
-    offsetAtt.setEndOffset(start+length);
-    return true;
-  }
-
-  /** @deprecated */
   public final Token next(final Token reusableToken) throws IOException {
     assert reusableToken != null;
     reusableToken.clear();

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/ISOLatin1AccentFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/ISOLatin1AccentFilter.java?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/ISOLatin1AccentFilter.java (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/ISOLatin1AccentFilter.java Sat Jul  4 20:08:54 2009
@@ -1,7 +1,5 @@
 package org.apache.lucene.analysis;
 
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
-
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -29,33 +27,11 @@
 public class ISOLatin1AccentFilter extends TokenFilter {
   public ISOLatin1AccentFilter(TokenStream input) {
     super(input);
-    termAtt = (TermAttribute) addAttribute(TermAttribute.class);
   }
 
   private char[] output = new char[256];
   private int outputPos;
-  private TermAttribute termAtt;
-    
-  public final boolean incrementToken() throws java.io.IOException {    
-    if (input.incrementToken()) {
-      final char[] buffer = termAtt.termBuffer();
-      final int length = termAtt.termLength();
-      // If no characters actually require rewriting then we
-      // just return token as-is:
-      for(int i=0;i<length;i++) {
-        final char c = buffer[i];
-        if (c >= '\u00c0' && c <= '\uFB06') {
-          removeAccents(buffer, length);
-          termAtt.setTermBuffer(output, 0, outputPos);
-          break;
-        }
-      }
-      return true;
-    } else
-      return false;
-  }
-  
-  /** @deprecated */
+
   public final Token next(final Token reusableToken) throws java.io.IOException {
     assert reusableToken != null;
     Token nextToken = input.next(reusableToken);
@@ -265,7 +241,7 @@
         case '\uFB06': // st
             output[outputPos++] = 's';
             output[outputPos++] = 't';
-          break;
+        	break;
         default :
           output[outputPos++] = c;
           break;

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/KeywordTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/KeywordTokenizer.java?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/KeywordTokenizer.java (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/KeywordTokenizer.java Sat Jul  4 20:08:54 2009
@@ -20,9 +20,6 @@
 import java.io.IOException;
 import java.io.Reader;
 
-import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
-
 /**
  * Emits the entire input as a single token.
  */
@@ -31,9 +28,7 @@
   private static final int DEFAULT_BUFFER_SIZE = 256;
 
   private boolean done;
-  private TermAttribute termAtt;
-  private OffsetAttribute offsetAtt;
-  
+
   public KeywordTokenizer(Reader input) {
     this(input, DEFAULT_BUFFER_SIZE);
   }
@@ -41,32 +36,8 @@
   public KeywordTokenizer(Reader input, int bufferSize) {
     super(input);
     this.done = false;
-    termAtt = (TermAttribute) addAttribute(TermAttribute.class);
-    offsetAtt = (OffsetAttribute) addAttribute(OffsetAttribute.class);
-  }
-  
-  public boolean incrementToken() throws IOException {
-    if (!done) {
-      done = true;
-      int upto = 0;
-      termAtt.clear();
-      char[] buffer = termAtt.termBuffer();
-      while (true) {
-        final int length = input.read(buffer, upto, buffer.length-upto);
-        if (length == -1) break;
-        upto += length;
-        if (upto == buffer.length)
-          buffer = termAtt.resizeTermBuffer(1+buffer.length);
-      }
-      termAtt.setTermLength(upto);
-      offsetAtt.setStartOffset(0);
-      offsetAtt.setEndOffset(upto);
-      return true;
-    }
-    return false;
   }
 
-  /** @deprecated */
   public Token next(final Token reusableToken) throws IOException {
     assert reusableToken != null;
     if (!done) {

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/LengthFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/LengthFilter.java?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/LengthFilter.java (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/LengthFilter.java Sat Jul  4 20:08:54 2009
@@ -19,8 +19,6 @@
 
 import java.io.IOException;
 
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
-
 /**
  * Removes words that are too long and too short from the stream.
  *
@@ -31,8 +29,6 @@
 
   final int min;
   final int max;
-  
-  private TermAttribute termAtt;
 
   /**
    * Build a filter that removes words that are too long or too
@@ -43,28 +39,10 @@
     super(in);
     this.min = min;
     this.max = max;
-    termAtt = (TermAttribute) addAttribute(TermAttribute.class);
-  }
-  
-  /**
-   * Returns the next input Token whose term() is the right len
-   */
-  public final boolean incrementToken() throws IOException {
-    // return the first non-stop word found
-    while (input.incrementToken()) {
-      int len = termAtt.termLength();
-      if (len >= min && len <= max) {
-          return true;
-      }
-      // note: else we ignore it but should we index each part of it?
-    }
-    // reached EOS -- return null
-    return false;
   }
 
   /**
    * Returns the next input Token whose term() is the right len
-   * @deprecated
    */
   public final Token next(final Token reusableToken) throws IOException
   {

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/LowerCaseFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/LowerCaseFilter.java?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/LowerCaseFilter.java (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/LowerCaseFilter.java Sat Jul  4 20:08:54 2009
@@ -19,8 +19,6 @@
 
 import java.io.IOException;
 
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
-
 /**
  * Normalizes token text to lower case.
  *
@@ -29,25 +27,8 @@
 public final class LowerCaseFilter extends TokenFilter {
   public LowerCaseFilter(TokenStream in) {
     super(in);
-    termAtt = (TermAttribute) addAttribute(TermAttribute.class);
   }
 
-  private TermAttribute termAtt;
-  
-  public final boolean incrementToken() throws IOException {
-    if (input.incrementToken()) {
-
-      final char[] buffer = termAtt.termBuffer();
-      final int length = termAtt.termLength();
-      for(int i=0;i<length;i++)
-        buffer[i] = Character.toLowerCase(buffer[i]);
-
-      return true;
-    } else
-      return false;
-  }
-  
-  /** @deprecated */
   public final Token next(final Token reusableToken) throws IOException {
     assert reusableToken != null;
     Token nextToken = input.next(reusableToken);

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/PorterStemFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/PorterStemFilter.java?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/PorterStemFilter.java (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/PorterStemFilter.java Sat Jul  4 20:08:54 2009
@@ -19,8 +19,6 @@
 
 import java.io.IOException;
 
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
-
 /** Transforms the token stream as per the Porter stemming algorithm.
     Note: the input to the stemming filter must already be in lower case,
     so you will need to use LowerCaseFilter or LowerCaseTokenizer farther
@@ -41,24 +39,12 @@
 */
 public final class PorterStemFilter extends TokenFilter {
   private PorterStemmer stemmer;
-  private TermAttribute termAtt;
 
   public PorterStemFilter(TokenStream in) {
     super(in);
     stemmer = new PorterStemmer();
-    termAtt = (TermAttribute) addAttribute(TermAttribute.class);
   }
 
-  public final boolean incrementToken() throws IOException {
-    if (!input.incrementToken())
-      return false;
-
-    if (stemmer.stem(termAtt.termBuffer(), 0, termAtt.termLength()))
-      termAtt.setTermBuffer(stemmer.getResultBuffer(), 0, stemmer.getResultLength());
-    return true;
-  }
-  
-  /** @deprecated */
   public final Token next(final Token reusableToken) throws IOException {
     assert reusableToken != null;
     Token nextToken = input.next(reusableToken);

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/SinkTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/SinkTokenizer.java?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/SinkTokenizer.java (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/SinkTokenizer.java Sat Jul  4 20:08:54 2009
@@ -22,8 +22,6 @@
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.lucene.util.AttributeSource;
-
 
 /**
  * A SinkTokenizer can be used to cache Tokens for use in an Analyzer
@@ -34,7 +32,7 @@
 public class SinkTokenizer extends Tokenizer {
   protected List/*<Token>*/ lst = new ArrayList/*<Token>*/();
   protected Iterator/*<Token>*/ iter;
-  
+
   public SinkTokenizer(List/*<Token>*/ input) {
     this.lst = input;
     if (this.lst == null) this.lst = new ArrayList/*<Token>*/();
@@ -64,29 +62,9 @@
   }
 
   /**
-   * Increments this stream to the next token out of the list of cached tokens
-   * @throws IOException
-   */
-  public boolean incrementToken() throws IOException {
-    if (iter == null) iter = lst.iterator();
-    // Since this TokenStream can be reset we have to maintain the tokens as immutable
-    if (iter.hasNext()) {
-      AttributeSource state = (AttributeSource) iter.next();
-      state.restoreState(this);
-      return true;
-    }
-    return false;
-  }
-
-  public void add(AttributeSource source) throws IOException {
-    lst.add(source); 
-  }
-  
-  /**
    * Returns the next token out of the list of cached tokens
    * @return The next {@link org.apache.lucene.analysis.Token} in the Sink.
    * @throws IOException
-   * @deprecated
    */
   public Token next(final Token reusableToken) throws IOException {
     assert reusableToken != null;
@@ -99,6 +77,8 @@
     return null;
   }
 
+
+
   /**
    * Override this method to cache only certain tokens, or new tokens based
    * on the old tokens.

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/StopFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/StopFilter.java?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/StopFilter.java (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/StopFilter.java Sat Jul  4 20:08:54 2009
@@ -21,9 +21,6 @@
 import java.util.Arrays;
 import java.util.Set;
 
-import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
-
 /**
  * Removes stop words from a token stream.
  */
@@ -35,9 +32,6 @@
   private final CharArraySet stopWords;
   private boolean enablePositionIncrements = ENABLE_POSITION_INCREMENTS_DEFAULT;
 
-  private TermAttribute termAtt;
-  private PositionIncrementAttribute posIncrAtt;
-  
   /**
    * Construct a token stream filtering the given input.
    */
@@ -53,7 +47,6 @@
   public StopFilter(TokenStream in, String[] stopWords, boolean ignoreCase) {
     super(in);
     this.stopWords = (CharArraySet)makeStopSet(stopWords, ignoreCase);
-    init();
   }
 
 
@@ -81,7 +74,6 @@
       this.stopWords = new CharArraySet(stopWords.size(), ignoreCase);
       this.stopWords.addAll(stopWords);
     }
-    init();
   }
 
   /**
@@ -93,11 +85,6 @@
   public StopFilter(TokenStream in, Set stopWords) {
     this(in, stopWords, false);
   }
-  
-  public void init() {
-    termAtt = (TermAttribute) addAttribute(TermAttribute.class);
-    posIncrAtt = (PositionIncrementAttribute) addAttribute(PositionIncrementAttribute.class);
-  }
 
   /**
    * Builds a Set from an array of stop words,
@@ -122,29 +109,9 @@
     stopSet.addAll(Arrays.asList(stopWords));
     return stopSet;
   }
-  
-  /**
-   * Returns the next input Token whose term() is not a stop word.
-   */
-  public final boolean incrementToken() throws IOException {
-    // return the first non-stop word found
-    int skippedPositions = 0;
-    while (input.incrementToken()) {
-      if (!stopWords.contains(termAtt.termBuffer(), 0, termAtt.termLength())) {
-        if (enablePositionIncrements) {
-          posIncrAtt.setPositionIncrement(posIncrAtt.getPositionIncrement() + skippedPositions);
-        }
-        return true;
-      }
-      skippedPositions += posIncrAtt.getPositionIncrement();
-    }
-    // reached EOS -- return null
-    return false;
-  }
 
   /**
    * Returns the next input Token whose term() is not a stop word.
-   * @deprecated
    */
   public final Token next(final Token reusableToken) throws IOException {
     assert reusableToken != null;

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/TeeTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/TeeTokenFilter.java?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/TeeTokenFilter.java (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/TeeTokenFilter.java Sat Jul  4 20:08:54 2009
@@ -18,7 +18,6 @@
 package org.apache.lucene.analysis;
 
 import java.io.IOException;
-import java.util.Iterator;
 
 
 /**
@@ -61,21 +60,8 @@
   public TeeTokenFilter(TokenStream input, SinkTokenizer sink) {
     super(input);
     this.sink = sink;
-    Iterator it = getAttributesIterator();
-    while (it.hasNext()) {
-      sink.addAttribute(it.next().getClass());
-    }
-  }
-  
-  public boolean incrementToken() throws IOException {
-    if (input.incrementToken()) {
-      sink.add(captureState());
-      return true;
-    }
-    return false;
   }
 
-  /** @deprecated */
   public Token next(final Token reusableToken) throws IOException {
     assert reusableToken != null;
     Token nextToken = input.next(reusableToken);

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/Token.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/Token.java?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/Token.java (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/Token.java Sat Jul  4 20:08:54 2009
@@ -21,11 +21,7 @@
 import org.apache.lucene.index.TermPositions;     // for javadoc
 import org.apache.lucene.util.ArrayUtil;
 
-/** 
-  This class is now deprecated and a new TokenStream API was introduced with Lucene 2.9.
-  See Javadocs in {@link TokenStream} for further details.
-  <p> 
-  A Token is an occurrence of a term from the text of a field.  It consists of
+/** A Token is an occurrence of a term from the text of a field.  It consists of
   a term's text, the start and end offset of the term in the text of the field,
   and a type string.
   <p>
@@ -118,8 +114,6 @@
   </p>
 
   @see org.apache.lucene.index.Payload
-  @deprecated A new TokenStream API was introduced with Lucene 2.9.
-              See javadocs in {@link TokenStream} for further details.
 */
 public class Token implements Cloneable {
 

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/TokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/TokenFilter.java?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/TokenFilter.java (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/TokenFilter.java Sat Jul  4 20:08:54 2009
@@ -22,16 +22,9 @@
 /** A TokenFilter is a TokenStream whose input is another token stream.
   <p>
   This is an abstract class.
-  NOTE: subclasses must override 
-  {@link #incrementToken()} if the new TokenStream API is used
-  and {@link #next(Token)} or {@link #next()} if the old
-  TokenStream API is used.
- * <p><font color="#FF0000">
- * WARNING: The status of the new TokenStream, AttributeSource and Attributes is experimental. 
- * The APIs introduced in these classes with Lucene 2.9 might change in the future. 
- * We will make our best efforts to keep the APIs backwards-compatible.</font>
-  <p>
-  See {@link TokenStream}
+  NOTE: subclasses must override {@link #next(Token)}.  It's
+  also OK to instead override {@link #next()} but that
+  method is now deprecated in favor of {@link #next(Token)}.
   */
 public abstract class TokenFilter extends TokenStream {
   /** The source of tokens for this filter. */
@@ -39,10 +32,9 @@
 
   /** Construct a token stream filtering the given input. */
   protected TokenFilter(TokenStream input) {
-    super(input);
     this.input = input;
   }
-    
+
   /** Close the input TokenStream. */
   public void close() throws IOException {
     input.close();
@@ -53,17 +45,4 @@
     super.reset();
     input.reset();
   }
-  
-  public boolean useNewAPI() {
-    return input.useNewAPI();
-  }
-
-  /**
-   * Sets whether or not to use the new TokenStream API. Settings this
-   * will apply to this Filter and all TokenStream/Filters upstream.
-   */
-  public void setUseNewAPI(boolean use) {
-    input.setUseNewAPI(use);
-  }
-
 }

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/TokenStream.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/TokenStream.java?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/TokenStream.java (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/TokenStream.java Sat Jul  4 20:08:54 2009
@@ -17,12 +17,9 @@
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.Iterator;
-
 import org.apache.lucene.index.Payload;
-import org.apache.lucene.util.Attribute;
-import org.apache.lucene.util.AttributeSource;
+
+import java.io.IOException;
 
 /** A TokenStream enumerates the sequence of tokens, either from
   fields of a document or from query text.
@@ -34,140 +31,13 @@
   <li>{@link TokenFilter}, a TokenStream
   whose input is another TokenStream.
   </ul>
-  A new TokenStream API is introduced with Lucene 2.9. Since
-  2.9 Token is deprecated and the preferred way to store
-  the information of a token is to use {@link Attribute}s.
-  <p>
-  For that reason TokenStream extends {@link AttributeSource}
-  now. Note that only one instance per {@link Attribute} is
-  created and reused for every token. This approach reduces
-  object creations and allows local caching of references to
-  the {@link Attribute}s. See {@link #incrementToken()} for further details.
-  <p>
-  <b>The workflow of the new TokenStream API is as follows:</b>
-  <ol>
-    <li>Instantiation of TokenStream/TokenFilters which add/get attributes
-        to/from the {@link AttributeSource}. 
-    <li>The consumer calls {@link TokenStream#reset()}.
-    <li>the consumer retrieves attributes from the
-        stream and stores local references to all attributes it wants to access
-    <li>The consumer calls {@link #incrementToken()} until it returns false and
-        consumes the attributes after each call.    
-  </ol>
-  To make sure that filters and consumers know which attributes are available
-  the attributes must be added in the during instantiation. Filters and 
-  consumers are not required to check for availability of attributes in {@link #incrementToken()}.
-  <p>
-  Sometimes it is desirable to capture a current state of a
-  TokenStream, e. g. for buffering purposes (see {@link CachingTokenFilter},
-  {@link TeeTokenFilter}/{@link SinkTokenizer}). For this usecase
-  {@link AttributeSource#captureState()} and {@link AttributeSource#restoreState(AttributeSource)} can be used.  
-  <p>
-  <b>NOTE:</b> In order to enable the new API the method
-  {@link #useNewAPI()} has to be called with useNewAPI=true.
-  Otherwise the deprecated method {@link #next(Token)} will 
-  be used by Lucene consumers (indexer and queryparser) to
-  consume the tokens. {@link #next(Token)} will be removed
-  in Lucene 3.0.
-  <p>
-  NOTE: To use the old API subclasses must override {@link #next(Token)}.
-  It's also OK to instead override {@link #next()} but that
-  method is slower compared to {@link #next(Token)}.
- * <p><font color="#FF0000">
- * WARNING: The status of the new TokenStream, AttributeSource and Attributes is experimental. 
- * The APIs introduced in these classes with Lucene 2.9 might change in the future. 
- * We will make our best efforts to keep the APIs backwards-compatible.</font>
+  NOTE: subclasses must override {@link #next(Token)}.  It's
+  also OK to instead override {@link #next()} but that
+  method is now deprecated in favor of {@link #next(Token)}.
   */
 
-public abstract class TokenStream extends AttributeSource {
-  private static boolean useNewAPIDefault = false;
-  private boolean useNewAPI = useNewAPIDefault;
-  
-  protected TokenStream() {
-    super();
-  }
-  
-  protected TokenStream(AttributeSource input) {
-    super(input);
-  }
-
-  /**
-   * Returns whether or not the new TokenStream APIs are used
-   * by default. 
-   * (see {@link #incrementToken()}, {@link AttributeSource}).
-   */
-  public static boolean useNewAPIDefault() {
-    return useNewAPIDefault;
-  }
-
-  /**
-   * Use this API to enable or disable the new TokenStream API.
-   * by default. Can be overridden by calling {@link #setUseNewAPI(boolean)}. 
-   * (see {@link #incrementToken()}, {@link AttributeSource}).
-   * <p>
-   * If set to true, the indexer will call {@link #incrementToken()} 
-   * to consume Tokens from this stream.
-   * <p>
-   * If set to false, the indexer will call {@link #next(Token)}
-   * instead. 
-   */
-  public static void setUseNewAPIDefault(boolean use) {
-    useNewAPIDefault = use;
-  }
-  
-  /**
-   * Returns whether or not the new TokenStream APIs are used 
-   * for this stream.
-   * (see {@link #incrementToken()}, {@link AttributeSource}).
-   */
-  public boolean useNewAPI() {
-    return useNewAPI;
-  }
+public abstract class TokenStream {
 
-  /**
-   * Use this API to enable or disable the new TokenStream API
-   * for this stream. Overrides {@link #setUseNewAPIDefault(boolean)}.
-   * (see {@link #incrementToken()}, {@link AttributeSource}).
-   * <p>
-   * If set to true, the indexer will call {@link #incrementToken()} 
-   * to consume Tokens from this stream.
-   * <p>
-   * If set to false, the indexer will call {@link #next(Token)}
-   * instead. 
-   * <p>
-   * <b>NOTE: All streams and filters in one chain must use the
-   * same API. </b>
-   */
-  public void setUseNewAPI(boolean use) {
-    useNewAPI = use;
-  }
-    	
-	/**
-	 * Consumers (e. g. the indexer) use this method to advance the stream 
-	 * to the next token. Implementing classes must implement this method 
-	 * and update the appropriate {@link Attribute}s with content of the 
-	 * next token.
-	 * <p>
-	 * This method is called for every token of a document, so an efficient
-	 * implementation is crucial for good performance. To avoid calls to 
-	 * {@link #addAttribute(Class)} and {@link #getAttribute(Class)} and
-	 * downcasts, references to all {@link Attribute}s that this stream uses 
-	 * should be retrieved during instantiation.   
-	 * <p>
-	 * To make sure that filters and consumers know which attributes are available
-   * the attributes must be added during instantiation. Filters and 
-   * consumers are not required to check for availability of attributes in {@link #incrementToken()}.
-	 * 
-	 * @return false for end of stream; true otherwise
-	 *
-	 * <p>
-	 * <b>Note that this method will be defined abstract in Lucene 3.0.<b>
-	 */
-	public boolean incrementToken() throws IOException {
-	  // subclasses must implement this method; will be made abstract in Lucene 3.0
-	  return false;
-	}
-	
   /** Returns the next token in the stream, or null at EOS.
    *  @deprecated The returned Token is a "full private copy" (not
    *  re-used across calls to next()) but will be slower
@@ -214,8 +84,6 @@
    *  is not required to check for null before using it, but it is a
    *  good idea to assert that it is not null.)
    *  @return next token in the stream or null if end-of-stream was hit
-   *  @deprecated The new {@link #incrementToken()} and {@link AttributeSource}
-   *  APIs should be used instead. See also {@link #useNewAPI()}.
    */
   public Token next(final Token reusableToken) throws IOException {
     // We don't actually use inputToken, but still add this assert
@@ -239,25 +107,4 @@
   
   /** Releases resources associated with this stream. */
   public void close() throws IOException {}
-  
-  public String toString() {
-    StringBuffer sb = new StringBuffer();
-    sb.append('(');
-    
-    if (hasAttributes()) {
-      // TODO Java 1.5
-      //Iterator<Attribute> it = attributes.values().iterator();
-      Iterator it = getAttributesIterator();
-      if (it.hasNext()) {
-        sb.append(it.next().toString());
-      }
-      while (it.hasNext()) {
-        sb.append(',');
-        sb.append(it.next().toString());
-      }
-    }
-    sb.append(')');
-    return sb.toString();
-  }
-
 }

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/Tokenizer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/Tokenizer.java?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/Tokenizer.java (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/Tokenizer.java Sat Jul  4 20:08:54 2009
@@ -24,23 +24,12 @@
   <p>
   This is an abstract class.
   <p>
-  <b>NOTE:</b> In order to enable the new API the method
-  {@link #useNewAPI()} has to be called with useNewAPI=true.
-  Otherwise the deprecated method {@link #next(Token)} will 
-  be used by Lucene consumers (indexer and queryparser) to
-  consume the tokens. {@link #next(Token)} will be removed
-  in Lucene 3.0.
+  NOTE: subclasses must override {@link #next(Token)}.  It's
+  also OK to instead override {@link #next()} but that
+  method is now deprecated in favor of {@link #next(Token)}.
   <p>
-  NOTE: To use the old API subclasses must override {@link #next(Token)}.
-  It's also OK to instead override {@link #next()} but that
-  method is slower compared to {@link #next(Token)}.
- <p>
   NOTE: subclasses overriding {@link #next(Token)} must  
   call {@link Token#clear()}.
- * <p><font color="#FF0000">
- * WARNING: The status of the new TokenStream, AttributeSource and Attributes is experimental. 
- * The APIs introduced in these classes with Lucene 2.9 might change in the future. 
- * We will make our best efforts to keep the APIs backwards-compatible.</font>
  */
 
 public abstract class Tokenizer extends TokenStream {

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/package.html
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/package.html?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/package.html (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/package.html Sat Jul  4 20:08:54 2009
@@ -35,7 +35,8 @@
 <h2>Tokenization</h2>
 <p>
 Plain text passed to Lucene for indexing goes through a process generally called tokenization &ndash; namely breaking of the 
-input text into small indexing elements &ndash; tokens.
+input text into small indexing elements &ndash; 
+{@link org.apache.lucene.analysis.Token Tokens}.
 The way input text is broken into tokens very 
 much dictates further capabilities of search upon that text. 
 For instance, sentences beginnings and endings can be identified to provide for more accurate phrase 
@@ -71,13 +72,12 @@
     <li>{@link org.apache.lucene.analysis.Analyzer} &ndash; An Analyzer is responsible for building a {@link org.apache.lucene.analysis.TokenStream} which can be consumed
     by the indexing and searching processes.  See below for more information on implementing your own Analyzer.</li>
     <li>{@link org.apache.lucene.analysis.Tokenizer} &ndash; A Tokenizer is a {@link org.apache.lucene.analysis.TokenStream} and is responsible for breaking
-    up incoming text into tokens. In most cases, an Analyzer will use a Tokenizer as the first step in
+    up incoming text into {@link org.apache.lucene.analysis.Token}s.  In most cases, an Analyzer will use a Tokenizer as the first step in
     the analysis process.</li>
     <li>{@link org.apache.lucene.analysis.TokenFilter} &ndash; A TokenFilter is also a {@link org.apache.lucene.analysis.TokenStream} and is responsible
-    for modifying tokenss that have been created by the Tokenizer.  Common modifications performed by a
+    for modifying {@link org.apache.lucene.analysis.Token}s that have been created by the Tokenizer.  Common modifications performed by a
     TokenFilter are: deletion, stemming, synonym injection, and down casing.  Not all Analyzers require TokenFilters</li>
   </ul>
-  <b>Since Lucene 2.9 the TokenStream API was changed. Please see section "New TokenStream API" below for details.</b>
 </p>
 <h2>Hints, Tips and Traps</h2>
 <p>
@@ -140,8 +140,9 @@
   <PRE>
       Analyzer analyzer = new StandardAnalyzer(); // or any other analyzer
       TokenStream ts = analyzer.tokenStream("myfield",new StringReader("some text goes here"));
-      while (ts.incrementToken()) {
-        System.out.println("token: "+ts));
+      Token t = ts.next();
+      while (t!=null) {
+        System.out.println("token: "+t));
         t = ts.next();
       }
   </PRE>
@@ -178,7 +179,7 @@
 <p>
   The following sections discuss some aspects of implementing your own analyzer.
 </p>
-<h3>Field Section Boundaries</h3>
+<h3>Field Section Boundaries</h2>
 <p>
   When {@link org.apache.lucene.document.Document#add(org.apache.lucene.document.Fieldable) document.add(field)}
   is called multiple times for the same field name, we could say that each such call creates a new 
@@ -207,10 +208,10 @@
       };
   </PRE>
 </p>
-<h3>Token Position Increments</h3>
+<h3>Token Position Increments</h2>
 <p>
    By default, all tokens created by Analyzers and Tokenizers have a 
-   {@link org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute#getPositionIncrement() position increment} of one.
+   {@link org.apache.lucene.analysis.Token#getPositionIncrement() position increment} of one.
    This means that the position stored for that token in the index would be one more than
    that of the previous token.
    Recall that phrase and proximity searches rely on position info.
@@ -226,29 +227,26 @@
    If this behavior does not fit the application needs,
    a modified analyzer can be used, that would increment further the positions of
    tokens following a removed stop word, using
-   {@link org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute#setPositionIncrement(int)}.
+   {@link org.apache.lucene.analysis.Token#setPositionIncrement(int)}.
    This can be done with something like:
    <PRE>
       public TokenStream tokenStream(final String fieldName, Reader reader) {
         final TokenStream ts = someAnalyzer.tokenStream(fieldName, reader);
         TokenStream res = new TokenStream() {
-          TermAttribute termAtt = (TermAttribute) addAttribute(TermAttribute.class);
-          PositionIncrementAttribute posIncrAtt = (PositionIncrementAttribute) addAttribute(PositionIncrementAttribute.class);
-        
-          public boolean incrementToken() throws IOException {
+          public Token next() throws IOException {
             int extraIncrement = 0;
             while (true) {
-              boolean hasNext = ts.incrementToken();
-              if (hasNext) {
-                if (stopWords.contains(termAtt.term())) {
+              Token t = ts.next();
+              if (t!=null) {
+                if (stopWords.contains(t.termText())) {
                   extraIncrement++; // filter this word
                   continue;
                 } 
                 if (extraIncrement>0) {
-                  posIncrAtt.setPositionIncrement(posIncrAtt.getPositionIncrement()+extraIncrement);
+                  t.setPositionIncrement(t.getPositionIncrement()+extraIncrement);
                 }
               }
-              return hasNext;
+              return t;
             }
           }
         };
@@ -270,336 +268,5 @@
          same position as that token, and so would they be seen by phrase and proximity searches.</li>
    </ol>
 </p>
-<h2>New TokenStream API</h2>
-<p>
-	With Lucene 2.9 we introduce a new TokenStream API. The old API used to produce Tokens. A Token
-	has getter and setter methods for different properties like positionIncrement and termText.
-	While this approach was sufficient for the default indexing format, it is not versatile enough for
-	Flexible Indexing, a term which summarizes the effort of making the Lucene indexer pluggable and extensible for custom
-	index formats.
-</p>
-<p>
-A fully customizable indexer means that users will be able to store custom data structures on disk. Therefore an API
-is necessary that can transport custom types of data from the documents to the indexer.
-</p>
-<h3>Attribute and AttributeSource</h3> 
-Lucene 2.9 therefore introduces a new pair of classes called {@link org.apache.lucene.util.Attribute} and
-{@link org.apache.lucene.util.AttributeSource}. An Attribute serves as a
-particular piece of information about a text token. For example, {@link org.apache.lucene.analysis.tokenattributes.TermAttribute}
- contains the term text of a token, and {@link org.apache.lucene.analysis.tokenattributes.OffsetAttribute} contains the start and end character offsets of a token.
-An AttributeSource is a collection of Attributes with a restriction: there may be only one instance of each attribute type. TokenStream now extends AttributeSource, which
-means that one can add Attributes to a TokenStream. Since TokenFilter extends TokenStream, all filters are also
-AttributeSources.
-<p>
-	Lucene now provides six Attributes out of the box, which replace the variables the Token class has:
-	<ul>
-	  <li>{@link org.apache.lucene.analysis.tokenattributes.TermAttribute}<p>The term text of a token.</p></li>
-  	  <li>{@link org.apache.lucene.analysis.tokenattributes.OffsetAttribute}<p>The start and end offset of token in characters.</p></li>
-	  <li>{@link org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute}<p>See above for detailed information about position increment.</p></li>
-	  <li>{@link org.apache.lucene.analysis.tokenattributes.PayloadAttribute}<p>The payload that a Token can optionally have.</p></li>
-	  <li>{@link org.apache.lucene.analysis.tokenattributes.TypeAttribute}<p>The type of the token. Default is 'word'.</p></li>
-	  <li>{@link org.apache.lucene.analysis.tokenattributes.FlagsAttribute}<p>Optional flags a token can have.</p></li>
-	</ul>
-</p>
-<h3>Using the new TokenStream API</h3>
-There are a few important things to know in order to use the new API efficiently which are summarized here. You may want
-to walk through the example below first and come back to this section afterwards.
-<ol><li>
-Please keep in mind that an AttributeSource can only have one instance of a particular Attribute. Furthermore, if 
-a chain of a TokenStream and multiple TokenFilters is used, then all TokenFilters in that chain share the Attributes
-with the TokenStream.
-</li>
-<br>
-<li>
-Attribute instances are reused for all tokens of a document. Thus, a TokenStream/-Filter needs to update
-the appropriate Attribute(s) in incrementToken(). The consumer, commonly the Lucene indexer, consumes the data in the
-Attributes and then calls incrementToken() again until it retuns false, which indicates that the end of the stream
-was reached. This means that in each call of incrementToken() a TokenStream/-Filter can safely overwrite the data in
-the Attribute instances.
-</li>
-<br>
-<li>
-For performance reasons a TokenStream/-Filter should add/get Attributes during instantiation; i.e., create an attribute in the
-constructor and store references to it in an instance variable.  Using an instance variable instead of calling addAttribute()/getAttribute() 
-in incrementToken() will avoid expensive casting and attribute lookups for every token in the document.
-</li>
-<br>
-<li>
-All methods in AttributeSource are idempotent, which means calling them multiple times always yields the same
-result. This is especially important to know for addAttribute(). The method takes the <b>type</b> (<code>Class</code>)
-of an Attribute as an argument and returns an <b>instance</b>. If an Attribute of the same type was previously added, then
-the already existing instance is returned, otherwise a new instance is created and returned. Therefore TokenStreams/-Filters
-can safely call addAttribute() with the same Attribute type multiple times.
-</li></ol>
-<h3>Example</h3>
-In this example we will create a WhiteSpaceTokenizer and use a LengthFilter to suppress all words that only
-have two or less characters. The LengthFilter is part of the Lucene core and its implementation will be explained
-here to illustrate the usage of the new TokenStream API.<br>
-Then we will develop a custom Attribute, a PartOfSpeechAttribute, and add another filter to the chain which
-utilizes the new custom attribute, and call it PartOfSpeechTaggingFilter.
-<h4>Whitespace tokenization</h4>
-<pre>
-public class MyAnalyzer extends Analyzer {
-
-  public TokenStream tokenStream(String fieldName, Reader reader) {
-    TokenStream stream = new WhitespaceTokenizer(reader);
-    return stream;
-  }
-  
-  public static void main(String[] args) throws IOException {
-    // text to tokenize
-    final String text = "This is a demo of the new TokenStream API";
-    
-    MyAnalyzer analyzer = new MyAnalyzer();
-    TokenStream stream = analyzer.tokenStream("field", new StringReader(text));
-    
-    // get the TermAttribute from the TokenStream
-    TermAttribute termAtt = (TermAttribute) stream.getAttribute(TermAttribute.class);
-    
-    // print all tokens until stream is exhausted
-    while (stream.incrementToken()) {
-      System.out.println(termAtt.term());
-    }
-  }
-}
-</pre>
-In this easy example a simple white space tokenization is performed. In main() a loop consumes the stream and
-prints the term text of the tokens by accessing the TermAttribute that the WhitespaceTokenizer provides. 
-Here is the output:
-<pre>
-This
-is
-a
-demo
-of
-the
-new
-TokenStream
-API
-</pre>
-<h4>Adding a LengthFilter</h4>
-We want to suppress all tokens that have 2 or less characters. We can do that easily by adding a LengthFilter 
-to the chain. Only the tokenStream() method in our analyzer needs to be changed:
-<pre>
-  public TokenStream tokenStream(String fieldName, Reader reader) {
-    TokenStream stream = new WhitespaceTokenizer(reader);
-    stream = new LengthFilter(stream, 3, Integer.MAX_VALUE);
-    return stream;
-  }
-</pre>
-Note how now only words with 3 or more characters are contained in the output:
-<pre>
-This
-demo
-the
-new
-TokenStream
-API
-</pre>
-Now let's take a look how the LengthFilter is implemented (it is part of Lucene's core):
-<pre>
-public final class LengthFilter extends TokenFilter {
-
-  final int min;
-  final int max;
-  
-  private TermAttribute termAtt;
-
-  /**
-   * Build a filter that removes words that are too long or too
-   * short from the text.
-   */
-  public LengthFilter(TokenStream in, int min, int max)
-  {
-    super(in);
-    this.min = min;
-    this.max = max;
-    termAtt = (TermAttribute) addAttribute(TermAttribute.class);
-  }
-  
-  /**
-   * Returns the next input Token whose term() is the right len
-   */
-  public final boolean incrementToken() throws IOException
-  {
-    assert termAtt != null;
-    // return the first non-stop word found
-    while (input.incrementToken()) {
-      int len = termAtt.termLength();
-      if (len >= min && len <= max) {
-          return true;
-      }
-      // note: else we ignore it but should we index each part of it?
-    }
-    // reached EOS -- return null
-    return false;
-  }
-}
-</pre>
-The TermAttribute is added in the constructor and stored in the instance variable <code>termAtt</code>.
-Remember that there can only be a single instance of TermAttribute in the chain, so in our example the 
-<code>addAttribute()</code> call in LengthFilter returns the TermAttribute that the WhitespaceTokenizer already added. The tokens
-are retrieved from the input stream in the <code>incrementToken()</code> method. By looking at the term text
-in the TermAttribute the length of the term can be determined and too short or too long tokens are skipped. 
-Note how <code>incrementToken()</code> can efficiently access the instance variable; no attribute lookup or downcasting
-is neccessary. The same is true for the consumer, which can simply use local references to the Attributes.
-<h4>Adding a custom Attribute</h4>
-Now we're going to implement our own custom Attribute for part-of-speech tagging and call it consequently 
-<code>PartOfSpeechAttribute</code>:
-<pre>
-  public static enum PartOfSpeech {
-    Noun, Verb, Adjective, Adverb, Pronoun, Preposition, Conjunction, Article, Unknown
-  }
-  
-  public static final class PartOfSpeechAttribute extends Attribute {
-    
-    private PartOfSpeech pos = PartOfSpeech.Unknown;
-    
-    public void setPartOfSpeech(PartOfSpeech pos) {
-      this.pos = pos;
-    }
-    
-    public PartOfSpeech getPartOfSpeech() {
-      return pos;
-    }
-
-    public void clear() {
-      pos = PartOfSpeech.Unknown;
-    }
-
-    public void copyTo(Attribute target) {
-      ((PartOfSpeechAttribute) target).pos = pos;
-    }
-
-    public boolean equals(Object other) {
-      if (other == this) {
-        return true;
-      }
-      
-      if (other instanceof PartOfSpeechAttribute) {
-        return pos == ((PartOfSpeechAttribute) other).pos;
-      }
-   
-      return false;
-    }
-
-    public int hashCode() {
-      return pos.ordinal();
-    }
-
-    public String toString() {
-      return "PartOfSpeech=" + pos;
-    }
-  }
-</pre>
-This is a simple Attribute that has only a single variable that stores the part-of-speech of a token. It extends the
-new <code>Attribute</code> class and therefore implements its abstract methods <code>clear(), copyTo(), equals(), hashCode(), toString()</code>.
-Now we need a TokenFilter that can set this new PartOfSpeechAttribute for each token. In this example we show a very naive filter
-that tags every word with a leading upper-case letter as a 'Noun' and all other words as 'Unknown'.
-<pre>
-  public static class PartOfSpeechTaggingFilter extends TokenFilter {
-    PartOfSpeechAttribute posAtt;
-    TermAttribute termAtt;
-    
-    protected PartOfSpeechTaggingFilter(TokenStream input) {
-      super(input);
-      posAtt = (PartOfSpeechAttribute) addAttribute(PartOfSpeechAttribute.class);
-      termAtt = (TermAttribute) addAttribute(TermAttribute.class);
-    }
-    
-    public boolean incrementToken() throws IOException {
-      if (!input.incrementToken()) {return false;}
-      posAtt.setPartOfSpeech(determinePOS(termAtt.termBuffer(), 0, termAtt.termLength()));
-      return true;
-    }
-    
-    // determine the part of speech for the given term
-    protected PartOfSpeech determinePOS(char[] term, int offset, int length) {
-      // naive implementation that tags every uppercased word as noun
-      if (length > 0 && Character.isUpperCase(term[0])) {
-        return PartOfSpeech.Noun;
-      }
-      return PartOfSpeech.Unknown;
-    }
-  }
-</pre>
-Just like the LengthFilter, this new filter accesses the attributes it needs in the constructor and
-stores references in instance variables. Now we need to add the filter to the chain:
-<pre>
-  public TokenStream tokenStream(String fieldName, Reader reader) {
-    TokenStream stream = new WhitespaceTokenizer(reader);
-    stream = new LengthFilter(stream, 3, Integer.MAX_VALUE);
-    stream = new PartOfSpeechTaggingFilter(stream);
-    return stream;
-  }
-</pre>
-Now let's look at the output:
-<pre>
-This
-demo
-the
-new
-TokenStream
-API
-</pre>
-Apparently it hasn't changed, which shows that adding a custom attribute to a TokenStream/Filter chain does not
-affect any existing consumers, simply because they don't know the new Attribute. Now let's change the consumer
-to make use of the new PartOfSpeechAttribute and print it out:
-<pre>
-  public static void main(String[] args) throws IOException {
-    // text to tokenize
-    final String text = "This is a demo of the new TokenStream API";
-    
-    MyAnalyzer analyzer = new MyAnalyzer();
-    TokenStream stream = analyzer.tokenStream("field", new StringReader(text));
-    
-    // get the TermAttribute from the TokenStream
-    TermAttribute termAtt = (TermAttribute) stream.getAttribute(TermAttribute.class);
-    
-    // get the PartOfSpeechAttribute from the TokenStream
-    PartOfSpeechAttribute posAtt = (PartOfSpeechAttribute) stream.getAttribute(PartOfSpeechAttribute.class);
-    
-    // print all tokens until stream is exhausted
-    while (stream.incrementToken()) {
-      System.out.println(termAtt.term() + ": " + posAtt.getPartOfSpeech());
-    }
-  }
-</pre>
-The change that was made is to get the PartOfSpeechAttribute from the TokenStream and print out its contents in
-the while loop that consumes the stream. Here is the new output:
-<pre>
-This: Noun
-demo: Unknown
-the: Unknown
-new: Unknown
-TokenStream: Noun
-API: Noun
-</pre>
-Each word is now followed by its assigned PartOfSpeech tag. Of course this is a naive 
-part-of-speech tagging. The word 'This' should not even be tagged as noun; it is only spelled capitalized because it
-is the first word of a sentence. Actually this is a good opportunity for an excerise. To practice the usage of the new
-API the reader could now write an Attribute and TokenFilter that can specify for each word if it was the first token
-of a sentence or not. Then the PartOfSpeechTaggingFilter can make use of this knowledge and only tag capitalized words
-as nouns if not the first word of a sentence (we know, this is still not a correct behavior, but hey, it's a good exercise). 
-As a small hint, this is how the new Attribute class could begin:
-<pre>
-  public class FirstTokenOfSentenceAttribute extends Attribute {
-    
-    private boolean firstToken;
-    
-    public void setFirstToken(boolean firstToken) {
-      this.firstToken = firstToken;
-    }
-    
-    public boolean getFirstToken() {
-      return firstToken;
-    }
-
-    public void clear() {
-      firstToken = false;
-    }
-
-  ...
-</pre>
 </body>
 </html>

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/standard/StandardFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/standard/StandardFilter.java?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/standard/StandardFilter.java (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/standard/StandardFilter.java Sat Jul  4 20:08:54 2009
@@ -17,11 +17,9 @@
  * limitations under the License.
  */
 
-import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
-import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
 
 /** Normalizes tokens extracted with {@link StandardTokenizer}. */
 
@@ -31,53 +29,14 @@
   /** Construct filtering <i>in</i>. */
   public StandardFilter(TokenStream in) {
     super(in);
-    termAtt = (TermAttribute) addAttribute(TermAttribute.class);
-    typeAtt = (TypeAttribute) addAttribute(TypeAttribute.class);
   }
 
   private static final String APOSTROPHE_TYPE = StandardTokenizerImpl.TOKEN_TYPES[StandardTokenizerImpl.APOSTROPHE];
   private static final String ACRONYM_TYPE = StandardTokenizerImpl.TOKEN_TYPES[StandardTokenizerImpl.ACRONYM];
 
-  // this filters uses attribute type
-  private TypeAttribute typeAtt;
-  private TermAttribute termAtt;
-  
-  /** Returns the next token in the stream, or null at EOS.
-   * <p>Removes <tt>'s</tt> from the end of words.
-   * <p>Removes dots from acronyms.
-   */
-  public final boolean incrementToken() throws java.io.IOException {
-    if (!input.incrementToken()) {
-      return false;
-    }
-
-    char[] buffer = termAtt.termBuffer();
-    final int bufferLength = termAtt.termLength();
-    final String type = typeAtt.type();
-
-    if (type == APOSTROPHE_TYPE &&      // remove 's
-  bufferLength >= 2 &&
-        buffer[bufferLength-2] == '\'' &&
-        (buffer[bufferLength-1] == 's' || buffer[bufferLength-1] == 'S')) {
-      // Strip last 2 characters off
-      termAtt.setTermLength(bufferLength - 2);
-    } else if (type == ACRONYM_TYPE) {      // remove dots
-      int upto = 0;
-      for(int i=0;i<bufferLength;i++) {
-        char c = buffer[i];
-        if (c != '.')
-          buffer[upto++] = c;
-      }
-      termAtt.setTermLength(upto);
-    }
-
-    return true;
-  }
-  
   /** Returns the next token in the stream, or null at EOS.
    * <p>Removes <tt>'s</tt> from the end of words.
    * <p>Removes dots from acronyms.
-   * @deprecated
    */
   public final Token next(final Token reusableToken) throws java.io.IOException {
     assert reusableToken != null;

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/standard/StandardTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/standard/StandardTokenizer.java?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/standard/StandardTokenizer.java (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/standard/StandardTokenizer.java Sat Jul  4 20:08:54 2009
@@ -22,10 +22,6 @@
 
 import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.Tokenizer;
-import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
-import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
-import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
 import org.apache.lucene.analysis.CharReader;
 import org.apache.lucene.analysis.CharStream;
 
@@ -90,7 +86,7 @@
    * 
    * @deprecated this should be removed in the next release (3.0).
    */
-  private boolean replaceInvalidAcronym;
+  private boolean replaceInvalidAcronym = false;
     
   void setInput(Reader reader) {
     this.input = CharReader.get(reader);
@@ -109,13 +105,14 @@
     return maxTokenLength;
   }
 
-  /**
-   * Creates a new instance of the {@link StandardTokenizer}. Attaches the
-   * <code>input</code> to a newly created JFlex scanner.
-   */
-  public StandardTokenizer(Reader input) {
-    this(input, false);
-  }
+    /**
+     * Creates a new instance of the {@link StandardTokenizer}. Attaches the
+     * <code>input</code> to a newly created JFlex scanner.
+     */
+    public StandardTokenizer(Reader input) {
+	    this.input = CharReader.get(input);
+	    this.scanner = new StandardTokenizerImpl(input);
+    }
 
   /**
    * Creates a new instance of the {@link org.apache.lucene.analysis.standard.StandardTokenizer}.  Attaches
@@ -130,68 +127,13 @@
     this.replaceInvalidAcronym = replaceInvalidAcronym;
     setInput(input);
     this.scanner = new StandardTokenizerImpl(input);
-    termAtt = (TermAttribute) addAttribute(TermAttribute.class);
-    offsetAtt = (OffsetAttribute) addAttribute(OffsetAttribute.class);
-    posIncrAtt = (PositionIncrementAttribute) addAttribute(PositionIncrementAttribute.class);
-    typeAtt = (TypeAttribute) addAttribute(TypeAttribute.class);
   }
 
-  // this tokenizer generates three attributes:
-  // offset, positionIncrement and type
-  private TermAttribute termAtt;
-  private OffsetAttribute offsetAtt;
-  private PositionIncrementAttribute posIncrAtt;
-  private TypeAttribute typeAtt;
-
-  /*
-   * (non-Javadoc)
-   *
-   * @see org.apache.lucene.analysis.TokenStream#next()
-   */
-  public boolean incrementToken() throws IOException {
-    int posIncr = 1;
-
-    while(true) {
-      int tokenType = scanner.getNextToken();
-
-      if (tokenType == StandardTokenizerImpl.YYEOF) {
-        return false;
-      }
-
-      if (scanner.yylength() <= maxTokenLength) {
-        termAtt.clear();
-        posIncrAtt.setPositionIncrement(posIncr);
-        scanner.getText(termAtt);
-        final int start = scanner.yychar();
-        offsetAtt.setStartOffset(start);
-        offsetAtt.setEndOffset(start+termAtt.termLength());
-        // This 'if' should be removed in the next release. For now, it converts
-        // invalid acronyms to HOST. When removed, only the 'else' part should
-        // remain.
-        if (tokenType == StandardTokenizerImpl.ACRONYM_DEP) {
-          if (replaceInvalidAcronym) {
-            typeAtt.setType(StandardTokenizerImpl.TOKEN_TYPES[StandardTokenizerImpl.HOST]);
-            termAtt.setTermLength(termAtt.termLength() - 1); // remove extra '.'
-          } else {
-            typeAtt.setType(StandardTokenizerImpl.TOKEN_TYPES[StandardTokenizerImpl.ACRONYM]);
-          }
-        } else {
-          typeAtt.setType(StandardTokenizerImpl.TOKEN_TYPES[tokenType]);
-        }
-        return true;
-      } else
-        // When we skip a too-long term, we still increment the
-        // position increment
-        posIncr++;
-    }
-  }
-  
   /*
    * (non-Javadoc)
    *
    * @see org.apache.lucene.analysis.TokenStream#next()
    */
-  /** @deprecated */
   public Token next(final Token reusableToken) throws IOException {
       assert reusableToken != null;
       int posIncr = 1;

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/standard/StandardTokenizerImpl.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/standard/StandardTokenizerImpl.java?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/standard/StandardTokenizerImpl.java (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/standard/StandardTokenizerImpl.java Sat Jul  4 20:08:54 2009
@@ -30,7 +30,6 @@
 */
 
 import org.apache.lucene.analysis.Token;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
 
 
 /**
@@ -369,13 +368,6 @@
   t.setTermBuffer(zzBuffer, zzStartRead, zzMarkedPos-zzStartRead);
 }
 
-/**
- * Fills TermAttribute with the current token text.
- */
-final void getText(TermAttribute t) {
-  t.setTermBuffer(zzBuffer, zzStartRead, zzMarkedPos-zzStartRead);
-}
-
 
   /**
    * Creates a new scanner

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/standard/StandardTokenizerImpl.jflex
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/standard/StandardTokenizerImpl.jflex?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/standard/StandardTokenizerImpl.jflex (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/analysis/standard/StandardTokenizerImpl.jflex Sat Jul  4 20:08:54 2009
@@ -29,7 +29,6 @@
 */
 
 import org.apache.lucene.analysis.Token;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
 
 %%
 
@@ -70,14 +69,6 @@
 final void getText(Token t) {
   t.setTermBuffer(zzBuffer, zzStartRead, zzMarkedPos-zzStartRead);
 }
-
-/**
- * Fills TermAttribute with the current token text.
- */
-final void getText(TermAttribute t) {
-  t.setTermBuffer(zzBuffer, zzStartRead, zzMarkedPos-zzStartRead);
-}
-
 %}
 
 THAI       = [\u0E00-\u0E59]

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/index/DocInverter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/index/DocInverter.java?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/index/DocInverter.java (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/index/DocInverter.java Sat Jul  4 20:08:54 2009
@@ -17,14 +17,12 @@
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.Collection;
+import java.util.Map;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Collection;
 import java.util.Iterator;
-import java.util.Map;
-
-import org.apache.lucene.util.AttributeSource;
+import java.io.IOException;
 
 /** This is a DocFieldConsumer that inverts each field,
  *  separately, from a Document, and accepts a

Modified: lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/index/DocInverterPerField.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/index/DocInverterPerField.java?rev=791173&r1=791172&r2=791173&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/index/DocInverterPerField.java (original)
+++ lucene/java/branches/lucene_2_4_back_compat_tests/src/java/org/apache/lucene/index/DocInverterPerField.java Sat Jul  4 20:08:54 2009
@@ -22,8 +22,6 @@
 import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
-import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 
 /**
  * Holds state for inverting all occurrences of a single
@@ -81,14 +79,10 @@
         if (!field.isTokenized()) {		  // un-tokenized field
           String stringValue = field.stringValue();
           final int valueLength = stringValue.length();
-          perThread.singleTokenTokenStream.reinit(stringValue, 0, valueLength);
-          fieldState.attributeSource = perThread.singleTokenTokenStream;
-          perThread.localTokenStream.reset();
-          consumer.start(field);
-
+          Token token = perThread.localToken.reinit(stringValue, 0, valueLength);
           boolean success = false;
           try {
-            consumer.add();
+            consumer.add(token);
             success = true;
           } finally {
             if (!success)
@@ -128,22 +122,7 @@
 
           try {
             int offsetEnd = fieldState.offset-1;
-            
-            boolean useNewTokenStreamAPI = stream.useNewAPI();
-            Token localToken = null;
-            
-            if (useNewTokenStreamAPI) {
-              fieldState.attributeSource = stream;
-            } else {              
-              fieldState.attributeSource = perThread.localTokenStream;
-              localToken = perThread.localToken;
-            }         
-            
-            consumer.start(field);
-
-            OffsetAttribute offsetAttribute = (OffsetAttribute) fieldState.attributeSource.addAttribute(OffsetAttribute.class);
-            PositionIncrementAttribute posIncrAttribute = (PositionIncrementAttribute) fieldState.attributeSource.addAttribute(PositionIncrementAttribute.class);
-            
+            final Token localToken = perThread.localToken;
             for(;;) {
 
               // If we hit an exception in stream.next below
@@ -152,16 +131,10 @@
               // non-aborting and (above) this one document
               // will be marked as deleted, but still
               // consume a docID
-              Token token = null;
-              if (useNewTokenStreamAPI) {
-                if (!stream.incrementToken()) break;
-              } else {
-                token = stream.next(localToken);
-                if (token == null) break;
-                perThread.localTokenStream.set(token);
-              }
-              
-              final int posIncr = posIncrAttribute.getPositionIncrement();
+              Token token = stream.next(localToken);
+
+              if (token == null) break;
+              final int posIncr = token.getPositionIncrement();
               fieldState.position += posIncr - 1;
               if (posIncr == 0)
                 fieldState.numOverlap++;
@@ -174,14 +147,14 @@
                 // internal state of the consumer is now
                 // corrupt and should not be flushed to a
                 // new segment:
-                consumer.add();
+                consumer.add(token);
                 success = true;
               } finally {
                 if (!success)
                   docState.docWriter.setAborting();
               }
               fieldState.position++;
-              offsetEnd = fieldState.offset + offsetAttribute.endOffset();
+              offsetEnd = fieldState.offset + token.endOffset();
               if (++fieldState.length >= maxFieldLength) {
                 if (docState.infoStream != null)
                   docState.infoStream.println("maxFieldLength " +maxFieldLength+ " reached for field " + fieldInfo.name + ", ignoring following tokens");