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 2011/08/02 15:42:30 UTC

svn commit: r1153120 - in /lucene/dev/branches/blocktree_3030/lucene/src: java/org/apache/lucene/index/codecs/ java/org/apache/lucene/search/ java/org/apache/lucene/util/automaton/ test/org/apache/lucene/index/

Author: mikemccand
Date: Tue Aug  2 13:42:29 2011
New Revision: 1153120

URL: http://svn.apache.org/viewvc?rev=1153120&view=rev
Log:
LUCENE-3030: forgot to add CompiledAutomaton.java

Added:
    lucene/dev/branches/blocktree_3030/lucene/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java   (with props)
Modified:
    lucene/dev/branches/blocktree_3030/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java
    lucene/dev/branches/blocktree_3030/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsWriter.java
    lucene/dev/branches/blocktree_3030/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
    lucene/dev/branches/blocktree_3030/lucene/src/test/org/apache/lucene/index/TestTermsEnum.java

Modified: lucene/dev/branches/blocktree_3030/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/blocktree_3030/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java?rev=1153120&r1=1153119&r2=1153120&view=diff
==============================================================================
--- lucene/dev/branches/blocktree_3030/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java (original)
+++ lucene/dev/branches/blocktree_3030/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java Tue Aug  2 13:42:29 2011
@@ -634,7 +634,39 @@ public class BlockTreeTermsReader extend
 
         void load(BytesRef frameIndexData) throws IOException {
 
-          if (DEBUG) System.out.println("    load fp=" + fp + " fpOrig=" + fpOrig);
+          if (DEBUG) System.out.println("    load fp=" + fp + " fpOrig=" + fpOrig + " frameIndexData=" + frameIndexData);
+
+          if (frameIndexData != null && transitions.length != 0) {
+            // Floor frame
+            if (floorData.length < frameIndexData.length) {
+              this.floorData = new byte[ArrayUtil.oversize(frameIndexData.length, 1)];
+            }
+            System.arraycopy(frameIndexData.bytes, frameIndexData.offset, floorData, 0, frameIndexData.length);
+            floorDataReader.reset(floorData, 0, frameIndexData.length);
+            // Skip first long -- has redundant fp, hasTerms
+            // flag, isFloor flag
+            final long code = floorDataReader.readVLong();
+            if ((code & OUTPUT_FLAG_IS_FLOOR) != 0) {
+              numFollowFloorBlocks = floorDataReader.readVInt();
+              nextFloorLabel = floorDataReader.readByte() & 0xff;
+              if (DEBUG) System.out.println("    numFollowFloorBlocks=" + numFollowFloorBlocks + " nextFloorLabel=" + nextFloorLabel);
+              // Maybe skip floor blocks:
+              // nocommit -- must set nextFloorLabel to 256
+              // if we exhasuted it?
+              while (numFollowFloorBlocks != 0 && nextFloorLabel <= transitions[0].getMin()) {
+                if (DEBUG) System.out.println("    skip floor block!");
+                final long code2 = floorDataReader.readVLong();
+                fp = fpOrig + (code2 >>> 1);
+                numFollowFloorBlocks--;
+                if (numFollowFloorBlocks != 0) {
+                  nextFloorLabel = floorDataReader.readByte() & 0xff;
+                } else {
+                  nextFloorLabel = 256;
+                }
+              }
+            }
+          }
+
           in.seek(fp);
           int code = in.readVInt();
           entCount = code >> 1;
@@ -661,22 +693,6 @@ public class BlockTreeTermsReader extend
           statsReader.reset(statBytes, 0, numBytes);
           metaDataUpto = 0;
 
-          if (frameIndexData != null && !isLastInFloor) {
-            // Floor frame
-            if (floorData.length < frameIndexData.length) {
-              this.floorData = new byte[ArrayUtil.oversize(frameIndexData.length, 1)];
-            }
-            if (DEBUG) System.out.println("    frameIndexData=" + frameIndexData);
-            System.arraycopy(frameIndexData.bytes, frameIndexData.offset, floorData, 0, frameIndexData.length);
-            floorDataReader.reset(floorData, 0, frameIndexData.length);
-            // Skip first long -- has redundant fp, hasTerms
-            // flag, isFloor flag
-            floorDataReader.readVLong();
-            numFollowFloorBlocks = floorDataReader.readVInt();
-            nextFloorLabel = floorDataReader.readByte() & 0xff;
-            if (DEBUG) System.out.println("    numFollowFloorBlocks=" + numFollowFloorBlocks + " nextFloorLabel=" + nextFloorLabel);
-          }
-
           termState.termBlockOrd = 0;
           nextEnt = 0;
           
@@ -991,6 +1007,8 @@ public class BlockTreeTermsReader extend
             }
           }
         }
+
+        assert false;
       }
 
       @Override

