You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2014/03/21 01:33:54 UTC

svn commit: r1579846 - in /lucene/dev/trunk/lucene: ./ analysis/common/src/java/org/apache/lucene/analysis/th/ analysis/common/src/java/org/apache/lucene/analysis/util/ analysis/common/src/resources/META-INF/services/ analysis/common/src/test/org/apach...

Author: rmuir
Date: Fri Mar 21 00:33:53 2014
New Revision: 1579846

URL: http://svn.apache.org/r1579846
Log:
LUCENE-4984: Fix ThaiWordFilter, smartcn WordTokenFilter

Added:
    lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiTokenizer.java   (with props)
    lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiTokenizerFactory.java   (with props)
    lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/SegmentingTokenizerBase.java   (with props)
    lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/th/TestThaiTokenizerFactory.java   (with props)
    lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/util/TestSegmentingTokenizerBase.java   (with props)
    lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/HMMChineseTokenizer.java   (with props)
    lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/HMMChineseTokenizerFactory.java   (with props)
    lucene/dev/trunk/lucene/analysis/smartcn/src/test/org/apache/lucene/analysis/cn/smart/TestHMMChineseTokenizerFactory.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiAnalyzer.java
    lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiWordFilter.java
    lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiWordFilterFactory.java
    lucene/dev/trunk/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenizerFactory
    lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
    lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/th/TestThaiAnalyzer.java
    lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/th/TestThaiWordFilterFactory.java
    lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SentenceTokenizer.java
    lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SmartChineseAnalyzer.java
    lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SmartChineseSentenceTokenizerFactory.java
    lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SmartChineseWordTokenFilterFactory.java
    lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/WordTokenFilter.java
    lucene/dev/trunk/lucene/analysis/smartcn/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenizerFactory
    lucene/dev/trunk/lucene/analysis/smartcn/src/test/org/apache/lucene/analysis/cn/smart/TestSmartChineseFactories.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1579846&r1=1579845&r2=1579846&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Fri Mar 21 00:33:53 2014
@@ -119,6 +119,10 @@ New Features
   set of matching hits, in cases where there are millions of hits.
   (Rob Audenaerde, Gilad Barkai, Shai Erera)
 
+* LUCENE-4984: Add SegmentingTokenizerBase, abstract class for tokenizers
+  that want to do two-pass tokenization such as by sentence and then by word.
+  (Robert Muir)
+
 API Changes
 
 * LUCENE-5454: Add RandomAccessOrds, an optional extension of SortedSetDocValues
@@ -150,6 +154,11 @@ API Changes
   is complete in the sense of the top N or not. Consumers of this API should assert
   on the completeness if the bounded queue size is know ahead of time. (Simon Willnauer)
 
+* LUCENE-4984: Deprecate ThaiWordFilter and smartcn SentenceTokenizer and WordTokenFilter.
+  These filters would not work correctly with CharFilters and could not be safely placed
+  at an arbitrary position in the analysis chain. Use ThaiTokenizer and HMMChineseTokenizer
+  instead. (Robert Muir)
+
 Optimizations
 
 * LUCENE-5468: HunspellStemFilter uses 10 to 100x less RAM. It also loads

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiAnalyzer.java?rev=1579846&r1=1579845&r2=1579846&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiAnalyzer.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiAnalyzer.java Fri Mar 21 00:33:53 2014
@@ -102,11 +102,17 @@ public final class ThaiAnalyzer extends 
    */
   @Override
   protected TokenStreamComponents createComponents(String fieldName) {
-    final Tokenizer source = new StandardTokenizer(matchVersion);
-    TokenStream result = new StandardFilter(matchVersion, source);
-    result = new LowerCaseFilter(matchVersion, result);
-    result = new ThaiWordFilter(matchVersion, result);
-    return new TokenStreamComponents(source, new StopFilter(matchVersion,
-        result, stopwords));
+    if (matchVersion.onOrAfter(Version.LUCENE_48)) {
+      final Tokenizer source = new ThaiTokenizer();
+      TokenStream result = new LowerCaseFilter(matchVersion, source);
+      result = new StopFilter(matchVersion, result, stopwords);
+      return new TokenStreamComponents(source, result);
+    } else {
+      final Tokenizer source = new StandardTokenizer(matchVersion);
+      TokenStream result = new StandardFilter(matchVersion, source);
+      result = new LowerCaseFilter(matchVersion, result);
+      result = new ThaiWordFilter(matchVersion, result);
+      return new TokenStreamComponents(source, new StopFilter(matchVersion, result, stopwords));
+    }
   }
 }

