You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2013/05/27 00:13:32 UTC

svn commit: r1486483 - in /lucene/dev/branches/lucene5012/lucene/analysis/common/src: java/org/apache/lucene/analysis/stages/ test/org/apache/lucene/analysis/stages/

Author: mikemccand
Date: Sun May 26 22:13:31 2013
New Revision: 1486483

URL: http://svn.apache.org/r1486483
Log:
LUCENE-5012: add CharFilter, fix some bugs with SynFilter, add new InsertDeletedPunctuationStage

Added:
    lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/InsertDeletedPunctuationStage.java   (with props)
Modified:
    lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/AutomatonStage.java
    lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/CharTokenizerStage.java
    lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/Stage.java
    lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/StageAnalyzer.java
    lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/SynonymFilterStage.java
    lucene/dev/branches/lucene5012/lucene/analysis/common/src/test/org/apache/lucene/analysis/stages/TestStages.java

Modified: lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/AutomatonStage.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/AutomatonStage.java?rev=1486483&r1=1486482&r2=1486483&view=diff
==============================================================================
--- lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/AutomatonStage.java (original)
+++ lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/AutomatonStage.java Sun May 26 22:13:31 2013
@@ -98,6 +98,9 @@ public class AutomatonStage extends Stag
     if (prevStage.next()) {
       BytesRef term = termAtt.getBytesRef();
       termAtt.fillBytesRef();
+      if (term.length == 0) {
+        throw new IllegalStateException("cannot handle empty-string term");
+      }
       State lastState = getFromState(arcAtt.from());
       for(int i=0;i<term.length;i++) {
         State toState;

Modified: lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/CharTokenizerStage.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/CharTokenizerStage.java?rev=1486483&r1=1486482&r2=1486483&view=diff
==============================================================================
--- lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/CharTokenizerStage.java (original)
+++ lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/CharTokenizerStage.java Sun May 26 22:13:31 2013
@@ -21,11 +21,8 @@ import java.io.IOException;
 import java.io.Reader;
 
 import org.apache.lucene.analysis.tokenattributes.ArcAttribute;
-import org.apache.lucene.analysis.tokenattributes.ArcAttributeImpl;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
-import org.apache.lucene.analysis.tokenattributes.CharTermAttributeImpl;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
-import org.apache.lucene.analysis.tokenattributes.OffsetAttributeImpl;
 import org.apache.lucene.analysis.util.CharacterUtils.CharacterBuffer;
 import org.apache.lucene.analysis.util.CharacterUtils;
 import org.apache.lucene.util.Attribute;
@@ -86,9 +83,7 @@ public abstract class CharTokenizerStage
             break;
           } else {
             // set final offset
-            // nocommit -- get charfilter working:
-            //final int finalOffset = correctOffset(offset);
-            final int finalOffset = offset;
+            final int finalOffset = correctOffset(offset);
             offsetAtt.setOffset(finalOffset, finalOffset);
             return false;
           }
@@ -117,9 +112,7 @@ public abstract class CharTokenizerStage
 
     termAtt.setLength(length);
 
-    // nocommit -- get charfilter working:
-    //offsetAtt.setOffset(correctOffset(start), correctOffset(start+length));
-    offsetAtt.setOffset(start, start+length);
+    offsetAtt.setOffset(correctOffset(start), correctOffset(start+length));
 
     int node = nodes.newNode();
     arcAtt.set(lastNode, node);

Added: lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/InsertDeletedPunctuationStage.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/InsertDeletedPunctuationStage.java?rev=1486483&view=auto
==============================================================================
--- lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/InsertDeletedPunctuationStage.java (added)
+++ lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/InsertDeletedPunctuationStage.java Sun May 26 22:13:31 2013
@@ -0,0 +1,173 @@
+package org.apache.lucene.analysis.stages;
+
+/*
+ * 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 org.apache.lucene.analysis.CharFilter;
+import org.apache.lucene.analysis.tokenattributes.ArcAttribute;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.DeletedAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.FixedBitSet;
+
+/** Uses a CharFilter to detect when punction occurs in the
+ *  input in between two tokens, and then as a Stage it will
+ *  re-insert [deleted] tokens when it notices the tokenizer
+ *  had deleted the punctuation.  E.g. this can be used to
+ *  prevent synonyms/phrases from matching across punctuation. */
+
+public class InsertDeletedPunctuationStage extends Stage {
+
+  private final DeletedAttribute delAttIn;
+  private final ArcAttribute arcAttIn;
+  private final CharTermAttribute termAttIn;
+  private final OffsetAttribute offsetAttIn;
+
+  private final ArcAttribute arcAttOut;
+  private final DeletedAttribute delAttOut;
+  private final CharTermAttribute termAttOut;
+  private final OffsetAttribute offsetAttOut;
+
+  private final String punctToken;
+
+  public InsertDeletedPunctuationStage(Stage prevStage, String punctToken) {
+    super(prevStage);
+    this.punctToken = punctToken;
+
+    delAttIn = prevStage.get(DeletedAttribute.class);
+    offsetAttIn = prevStage.get(OffsetAttribute.class);
+    arcAttIn = prevStage.get(ArcAttribute.class);
+    termAttIn = prevStage.get(CharTermAttribute.class);
+
+    delAttOut = create(DeletedAttribute.class);
+    offsetAttOut = create(OffsetAttribute.class);
+    arcAttOut = create(ArcAttribute.class);
+    termAttOut = create(CharTermAttribute.class);
+  }
+
+  private static class FindPunctuationCharFilter extends CharFilter {
+    FixedBitSet wasPunct = new FixedBitSet(128);
+    private int pos;
+
+    public FindPunctuationCharFilter(Reader input) {
+      super(input);
+    }
+
+    @Override
+    protected int correct(int offset) {
+      return offset;
+    }
+
+    @Override
+    public int read(char[] buffer, int offset, int length) throws IOException {
+      int count = input.read(buffer, offset, length);
+      for(int i=0;i<count;i++) {
+        if (isPunct(buffer[offset+i])) {
+          if (wasPunct.length() <= pos) {
+            int nextSize = ArrayUtil.oversize(pos+1, 1);
+            FixedBitSet nextBits = new FixedBitSet(nextSize);
+            nextBits.or(wasPunct);
+            wasPunct = nextBits;
+          }
+          wasPunct.set(pos);
+        }
+        pos++;
+      }
+
+      return count;
+    }
+
+    protected boolean isPunct(char ch) {
+      // TODO: use proper Character.isXXX apis:
+      return ch == '.' || ch == ',' || ch == ':' || ch == ';';
+    }
+  }
+
+  @Override
+  public void reset(Reader input) {
+    // nocommit this is iffy?  if an earlier stage also
+    // wraps, then, we are different offsets
+    charFilter = new FindPunctuationCharFilter(input);
+    super.reset(charFilter);
+    lastEndOffset = 0;
+    lastPunct = false;
+    nodeOffset = 0;
+  }
+
+  private FindPunctuationCharFilter charFilter;
+  private boolean lastPunct;
+  private int lastEndOffset;
+  private int nodeOffset;
+
+  @Override
+  public boolean next() throws IOException {
+    if (lastPunct) {
+      // Return previously buffered token:
+      copyToken();
+      lastPunct = false;
+      return true;
+    }
+
+    if (prevStage.next()) {
+      int startOffset = offsetAttIn.startOffset();
+      assert startOffset <= charFilter.wasPunct.length();
+      for(int i=lastEndOffset;i<startOffset;i++) {
+        if (charFilter.wasPunct.get(i)) {
+          // The gap between the end of the last token,
+          // and this token, had punctuation:
+          lastPunct = true;
+          break;
+        }
+      }
+
+      if (lastPunct) {
+        // We insert a new node and token here:
+
+        // nocommit this (single int nodeOffset) is too simplistic?
+        arcAttOut.set(arcAttIn.from() + nodeOffset, arcAttIn.from() + nodeOffset + 1);
+        delAttOut.set(true);
+        offsetAttOut.setOffset(lastEndOffset, startOffset);
+        // nocommit: should we copy over the actual punct chars...?
+        termAttOut.setEmpty();
+        termAttOut.append(punctToken);
+        nodeOffset++;
+      } else {
+        copyToken();
+      }
+      lastEndOffset = offsetAttIn.endOffset();
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  private void copyToken() {
+    if (delAttIn != null) {
+      delAttOut.set(delAttIn.deleted());
+    } else {
+      delAttOut.set(false);
+    }
+    termAttOut.setEmpty();
+    termAttOut.append(termAttIn);
+    offsetAttOut.setOffset(offsetAttIn.startOffset(), offsetAttIn.endOffset());
+    arcAttOut.set(arcAttIn.from()+nodeOffset, arcAttIn.to() + nodeOffset);
+  }
+}

Modified: lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/Stage.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/Stage.java?rev=1486483&r1=1486482&r2=1486483&view=diff
==============================================================================
--- lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/Stage.java (original)
+++ lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/Stage.java Sun May 26 22:13:31 2013
@@ -20,12 +20,11 @@ package org.apache.lucene.analysis.stage
 import java.io.IOException;
 import java.io.Reader;
 import java.lang.ref.WeakReference;
-import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.LinkedList;
-import java.util.List;
 import java.util.Map;
 
+import org.apache.lucene.analysis.CharFilter;
 import org.apache.lucene.util.Attribute;
 import org.apache.lucene.util.AttributeImpl;
 import org.apache.lucene.util.WeakIdentityMap;
@@ -39,7 +38,8 @@ public abstract class Stage {
   protected final NodeTracker nodes;
 
   // nocommit is all this hair really worth the separation
-  // of interface from impl?
+  // of interface from impl?  can we just have concrete
+  // attrs?
 
   private static final WeakIdentityMap<Class<? extends Attribute>, WeakReference<Class<? extends AttributeImpl>>> attClassImplMap =
     WeakIdentityMap.newConcurrentHashMap(false);
@@ -173,16 +173,30 @@ public abstract class Stage {
 
   public abstract boolean next() throws IOException;
 
+  // Only set for first Stage in a chain:
+  private Reader input;
+
   public void reset(Reader reader) {
     if (prevStage != null) {
       prevStage.reset(reader);
     } else {
       nodes.reset();
+      input = reader;
     }
   }
 
+  protected final int correctOffset(int currentOff) {
+    // nocommit should we strongly type this (like
+    // Tokenizer/TokenFilter today)?
+    if (input == null) {
+      throw new IllegalStateException("only first Stage can call correctOffset");
+    }
+    return (input instanceof CharFilter) ? ((CharFilter) input).correctOffset(currentOff) : currentOff;
+  }
+
+  // nocommit should we impl close()?  why?
+
   public boolean anyNodesCanChange() {
-    System.out.println("    nodes=" + nodes);
     return nodes.anyNodesCanChange();
   }
 }

Modified: lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/StageAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/StageAnalyzer.java?rev=1486483&r1=1486482&r2=1486483&view=diff
==============================================================================
--- lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/StageAnalyzer.java (original)
+++ lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/StageAnalyzer.java Sun May 26 22:13:31 2013
@@ -26,12 +26,13 @@ import org.apache.lucene.analysis.Analyz
 public abstract class StageAnalyzer extends Analyzer {
 
   @Override
-  protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+  protected TokenStreamComponents createComponents(String fieldName, Reader firstReader) {
     final Stage stage = getStages();
     // NOTE: it's really weird that this method takes a
     // reader in!  Building up the pipeline, and resetting
     // it w/ new Reader should be decoupled...
-    stage.reset(reader);
+    stage.reset(firstReader);
+
     return new TokenStreamComponents(null, new StageToTokenStream(stage)) {
       @Override
       protected void setReader(final Reader reader) throws IOException {

Modified: lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/SynonymFilterStage.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/SynonymFilterStage.java?rev=1486483&r1=1486482&r2=1486483&view=diff
==============================================================================
--- lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/SynonymFilterStage.java (original)
+++ lucene/dev/branches/lucene5012/lucene/analysis/common/src/java/org/apache/lucene/analysis/stages/SynonymFilterStage.java Sun May 26 22:13:31 2013
@@ -25,12 +25,8 @@ import java.util.List;
 
 import org.apache.lucene.analysis.synonym.SynonymMap;
 import org.apache.lucene.analysis.tokenattributes.ArcAttribute;
-import org.apache.lucene.analysis.tokenattributes.ArcAttributeImpl;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
-import org.apache.lucene.analysis.tokenattributes.CharTermAttributeImpl;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
-import org.apache.lucene.analysis.tokenattributes.OffsetAttributeImpl;
-import org.apache.lucene.analysis.util.CharacterUtils;
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.util.Attribute;
 import org.apache.lucene.util.BytesRef;
@@ -135,7 +131,12 @@ public class SynonymFilterStage extends 
 
   // nocommit we need reset!  make test that fails if only
   // partial consume
-
+  @Override
+  public void reset(Reader reader) {
+    super.reset(reader);
+    pending.clear();
+    pendingOutputs.clear();
+  }
 
   /** Extends all pending paths, and starts new paths,
    *  matching the current token. */
@@ -327,6 +328,11 @@ public class SynonymFilterStage extends 
                     arcAttIn.to());
       return true;
     } else {
+      // Prune any remaining partial matches:
+      for(int i=0;i<pending.size();i++) {
+        nodes.wontChange(pending.get(i).fromNode);
+      }
+      pending.clear();
       return false;
     }
   }

Modified: lucene/dev/branches/lucene5012/lucene/analysis/common/src/test/org/apache/lucene/analysis/stages/TestStages.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5012/lucene/analysis/common/src/test/org/apache/lucene/analysis/stages/TestStages.java?rev=1486483&r1=1486482&r2=1486483&view=diff
==============================================================================
--- lucene/dev/branches/lucene5012/lucene/analysis/common/src/test/org/apache/lucene/analysis/stages/TestStages.java (original)
+++ lucene/dev/branches/lucene5012/lucene/analysis/common/src/test/org/apache/lucene/analysis/stages/TestStages.java Sun May 26 22:13:31 2013
@@ -18,20 +18,20 @@ package org.apache.lucene.analysis.stage
  */
 
 import java.io.IOException;
+import java.io.Reader;
 import java.io.StringReader;
 import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.CharFilter;
 import org.apache.lucene.analysis.synonym.SynonymMap;
 import org.apache.lucene.analysis.tokenattributes.ArcAttribute;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
-import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.util.CharArraySet;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
-import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.BasicOperations;
 import org.apache.lucene.util.automaton.State;
@@ -78,14 +78,15 @@ public class TestStages extends BaseToke
   private void assertMatches(String text, Stage end, String... expectedStrings) throws IOException {
     AutomatonStage a = new AutomatonStage(new AssertingStage(end));
     CharTermAttribute termAtt = a.get(CharTermAttribute.class);
+    ArcAttribute arcAtt = a.get(ArcAttribute.class);
     for(int i=0;i<2;i++) {
       a.reset(new StringReader(text));
       while (a.next()) {
-        System.out.println("token=" + termAtt);
+        //System.out.println("token=" + termAtt + " from=" + arcAtt.from() + " to=" + arcAtt.to());
       }
       assertMatches(a.getAutomaton(), expectedStrings);
+      assertFalse(a.anyNodesCanChange());
     }
-    assertFalse(a.anyNodesCanChange());
   }
 
   public void testBasic() throws Exception {
@@ -122,6 +123,14 @@ public class TestStages extends BaseToke
                   "a b c foo", "x foo");
   }
 
+  public void testSyn2() throws Exception {
+    SynonymMap.Builder b = new SynonymMap.Builder(true);
+    add(b, "a b c", "x");
+    assertMatches("a b c",
+                  new SynonymFilterStage(new WhitespaceTokenizerStage(), b.build(), true),
+                  "a b c", "x");
+  }
+
   public void testSynAfterDecompound() throws Exception {
     SynonymMap.Builder b = new SynonymMap.Builder(true);
     add(b, "a b c", "x");
@@ -229,4 +238,67 @@ public class TestStages extends BaseToke
                      null,
                      new int[] {1, 1, 1});
   }
+
+  public class SillyCharFilter extends CharFilter {
+    public SillyCharFilter(Reader input) {
+      super(input);
+    }
+
+    @Override
+    public int read(char[] buffer, int offset, int length) throws IOException {
+      return input.read(buffer, offset, length);
+    }
+
+    @Override
+    protected int correct(int currentOff) {
+      return currentOff+1;
+    }
+  }
+
+  public void testCharFilter() throws Exception {
+    Analyzer a = new StageAnalyzer() {
+        @Override
+        protected Stage getStages() {
+          return new LowerCaseFilterStage(TEST_VERSION_CURRENT, new WhitespaceTokenizerStage());
+        }
+
+        @Override
+        protected Reader initReader(String fieldName, Reader input) {
+          return new SillyCharFilter(input);
+        }
+      };
+
+    // Same as testBasic, but all offsets
+    // (incl. finalOffset) have been "corrected" by +1:
+    assertTokenStreamContents(a.tokenStream("dummy", new StringReader("This is a test")),
+                              new String[] {"this", "is", "a", "test"},
+                              new int[] {1, 6, 9, 11},
+                              new int[] {5, 8, 10, 15},
+                              null,
+                              new int[] {1, 1, 1, 1},
+                              null,
+                              15);
+  }
+
+  static class WhitespaceOrPunctTokenizerStage extends CharTokenizerStage {
+    @Override
+    protected boolean isTokenChar(int c) {
+      return !Character.isWhitespace(c) && c != ',';
+    }
+  }
+
+  public void testInsertDeletedPunctuation() throws Exception {
+    SynonymMap.Builder b = new SynonymMap.Builder(true);
+    add(b, "a b c", "x");
+
+    Stage s = new SynonymFilterStage(new InsertDeletedPunctuationStage(new LowerCaseFilterStage(TEST_VERSION_CURRENT, new WhitespaceOrPunctTokenizerStage()), "p"),
+                                     b.build(), true);
+
+    // comma prevents syn match, even though tokenizer
+    // skipped it:
+    assertMatches("a, b c", s, "a p b c");
+
+    // no comma allows syn match:
+    assertMatches("a b c", s, "a b c", "x");
+  }
 }