Modified: lucene/dev/branches/blocktree_3030/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/blocktree_3030/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsWriter.java?rev=1153120&r1=1153119&r2=1153120&view=diff
==============================================================================
--- lucene/dev/branches/blocktree_3030/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsWriter.java (original)
+++ lucene/dev/branches/blocktree_3030/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsWriter.java Tue Aug  2 13:42:29 2011
@@ -88,7 +88,7 @@ import org.apache.lucene.util.fst.Util;
 
 public class BlockTreeTermsWriter extends FieldsConsumer {
 
-  public static boolean DEBUG = false;
+  public static boolean DEBUG = true;
   public static boolean DEBUG2 = false;
   public static boolean SAVE_DOT_FILES = false;
 

Modified: lucene/dev/branches/blocktree_3030/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/blocktree_3030/lucene/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1153120&r1=1153119&r2=1153120&view=diff
==============================================================================
--- lucene/dev/branches/blocktree_3030/lucene/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/blocktree_3030/lucene/src/java/org/apache/lucene/search/IndexSearcher.java Tue Aug  2 13:42:29 2011
@@ -850,7 +850,9 @@ public class IndexSearcher implements Cl
       } catch (InterruptedException e) {
         throw new ThreadInterruptedException(e);
       } catch (ExecutionException e) {
-        throw new RuntimeException(e);
+        // nocommit
+        e.getCause().printStackTrace(System.out);
+        throw new RuntimeException(e.getCause());
       } finally {
         --numTasks;
       }