Added: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiTokenizer.java?rev=1579846&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiTokenizer.java (added)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiTokenizer.java Fri Mar 21 00:33:53 2014
@@ -0,0 +1,106 @@
+package org.apache.lucene.analysis.th;
+
+/*
+ * 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 java.text.BreakIterator;
+import java.util.Locale;
+
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.analysis.util.CharArrayIterator;
+import org.apache.lucene.analysis.util.SegmentingTokenizerBase;
+
+/**
+ * Tokenizer that use {@link BreakIterator} to tokenize Thai text.
+ * <p>WARNING: this tokenizer may not be supported by all JREs.
+ *    It is known to work with Sun/Oracle and Harmony JREs.
+ *    If your application needs to be fully portable, consider using ICUTokenizer instead,
+ *    which uses an ICU Thai BreakIterator that will always be available.
+ */
+public class ThaiTokenizer extends SegmentingTokenizerBase {
+  /** 
+   * True if the JRE supports a working dictionary-based breakiterator for Thai.
+   * If this is false, this tokenizer will not work at all!
+   */
+  public static final boolean DBBI_AVAILABLE;
+  private static final BreakIterator proto = BreakIterator.getWordInstance(new Locale("th"));
+  static {
+    // check that we have a working dictionary-based break iterator for thai
+    proto.setText("ภาษาไทย");
+    DBBI_AVAILABLE = proto.isBoundary(4);
+  }
+  
+  /** used for breaking the text into sentences */
+  private static final BreakIterator sentenceProto = BreakIterator.getSentenceInstance(Locale.ROOT);
+  
+  private final BreakIterator wordBreaker;
+  private final CharArrayIterator wrapper = CharArrayIterator.newWordInstance();
+  
+  int sentenceStart;
+  int sentenceEnd;
+  
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+  private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+  
+  /** Creates a new ThaiTokenizer */
+  public ThaiTokenizer() {
+    this(AttributeFactory.DEFAULT_ATTRIBUTE_FACTORY);
+  }
+      
+  /** Creates a new ThaiTokenizer, supplying the AttributeFactory */
+  public ThaiTokenizer(AttributeFactory factory) {
+    super((BreakIterator)sentenceProto.clone());
+    if (!DBBI_AVAILABLE) {
+      throw new UnsupportedOperationException("This JRE does not have support for Thai segmentation");
+    }
+    wordBreaker = (BreakIterator)proto.clone();
+  }
+
+  @Override
+  protected void setNextSentence(int sentenceStart, int sentenceEnd) {
+    this.sentenceStart = sentenceStart;
+    this.sentenceEnd = sentenceEnd;
+    wrapper.setText(buffer, sentenceStart, sentenceEnd - sentenceStart);
+    wordBreaker.setText(wrapper);
+  }
+
+  @Override
+  protected boolean incrementWord() {
+    int start = wordBreaker.current();
+    if (start == BreakIterator.DONE) {
+      return false; // BreakIterator exhausted
+    }
+
+    // find the next set of boundaries, skipping over non-tokens
+    int end = wordBreaker.next();
+    while (end != BreakIterator.DONE &&
+           !Character.isLetterOrDigit(Character.codePointAt(buffer, sentenceStart + start, sentenceEnd))) {
+      start = end;
+      end = wordBreaker.next();
+    }
+
+    if (end == BreakIterator.DONE) {
+      return false; // BreakIterator exhausted
+    }
+
+    clearAttributes();
+    termAtt.copyBuffer(buffer, start, end - start);
+    offsetAtt.setOffset(correctOffset(offset + sentenceStart + start), correctOffset(offset + sentenceStart + end));
+    return true;
+  }
+}

Added: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiTokenizerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiTokenizerFactory.java?rev=1579846&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiTokenizerFactory.java (added)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiTokenizerFactory.java Fri Mar 21 00:33:53 2014
@@ -0,0 +1,50 @@
+package org.apache.lucene.analysis.th;
+
+/*
+ * 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 java.util.Map;
+
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.util.TokenizerFactory;
+import org.apache.lucene.util.AttributeSource;
+
+/** 
+ * Factory for {@link ThaiTokenizer}.
+ * <pre class="prettyprint">
+ * &lt;fieldType name="text_thai" class="solr.TextField" positionIncrementGap="100"&gt;
+ *   &lt;analyzer&gt;
+ *     &lt;tokenizer class="solr.ThaiTokenizerFactory"/&gt;
+ *   &lt;/analyzer&gt;
+ * &lt;/fieldType&gt;</pre>
+ */
+public class ThaiTokenizerFactory extends TokenizerFactory {
+  
+  /** Creates a new ThaiTokenizerFactory */
+  public ThaiTokenizerFactory(Map<String,String> args) {
+    super(args);
+    if (!args.isEmpty()) {
+      throw new IllegalArgumentException("Unknown parameters: " + args);
+    }
+  }
+  
+  @Override
+  public Tokenizer create(AttributeSource.AttributeFactory factory) {
+    return new ThaiTokenizer(factory);
+  }
+}
+

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiWordFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiWordFilter.java?rev=1579846&r1=1579845&r2=1579846&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiWordFilter.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiWordFilter.java Fri Mar 21 00:33:53 2014
@@ -37,19 +37,16 @@ import org.apache.lucene.util.Version;
  *    It is known to work with Sun/Oracle and Harmony JREs.
  *    If your application needs to be fully portable, consider using ICUTokenizer instead,
  *    which uses an ICU Thai BreakIterator that will always be available.
+ * @deprecated Use {@link ThaiTokenizer} instead.
  */