Added: lucene/dev/branches/blocktree_3030/lucene/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/blocktree_3030/lucene/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java?rev=1153120&view=auto
==============================================================================
--- lucene/dev/branches/blocktree_3030/lucene/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java (added)
+++ lucene/dev/branches/blocktree_3030/lucene/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java Tue Aug  2 13:42:29 2011
@@ -0,0 +1,221 @@
+package org.apache.lucene.util.automaton;
+
+/**
+ * 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.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.index.codecs.BlockTreeTermsWriter;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Immutable class holding compiled details for a given
+ * Automaton.  The Automaton is deterministic, must not have
+ * dead states but may not be minimal.
+ *
+ * @lucene.experimental
+ */
+public class CompiledAutomaton {
+  public final ByteRunAutomaton runAutomaton;
+  // TODO: would be nice if these sortedTransitions had "int
+  // to;" instead of "State to;" somehow:
+  public final Transition[][] sortedTransitions;
+  public final BytesRef commonSuffixRef;
+  public final boolean finite;
+
+  public final Automaton a;
+  public final Automaton utf8;
+
+  // nocommit -- move pulling of a TermsEnum into here, so
+  // that we can optimize for cases where a simpler enume
+  // (prefix enum, all terms, no terms, etc.) can be used
+    
+  public CompiledAutomaton(Automaton automaton, boolean finite) {
+    // nocommit
+    this.a = automaton;
+    Automaton utf8 = new UTF32ToUTF8().convert(automaton);
+    // nocommit
+    this.utf8 = utf8;
+    runAutomaton = new ByteRunAutomaton(utf8, true);
+    sortedTransitions = utf8.getSortedTransitions();
+    this.finite = finite;
+    if (finite) {
+      commonSuffixRef = null;
+    } else {
+      commonSuffixRef = SpecialOperations.getCommonSuffixBytesRef(utf8);
+    }
+  }
+
+  private static final boolean DEBUG = BlockTreeTermsWriter.DEBUG;
+
+  // nocommit -- needs tests; make sure we test infinite
+  // case (should just work?)
+
+  private BytesRef addTail(int state, BytesRef term, int idx, int leadLabel) {
+
+    // Find biggest transition that's < label
+    // TODO: use binary search here
+    Transition maxTransition = null;
+    for (Transition transition : sortedTransitions[state]) {
+      if (transition.min < leadLabel) {
+        maxTransition = transition;
+      }
+    }
+
+    assert maxTransition != null;
+
+    // Append floorLabel
+    final int floorLabel;
+    if (maxTransition.max > leadLabel-1) {
+      floorLabel = leadLabel-1;
+    } else {
+      floorLabel = maxTransition.max;
+    }
+    if (idx >= term.bytes.length) {
+      term.grow(1+idx);
+    }
+    if (DEBUG) System.out.println("  add floorLabel=" + (char) floorLabel + " idx=" + idx);
+    term.bytes[idx] = (byte) floorLabel;
+
+    state = maxTransition.to.getNumber();
+    idx++;
+
+    // Push down to last accept state
+    while (true) {
+      Transition[] transitions = sortedTransitions[state];
+      if (transitions.length == 0) {
+        assert runAutomaton.isAccept(state);
+        term.length = idx;
+        if (DEBUG) System.out.println("  return " + term.utf8ToString());
+        return term;
+      } else {
+        // We are pushing "top" -- so get last label of
+        // last transition:
+        assert transitions.length != 0;
+        Transition lastTransition = transitions[transitions.length-1];
+        if (idx >= term.bytes.length) {
+          term.grow(1+idx);
+        }
+        if (DEBUG) System.out.println("  push maxLabel=" + (char) lastTransition.max + " idx=" + idx);
+        term.bytes[idx] = (byte) lastTransition.max;
+        state = lastTransition.to.getNumber();
+        idx++;
+      }
+    }
+  }
+
+  /** Finds largest term accepted by this Automaton, that's
+   *  <= the provided input term.  The result is placed in
+   *  output; it's fine for output and input to point to
+   *  the same BytesRef.  The returned result is either the
+   *  provided output, or null if there is no floor term
+   *  (ie, the provided input term is before the first term
+   *  accepted by this Automaton). */
+  public BytesRef floor(BytesRef input, BytesRef output) {
+
+    // nocommit make sure we test empty string
+
+    output.offset = 0;
+    if (DEBUG) System.out.println("CA.floor input=" + input.utf8ToString());
+
+    int state = runAutomaton.getInitialState();
+
+    // Special case empty string:
+    if (input.length == 0) {
+      if (runAutomaton.isAccept(state)) {
+        output.length = 0;
+        return output;
+      } else {
+        return null;
+      }
+    }
+
+    final List<Integer> stack = new ArrayList<Integer>();
+
+    int idx = 0;
+    while (true) {
+      int label = input.bytes[input.offset + idx] & 0xff;
+      int nextState = runAutomaton.step(state, label);
+      if (DEBUG) System.out.println("  cycle label=" + (char) label + " nextState=" + nextState);
+
+      if (idx == input.length-1) {
+        if (nextState != -1 && runAutomaton.isAccept(nextState)) {
+          // Input string is accepted
+          if (idx >= output.bytes.length) {
+            output.grow(1+idx);
+          }
+          output.bytes[idx] = (byte) label;
+          output.length = input.length;
+          if (DEBUG) System.out.println("  input is accepted; return term=" + output.utf8ToString());
+          return output;
+        } else {
+          nextState = -1;
+        }
+      }
+
+      if (nextState == -1) {
+
+        // Pop back to a state that has a transition
+        // <= our label:
+        while (true) {
+          Transition[] transitions = sortedTransitions[state];
+          if (transitions.length == 0) {
+            assert runAutomaton.isAccept(state);
+            output.length = idx;
+            if (DEBUG) System.out.println("  return " + output.utf8ToString());
+            return output;
+          } else if (label-1 < transitions[0].min) {
+
+            if (runAutomaton.isAccept(state)) {
+              output.length = idx;
+              if (DEBUG) System.out.println("  return " + output.utf8ToString());
+              return output;
+            }
+            // pop
+            if (stack.size() == 0) {
+              if (DEBUG) System.out.println("  pop ord=" + idx + " return null");
+              return null;
+            } else {
+              state = stack.remove(stack.size()-1);
+              idx--;
+              if (DEBUG) System.out.println("  pop ord=" + (idx+1) + " label=" + (char) label + " first trans.min=" + (char) transitions[0].min);
+              label = input.bytes[input.offset + idx] & 0xff;
+            }
+
+          } else {
+            if (DEBUG) System.out.println("  stop pop ord=" + idx + " first trans.min=" + (char) transitions[0].min);
+            break;
+          }
+        }
+
+        if (DEBUG) System.out.println("  label=" + (char) label + " idx=" + idx);
+
+        return addTail(state, output, idx, label);
+        
+      } else {
+        if (idx >= output.bytes.length) {
+          output.grow(1+idx);
+        }
+        output.bytes[idx] = (byte) label;
+        stack.add(state);
+        state = nextState;
+        idx++;
+      }
+    }
+  }
+}

Modified: lucene/dev/branches/blocktree_3030/lucene/src/test/org/apache/lucene/index/TestTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/blocktree_3030/lucene/src/test/org/apache/lucene/index/TestTermsEnum.java?rev=1153120&r1=1153119&r2=1153120&view=diff
==============================================================================
--- lucene/dev/branches/blocktree_3030/lucene/src/test/org/apache/lucene/index/TestTermsEnum.java (original)
+++ lucene/dev/branches/blocktree_3030/lucene/src/test/org/apache/lucene/index/TestTermsEnum.java Tue Aug  2 13:42:29 2011
@@ -181,9 +181,7 @@ public class TestTermsEnum extends Lucen
     final Directory dir = newDirectory();
     final RandomIndexWriter w = new RandomIndexWriter(random, dir);
 
-    // nocommit
-    //final int numTerms = atLeast(1000);
-    final int numTerms = atLeast(10);
+    final int numTerms = atLeast(1000);
 
     final Set<String> terms = new HashSet<String>();
     final Collection<String> pendingTerms = new ArrayList<String>();