+@Deprecated
 public final class ThaiWordFilter extends TokenFilter {
   /** 
    * True if the JRE supports a working dictionary-based breakiterator for Thai.
    * If this is false, this filter will not work at all!
    */
-  public static final boolean DBBI_AVAILABLE;
+  public static final boolean DBBI_AVAILABLE = ThaiTokenizer.DBBI_AVAILABLE;
   private static final BreakIterator proto = BreakIterator.getWordInstance(new Locale("th"));
-  static {
-    // check that we have a working dictionary-based break iterator for thai
-    proto.setText("ภาษาไทย");
-    DBBI_AVAILABLE = proto.isBoundary(4);
-  }
   private final BreakIterator breaker = (BreakIterator) proto.clone();
   private final CharArrayIterator charIterator = CharArrayIterator.newWordInstance();
   

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiWordFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiWordFilterFactory.java?rev=1579846&r1=1579845&r2=1579846&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiWordFilterFactory.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/th/ThaiWordFilterFactory.java Fri Mar 21 00:33:53 2014
@@ -33,7 +33,9 @@ import org.apache.lucene.analysis.util.T
  *     &lt;filter class="solr.ThaiWordFilterFactory"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre>
+ * @deprecated Use {@link ThaiTokenizerFactory} instead
  */
+@Deprecated
 public class ThaiWordFilterFactory extends TokenFilterFactory {
   
   /** Creates a new ThaiWordFilterFactory */

Added: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/SegmentingTokenizerBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/SegmentingTokenizerBase.java?rev=1579846&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/SegmentingTokenizerBase.java (added)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/util/SegmentingTokenizerBase.java Fri Mar 21 00:33:53 2014
@@ -0,0 +1,202 @@
+package org.apache.lucene.analysis.util;
+
+/*
+ * 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 java.io.IOException;
+import java.io.Reader;
+
+import java.text.BreakIterator;
+
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+
+/**
+ * Breaks text into sentences with a {@link BreakIterator} and
+ * allows subclasses to decompose these sentences into words.
+ * <p>
+ * This can be used by subclasses that need sentence context 
+ * for tokenization purposes, such as CJK segmenters.
+ * <p>
+ * Additionally it can be used by subclasses that want to mark
+ * sentence boundaries (with a custom attribute, extra token, position
+ * increment, etc) for downstream processing.
+ * 
+ * @lucene.experimental
+ */
+public abstract class SegmentingTokenizerBase extends Tokenizer {
+  protected static final int BUFFERMAX = 1024;
+  protected final char buffer[] = new char[BUFFERMAX];
+  /** true length of text in the buffer */
+  private int length = 0; 
+  /** length in buffer that can be evaluated safely, up to a safe end point */
+  private int usableLength = 0; 
+  /** accumulated offset of previous buffers for this reader, for offsetAtt */
+  protected int offset = 0;
+  
+  private final BreakIterator iterator;
+  private final CharArrayIterator wrapper = CharArrayIterator.newSentenceInstance();
+
+  private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+
+  /**
+   * Construct a new SegmenterBase, using
+   * the provided BreakIterator for sentence segmentation.
+   * <p>
+   * Note that you should never share BreakIterators across different
+   * TokenStreams, instead a newly created or cloned one should always
+   * be provided to this constructor.
+   */
+  public SegmentingTokenizerBase(BreakIterator iterator) {
+    this(AttributeFactory.DEFAULT_ATTRIBUTE_FACTORY, iterator);
+  }
+  
+  /**
+   * Construct a new SegmenterBase, also supplying the AttributeFactory
+   */
+  public SegmentingTokenizerBase(AttributeFactory factory, BreakIterator iterator) {
+    super(factory);
+    this.iterator = iterator;
+  }
+
+  @Override
+  public final boolean incrementToken() throws IOException {
+    if (length == 0 || !incrementWord()) {
+      while (!incrementSentence()) {
+        refill();
+        if (length <= 0) // no more bytes to read;
+          return false;
+      }
+    }
+    
+    return true;
+  }
+  
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    wrapper.setText(buffer, 0, 0);
+    iterator.setText(wrapper);
+    length = usableLength = offset = 0;
+  }
+  
+  @Override
+  public final void end() throws IOException {
+    super.end();
+    final int finalOffset = correctOffset(length < 0 ? offset : offset + length);
+    offsetAtt.setOffset(finalOffset, finalOffset);
+  }  
+
+  /** Returns the last unambiguous break position in the text. */
+  private int findSafeEnd() {
+    for (int i = length - 1; i >= 0; i--)
+      if (isSafeEnd(buffer[i]))
+        return i + 1;
+    return -1;
+  }
+  
+  /** For sentence tokenization, these are the unambiguous break positions. */
+  protected boolean isSafeEnd(char ch) {
+    switch(ch) {
+      case 0x000D:
+      case 0x000A:
+      case 0x0085:
+      case 0x2028:
+      case 0x2029:
+        return true;
+      default:
+        return false;
+    }
+  }
+
+  /**
+   * Refill the buffer, accumulating the offset and setting usableLength to the
+   * last unambiguous break position
+   */
+  private void refill() throws IOException {
+    offset += usableLength;
+    int leftover = length - usableLength;
+    System.arraycopy(buffer, usableLength, buffer, 0, leftover);
+    int requested = buffer.length - leftover;
+    int returned = read(input, buffer, leftover, requested);
+    length = returned < 0 ? leftover : returned + leftover;
+    if (returned < requested) /* reader has been emptied, process the rest */
+      usableLength = length;
+    else { /* still more data to be read, find a safe-stopping place */
+      usableLength = findSafeEnd();
+      if (usableLength < 0)
+        usableLength = length; /*
+                                * more than IOBUFFER of text without breaks,
+                                * gonna possibly truncate tokens
+                                */
+    }
+
+    wrapper.setText(buffer, 0, Math.max(0, usableLength));
+    iterator.setText(wrapper);
+  }
+  
+  // TODO: refactor to a shared readFully somewhere
+  // (NGramTokenizer does this too):
+  /** commons-io's readFully, but without bugs if offset != 0 */
+  private static int read(Reader input, char[] buffer, int offset, int length) throws IOException {
+    assert length >= 0 : "length must not be negative: " + length;
+ 
+    int remaining = length;
+    while (remaining > 0) {
+      int location = length - remaining;
+      int count = input.read(buffer, offset + location, remaining);
+      if (-1 == count) { // EOF
+        break;
+      }
+      remaining -= count;
+    }
+    return length - remaining;
+  }
+
+  /**
+   * return true if there is a token from the buffer, or null if it is
+   * exhausted.
+   */
+  private boolean incrementSentence() throws IOException {
+    if (length == 0) // we must refill the buffer
+      return false;
+    
+    while (true) {
+      int start = iterator.current();
+
+      if (start == BreakIterator.DONE)
+        return false; // BreakIterator exhausted
+
+      // find the next set of boundaries
+      int end = iterator.next();
+
+      if (end == BreakIterator.DONE)
+        return false; // BreakIterator exhausted
+
+      setNextSentence(start, end);
+      if (incrementWord()) {
+        return true;
+      }
+    }
+  }
+  
+  /** Provides the next input sentence for analysis */
+  protected abstract void setNextSentence(int sentenceStart, int sentenceEnd);
+  
+  /** Returns true if another word is available */
+  protected abstract boolean incrementWord();
+}

Modified: lucene/dev/trunk/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenizerFactory
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenizerFactory?rev=1579846&r1=1579845&r2=1579846&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenizerFactory (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenizerFactory Fri Mar 21 00:33:53 2014
@@ -24,4 +24,5 @@ org.apache.lucene.analysis.pattern.Patte
 org.apache.lucene.analysis.standard.ClassicTokenizerFactory
 org.apache.lucene.analysis.standard.StandardTokenizerFactory
 org.apache.lucene.analysis.standard.UAX29URLEmailTokenizerFactory
+org.apache.lucene.analysis.th.ThaiTokenizerFactory
 org.apache.lucene.analysis.wikipedia.WikipediaTokenizerFactory

Modified: lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java?rev=1579846&r1=1579845&r2=1579846&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java Fri Mar 21 00:33:53 2014
@@ -77,7 +77,6 @@ import org.apache.lucene.analysis.payloa
 import org.apache.lucene.analysis.snowball.TestSnowball;
 import org.apache.lucene.analysis.standard.StandardTokenizer;
 import org.apache.lucene.analysis.synonym.SynonymMap;
-import org.apache.lucene.analysis.th.ThaiWordFilter;
 import org.apache.lucene.analysis.util.CharArrayMap;
 import org.apache.lucene.analysis.util.CharArraySet;
 import org.apache.lucene.analysis.wikipedia.WikipediaTokenizer;
@@ -165,8 +164,6 @@ public class TestRandomChains extends Ba
           // TODO: it seems to mess up offsets!?
           WikipediaTokenizer.class,
           // TODO: doesn't handle graph inputs
-          ThaiWordFilter.class,
-          // TODO: doesn't handle graph inputs
           CJKBigramFilter.class,
           // TODO: doesn't handle graph inputs (or even look at positionIncrement)
           HyphenatedWordsFilter.class,

Modified: lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/th/TestThaiAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/th/TestThaiAnalyzer.java?rev=1579846&r1=1579845&r2=1579846&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/th/TestThaiAnalyzer.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/th/TestThaiAnalyzer.java Fri Mar 21 00:33:53 2014
@@ -40,7 +40,7 @@ public class TestThaiAnalyzer extends Ba
   @Override
   public void setUp() throws Exception {
     super.setUp();
-    assumeTrue("JRE does not support Thai dictionary-based BreakIterator", ThaiWordFilter.DBBI_AVAILABLE);
+    assumeTrue("JRE does not support Thai dictionary-based BreakIterator", ThaiTokenizer.DBBI_AVAILABLE);
   }
   /* 
    * testcase for offsets
@@ -60,16 +60,6 @@ public class TestThaiAnalyzer extends Ba
         new int[] { 5, 2, 1 });
   }
   
-  public void testTokenType() throws Exception {
-      assertAnalyzesTo(new ThaiAnalyzer(TEST_VERSION_CURRENT, CharArraySet.EMPTY_SET), "การที่ได้ต้องแสดงว่างานดี ๑๒๓", 
-                       new String[] { "การ", "ที่", "ได้", "ต้อง", "แสดง", "ว่า", "งาน", "ดี", "๑๒๓" },
-                       new String[] { "<SOUTHEAST_ASIAN>", "<SOUTHEAST_ASIAN>", 
-                                      "<SOUTHEAST_ASIAN>", "<SOUTHEAST_ASIAN>", 
-                                      "<SOUTHEAST_ASIAN>", "<SOUTHEAST_ASIAN>",
-                                      "<SOUTHEAST_ASIAN>", "<SOUTHEAST_ASIAN>",
-                                      "<NUM>" });
-  }
-
   /*
    * Test that position increments are adjusted correctly for stopwords.
    */
@@ -127,15 +117,4 @@ public class TestThaiAnalyzer extends Ba
     ts.addAttribute(FlagsAttribute.class);
     assertTokenStreamContents(ts, new String[] { "ภาษา", "ไทย" });
   }
-  
-  public void testEmptyTerm() throws IOException {
-    Analyzer a = new Analyzer() {
-      @Override
-      protected TokenStreamComponents createComponents(String fieldName) {
-        Tokenizer tokenizer = new KeywordTokenizer();
-        return new TokenStreamComponents(tokenizer, new ThaiWordFilter(TEST_VERSION_CURRENT, tokenizer));
-      }
-    };
-    checkOneTerm(a, "", "");
-  }
 }

Added: lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/th/TestThaiTokenizerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/th/TestThaiTokenizerFactory.java?rev=1579846&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/th/TestThaiTokenizerFactory.java (added)
+++ lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/th/TestThaiTokenizerFactory.java Fri Mar 21 00:33:53 2014
@@ -0,0 +1,50 @@
+package org.apache.lucene.analysis.th;
+
+/*
+ * 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 java.io.StringReader;
+
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.util.BaseTokenStreamFactoryTestCase;
+
+/**
+ * Simple tests to ensure the Thai word filter factory is working.
+ */
+public class TestThaiTokenizerFactory extends BaseTokenStreamFactoryTestCase {
+  /**
+   * Ensure the filter actually decomposes text.
+   */
+  public void testWordBreak() throws Exception {
+    assumeTrue("JRE does not support Thai dictionary-based BreakIterator", ThaiTokenizer.DBBI_AVAILABLE);
+    Tokenizer tokenizer = tokenizerFactory("Thai").create();
+    tokenizer.setReader(new StringReader("การที่ได้ต้องแสดงว่างานดี"));
+    assertTokenStreamContents(tokenizer, new String[] {"การ", "ที่", "ได้",
+        "ต้อง", "แสดง", "ว่า", "งาน", "ดี"});
+  }
+  
+  /** Test that bogus arguments result in exception */
+  public void testBogusArguments() throws Exception {
+    assumeTrue("JRE does not support Thai dictionary-based BreakIterator", ThaiTokenizer.DBBI_AVAILABLE);
+    try {
+      tokenizerFactory("Thai", "bogusArg", "bogusValue");
+      fail();
+    } catch (IllegalArgumentException expected) {
+      assertTrue(expected.getMessage().contains("Unknown parameters"));
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/th/TestThaiWordFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/th/TestThaiWordFilterFactory.java?rev=1579846&r1=1579845&r2=1579846&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/th/TestThaiWordFilterFactory.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/th/TestThaiWordFilterFactory.java Fri Mar 21 00:33:53 2014
@@ -28,6 +28,7 @@ import org.apache.lucene.analysis.util.B
 /**
  * Simple tests to ensure the Thai word filter factory is working.
  */
+@Deprecated
 public class TestThaiWordFilterFactory extends BaseTokenStreamFactoryTestCase {
   /**
    * Ensure the filter actually decomposes text.

Added: lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/util/TestSegmentingTokenizerBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/util/TestSegmentingTokenizerBase.java?rev=1579846&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/util/TestSegmentingTokenizerBase.java (added)
+++ lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/util/TestSegmentingTokenizerBase.java Fri Mar 21 00:33:53 2014
@@ -0,0 +1,220 @@
+package org.apache.lucene.analysis.util;
+
+/*
+ * 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 java.io.IOException;
+import java.text.BreakIterator;
+import java.util.Arrays;
+import java.util.Locale;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+
+/** Basic tests for {@link SegmentingTokenizerBase} */
+public class TestSegmentingTokenizerBase extends BaseTokenStreamTestCase {
+  private Analyzer sentence = new Analyzer() {
+    @Override
+    protected TokenStreamComponents createComponents(String fieldName) {
+      return new TokenStreamComponents(new WholeSentenceTokenizer());
+    }
+  };
+  
+  private Analyzer sentenceAndWord = new Analyzer() {
+    @Override
+    protected TokenStreamComponents createComponents(String fieldName) {
+      return new TokenStreamComponents(new SentenceAndWordTokenizer());
+    }
+  };
+  
+  /** Some simple examples, just outputting the whole sentence boundaries as "terms" */
+  public void testBasics() throws IOException {
+    assertAnalyzesTo(sentence, "The acronym for United States is U.S. but this doesn't end a sentence",
+        new String[] { "The acronym for United States is U.S. but this doesn't end a sentence"}
+    );
+    assertAnalyzesTo(sentence, "He said, \"Are you going?\" John shook his head.",
+        new String[] { "He said, \"Are you going?\" ", 
+                       "John shook his head." }
+    );
+  }
+  
+  /** Test a subclass that sets some custom attribute values */
+  public void testCustomAttributes() throws IOException {
+    assertAnalyzesTo(sentenceAndWord, "He said, \"Are you going?\" John shook his head.",
+        new String[] { "He", "said", "Are", "you", "going", "John", "shook", "his", "head" },
+        new int[] { 0, 3, 10, 14, 18, 26, 31, 37, 41 },
+        new int[] { 2, 7, 13, 17, 23, 30, 36, 40, 45 },
+        new int[] { 1, 1,  1,  1,  1,  2,  1,  1,  1 }
+    );
+  }
+  
+  /** Tests tokenstream reuse */
+  public void testReuse() throws IOException {
+    assertAnalyzesTo(sentenceAndWord, "He said, \"Are you going?\"",
+        new String[] { "He", "said", "Are", "you", "going" },
+        new int[] { 0, 3, 10, 14, 18 },
+        new int[] { 2, 7, 13, 17, 23 },
+        new int[] { 1, 1,  1,  1,  1,}
+    );
+    assertAnalyzesTo(sentenceAndWord, "John shook his head.",
+        new String[] { "John", "shook", "his", "head" },
+        new int[] { 0,  5, 11, 15 },
+        new int[] { 4, 10, 14, 19 },
+        new int[] { 1,  1,  1,  1 }
+    );
+  }
+  
+  /** Tests TokenStream.end() */
+  public void testEnd() throws IOException {
+    // BaseTokenStreamTestCase asserts that end() is set to our StringReader's length for us here.
+    // we add some junk whitespace to the end just to test it.
+    assertAnalyzesTo(sentenceAndWord, "John shook his head          ",
+        new String[] { "John", "shook", "his", "head" }
+    );
+    assertAnalyzesTo(sentenceAndWord, "John shook his head.          ",
+        new String[] { "John", "shook", "his", "head" }
+    );
+  }
+  
+  /** Tests terms which span across boundaries */
+  public void testHugeDoc() throws IOException {
+    StringBuilder sb = new StringBuilder();
+    char whitespace[] = new char[4094];
+    Arrays.fill(whitespace, '\n');
+    sb.append(whitespace);
+    sb.append("testing 1234");
+    String input = sb.toString();
+    assertAnalyzesTo(sentenceAndWord, input, new String[] { "testing", "1234" });
+  }
+  
+  /** Tests the handling of binary/malformed data */
+  public void testHugeTerm() throws IOException {
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < 10240; i++) {
+      sb.append('a');
+    }
+    String input = sb.toString();
+    char token[] = new char[1024];
+    Arrays.fill(token, 'a');
+    String expectedToken = new String(token);
+    String expected[] = { 
+        expectedToken, expectedToken, expectedToken, 
+        expectedToken, expectedToken, expectedToken,
+        expectedToken, expectedToken, expectedToken,
+        expectedToken
+    };
+    assertAnalyzesTo(sentence, input, expected);
+  }
+  
+  /** blast some random strings through the analyzer */
+  public void testRandomStrings() throws Exception {
+    checkRandomData(random(), sentence, 10000*RANDOM_MULTIPLIER);
+    checkRandomData(random(), sentenceAndWord, 10000*RANDOM_MULTIPLIER);
+  }
+
+  // some tokenizers for testing
+  
+  /** silly tokenizer that just returns whole sentences as tokens */
+  static class WholeSentenceTokenizer extends SegmentingTokenizerBase {
+    int sentenceStart, sentenceEnd;
+    boolean hasSentence;
+    
+    private CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+    private OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+    
+    public WholeSentenceTokenizer() {
+      super(BreakIterator.getSentenceInstance(Locale.ROOT));
+    }
+
+    @Override
+    protected void setNextSentence(int sentenceStart, int sentenceEnd) {
+      this.sentenceStart = sentenceStart;
+      this.sentenceEnd = sentenceEnd;
+      hasSentence = true;
+    }
+
+    @Override
+    protected boolean incrementWord() {
+      if (hasSentence) {
+        hasSentence = false;
+        clearAttributes();
+        termAtt.copyBuffer(buffer, sentenceStart, sentenceEnd-sentenceStart);
+        offsetAtt.setOffset(correctOffset(offset+sentenceStart), correctOffset(offset+sentenceEnd));
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+  
+  /** 
+   * simple tokenizer, that bumps posinc + 1 for tokens after a 
+   * sentence boundary to inhibit phrase queries without slop.
+   */
+  static class SentenceAndWordTokenizer extends SegmentingTokenizerBase {
+    int sentenceStart, sentenceEnd;
+    int wordStart, wordEnd;
+    int posBoost = -1; // initially set to -1 so the first word in the document doesn't get a pos boost
+    
+    private CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+    private OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+    private PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
+    
+    public SentenceAndWordTokenizer() {
+      super(BreakIterator.getSentenceInstance(Locale.ROOT));
+    }
+
+    @Override
+    protected void setNextSentence(int sentenceStart, int sentenceEnd) {
+      this.wordStart = this.wordEnd = this.sentenceStart = sentenceStart;
+      this.sentenceEnd = sentenceEnd;
+      posBoost++;
+    }
+    
+    @Override
+    public void reset() throws IOException {
+      super.reset();
+      posBoost = -1;
+    }
+
+    @Override
+    protected boolean incrementWord() {
+      wordStart = wordEnd;
+      while (wordStart < sentenceEnd) {
+        if (Character.isLetterOrDigit(buffer[wordStart]))
+          break;
+        wordStart++;
+      }
+      
+      if (wordStart == sentenceEnd) return false;
+      
+      wordEnd = wordStart+1;
+      while (wordEnd < sentenceEnd && Character.isLetterOrDigit(buffer[wordEnd]))
+        wordEnd++;
+      
+      clearAttributes();
+      termAtt.copyBuffer(buffer, wordStart, wordEnd-wordStart);
+      offsetAtt.setOffset(correctOffset(offset+wordStart), correctOffset(offset+wordEnd));
+      posIncAtt.setPositionIncrement(posIncAtt.getPositionIncrement() + posBoost);
+      posBoost = 0;
+      return true;
+    }
+  }
+}

Added: lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/HMMChineseTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/HMMChineseTokenizer.java?rev=1579846&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/HMMChineseTokenizer.java (added)
+++ lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/HMMChineseTokenizer.java Fri Mar 21 00:33:53 2014
@@ -0,0 +1,83 @@
+package org.apache.lucene.analysis.cn.smart;
+
+/*
+ * 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 java.io.IOException;
+import java.text.BreakIterator;
+import java.util.Iterator;
+import java.util.Locale;
+
+import org.apache.lucene.analysis.cn.smart.hhmm.SegToken;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
+import org.apache.lucene.analysis.util.SegmentingTokenizerBase;
+
+/**
+ * Tokenizer for Chinese or mixed Chinese-English text.
+ * <p>
+ * The analyzer uses probabilistic knowledge to find the optimal word segmentation for Simplified Chinese text.
+ * The text is first broken into sentences, then each sentence is segmented into words.
+ */
+public class HMMChineseTokenizer extends SegmentingTokenizerBase {
+  /** used for breaking the text into sentences */
+  private static final BreakIterator sentenceProto = BreakIterator.getSentenceInstance(Locale.ROOT);
+  
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+  private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+  private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class);
+  
+  private final WordSegmenter wordSegmenter = new WordSegmenter();
+  private Iterator<SegToken> tokens;
+
+  /** Creates a new HMMChineseTokenizer */
+  public HMMChineseTokenizer() {
+    this(AttributeFactory.DEFAULT_ATTRIBUTE_FACTORY);
+  }
+      
+  /** Creates a new HMMChineseTokenizer, supplying the AttributeFactory */
+  public HMMChineseTokenizer(AttributeFactory factory) {
+    super((BreakIterator)sentenceProto.clone());
+  }
+
+  @Override
+  protected void setNextSentence(int sentenceStart, int sentenceEnd) {
+    String sentence = new String(buffer, sentenceStart, sentenceEnd - sentenceStart);
+    tokens = wordSegmenter.segmentSentence(sentence, offset + sentenceStart).iterator();
+  }
+
+  @Override
+  protected boolean incrementWord() {
+    if (tokens == null || !tokens.hasNext()) {
+      return false;
+    } else {
+      SegToken token = tokens.next();
+      clearAttributes();
+      termAtt.copyBuffer(token.charArray, 0, token.charArray.length);
+      offsetAtt.setOffset(correctOffset(token.startOffset), correctOffset(token.endOffset));
+      typeAtt.setType("word");
+      return true;
+    }
+  }
+  
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    tokens = null;
+  }
+}

Added: lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/HMMChineseTokenizerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/HMMChineseTokenizerFactory.java?rev=1579846&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/HMMChineseTokenizerFactory.java (added)
+++ lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/HMMChineseTokenizerFactory.java Fri Mar 21 00:33:53 2014
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.analysis.cn.smart;
+
+import java.util.Map;
+
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.util.TokenizerFactory;
+import org.apache.lucene.util.AttributeSource.AttributeFactory;
+
+/**
+ * Factory for {@link HMMChineseTokenizer}
+ * <p>
+ * Note: this class will currently emit tokens for punctuation. So you should either add
+ * a WordDelimiterFilter after to remove these (with concatenate off), or use the 
+ * SmartChinese stoplist with a StopFilterFactory via:
+ * <code>words="org/apache/lucene/analysis/cn/smart/stopwords.txt"</code>
+ * @lucene.experimental
+ */
+public final class HMMChineseTokenizerFactory extends TokenizerFactory {
+
+  /** Creates a new HMMChineseTokenizerFactory */
+  public HMMChineseTokenizerFactory(Map<String,String> args) {
+    super(args);
+    if (!args.isEmpty()) {
+      throw new IllegalArgumentException("Unknown parameters: " + args);
+    }
+  }
+
+  @Override
+  public Tokenizer create(AttributeFactory factory) {
+    return new HMMChineseTokenizer(factory);
+  }
+}

Modified: lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SentenceTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SentenceTokenizer.java?rev=1579846&r1=1579845&r2=1579846&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SentenceTokenizer.java (original)
+++ lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SentenceTokenizer.java Fri Mar 21 00:33:53 2014
@@ -32,7 +32,9 @@ import org.apache.lucene.util.AttributeS
  * The output tokens can then be broken into words with {@link WordTokenFilter}
  * </p>
  * @lucene.experimental
+ * @deprecated Use {@link HMMChineseTokenizer} instead
  */
+@Deprecated
 public final class SentenceTokenizer extends Tokenizer {
 
   /**

Modified: lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SmartChineseAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SmartChineseAnalyzer.java?rev=1579846&r1=1579845&r2=1579846&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SmartChineseAnalyzer.java (original)
+++ lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SmartChineseAnalyzer.java Fri Mar 21 00:33:53 2014
@@ -137,8 +137,15 @@ public final class SmartChineseAnalyzer 
 
   @Override
   public TokenStreamComponents createComponents(String fieldName) {
-    Tokenizer tokenizer = new SentenceTokenizer();
-    TokenStream result = new WordTokenFilter(tokenizer);
+    final Tokenizer tokenizer;
+    TokenStream result;
+    if (matchVersion.onOrAfter(Version.LUCENE_48)) {
+      tokenizer = new HMMChineseTokenizer();
+      result = tokenizer;
+    } else {
+      tokenizer = new SentenceTokenizer();
+      result = new WordTokenFilter(tokenizer);
+    }
     // result = new LowerCaseFilter(result);
     // LowerCaseFilter is not needed, as SegTokenFilter lowercases Basic Latin text.
     // The porter stemming is too strict, this is not a bug, this is a feature:)

Modified: lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SmartChineseSentenceTokenizerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SmartChineseSentenceTokenizerFactory.java?rev=1579846&r1=1579845&r2=1579846&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SmartChineseSentenceTokenizerFactory.java (original)
+++ lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SmartChineseSentenceTokenizerFactory.java Fri Mar 21 00:33:53 2014
@@ -26,7 +26,9 @@ import org.apache.lucene.util.AttributeS
 /**
  * Factory for the SmartChineseAnalyzer {@link SentenceTokenizer}
  * @lucene.experimental
+ * @deprecated Use {@link HMMChineseTokenizerFactory} instead
  */
+@Deprecated
 public class SmartChineseSentenceTokenizerFactory extends TokenizerFactory {
   
   /** Creates a new SmartChineseSentenceTokenizerFactory */

Modified: lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SmartChineseWordTokenFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SmartChineseWordTokenFilterFactory.java?rev=1579846&r1=1579845&r2=1579846&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SmartChineseWordTokenFilterFactory.java (original)
+++ lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/SmartChineseWordTokenFilterFactory.java Fri Mar 21 00:33:53 2014
@@ -32,7 +32,9 @@ import org.apache.lucene.analysis.util.T
  * SmartChinese stoplist with a StopFilterFactory via:
  * <code>words="org/apache/lucene/analysis/cn/smart/stopwords.txt"</code>
  * @lucene.experimental
+ * @deprecated Use {@link HMMChineseTokenizerFactory} instead
  */
+@Deprecated
 public class SmartChineseWordTokenFilterFactory extends TokenFilterFactory {
   
   /** Creates a new SmartChineseWordTokenFilterFactory */

Modified: lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/WordTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/WordTokenFilter.java?rev=1579846&r1=1579845&r2=1579846&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/WordTokenFilter.java (original)
+++ lucene/dev/trunk/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/WordTokenFilter.java Fri Mar 21 00:33:53 2014
@@ -31,7 +31,9 @@ import org.apache.lucene.analysis.tokena
 /**
  * A {@link TokenFilter} that breaks sentences into words.
  * @lucene.experimental
+ * @deprecated Use {@link HMMChineseTokenizer} instead.
  */
+@Deprecated
 public final class WordTokenFilter extends TokenFilter {
 
   private WordSegmenter wordSegmenter;

Modified: lucene/dev/trunk/lucene/analysis/smartcn/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenizerFactory
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/smartcn/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenizerFactory?rev=1579846&r1=1579845&r2=1579846&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/smartcn/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenizerFactory (original)
+++ lucene/dev/trunk/lucene/analysis/smartcn/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenizerFactory Fri Mar 21 00:33:53 2014
@@ -13,4 +13,5 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
+org.apache.lucene.analysis.cn.smart.HMMChineseTokenizerFactory
 org.apache.lucene.analysis.cn.smart.SmartChineseSentenceTokenizerFactory

Added: lucene/dev/trunk/lucene/analysis/smartcn/src/test/org/apache/lucene/analysis/cn/smart/TestHMMChineseTokenizerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/smartcn/src/test/org/apache/lucene/analysis/cn/smart/TestHMMChineseTokenizerFactory.java?rev=1579846&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/analysis/smartcn/src/test/org/apache/lucene/analysis/cn/smart/TestHMMChineseTokenizerFactory.java (added)
+++ lucene/dev/trunk/lucene/analysis/smartcn/src/test/org/apache/lucene/analysis/cn/smart/TestHMMChineseTokenizerFactory.java Fri Mar 21 00:33:53 2014
@@ -0,0 +1,56 @@
+package org.apache.lucene.analysis.cn.smart;
+
+/*
+ * 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 java.io.Reader;
+import java.io.StringReader;
+import java.util.HashMap;
+
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.util.TokenizerFactory;
+
+/** 
+ * Tests for {@link HMMChineseTokenizerFactory}
+ */
+public class TestHMMChineseTokenizerFactory extends BaseTokenStreamTestCase {
+  
+  /** Test showing the behavior */
+  public void testSimple() throws Exception {
+    Reader reader = new StringReader("我购买了道具和服装。");
+    TokenizerFactory factory = new HMMChineseTokenizerFactory(new HashMap<String,String>());
+    Tokenizer tokenizer = factory.create();
+    tokenizer.setReader(reader);
+    // TODO: fix smart chinese to not emit punctuation tokens
+    // at the moment: you have to clean up with WDF, or use the stoplist, etc
+    assertTokenStreamContents(tokenizer, 
+       new String[] { "我", "购买", "了", "道具", "和", "服装", "," });
+  }
+  
+  /** Test that bogus arguments result in exception */
+  public void testBogusArguments() throws Exception {
+    try {
+      new HMMChineseTokenizerFactory(new HashMap<String,String>() {{
+        put("bogusArg", "bogusValue");
+      }});
+      fail();
+    } catch (IllegalArgumentException expected) {
+      assertTrue(expected.getMessage().contains("Unknown parameters"));
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/analysis/smartcn/src/test/org/apache/lucene/analysis/cn/smart/TestSmartChineseFactories.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/smartcn/src/test/org/apache/lucene/analysis/cn/smart/TestSmartChineseFactories.java?rev=1579846&r1=1579845&r2=1579846&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/smartcn/src/test/org/apache/lucene/analysis/cn/smart/TestSmartChineseFactories.java (original)
+++ lucene/dev/trunk/lucene/analysis/smartcn/src/test/org/apache/lucene/analysis/cn/smart/TestSmartChineseFactories.java Fri Mar 21 00:33:53 2014
@@ -30,6 +30,7 @@ import org.apache.lucene.analysis.Tokeni
  * Tests for {@link SmartChineseSentenceTokenizerFactory} and 
  * {@link SmartChineseWordTokenFilterFactory}
  */
+@Deprecated
 public class TestSmartChineseFactories extends BaseTokenStreamTestCase {
   /** Test showing the behavior with whitespace */
   public void testSimple() throws Exception {