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 2011/02/06 19:17:01 UTC

svn commit: r1067720 - in /lucene/dev/trunk/lucene/src/java/org/apache/lucene: search/ util/automaton/

Author: rmuir
Date: Sun Feb  6 18:17:00 2011
New Revision: 1067720

URL: http://svn.apache.org/viewvc?rev=1067720&view=rev
Log:
LUCENE-2907: fix automaton thread hazard: change query to compute immutable compiled form in ctor

Modified:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/Automaton.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/ByteRunAutomaton.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/RunAutomaton.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java?rev=1067720&r1=1067719&r2=1067720&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java Sun Feb  6 18:17:00 2011
@@ -18,15 +18,15 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
+import java.io.Serializable;
 
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.AutomatonTermsEnum.CompiledAutomaton;
 import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.AttributeSource;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.BasicAutomata;
 import org.apache.lucene.util.automaton.BasicOperations;
 import org.apache.lucene.util.automaton.MinimizationOperations;
@@ -56,9 +56,16 @@ public class AutomatonQuery extends Mult
   /** term containing the field, and possibly some pattern structure */
   protected final Term term;
 
-  transient ByteRunAutomaton runAutomaton;
-  transient boolean isFinite;
-  transient BytesRef commonSuffixRef;
+  /** 
+   * abstraction for returning a termsenum:
+   * in the ctor the query computes one of these, the actual
+   * implementation depends upon the automaton's structure.
+   */
+  private abstract class TermsEnumFactory implements Serializable {
+    protected abstract TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException;
+  }
+  
+  private final TermsEnumFactory factory;
 
   /**
    * Create a new AutomatonQuery from an {@link Automaton}.
@@ -68,60 +75,77 @@ public class AutomatonQuery extends Mult
    * @param automaton Automaton to run, terms that are accepted are considered a
    *        match.
    */
-  public AutomatonQuery(Term term, Automaton automaton) {
+  public AutomatonQuery(final Term term, Automaton automaton) {
     super(term.field());
     this.term = term;
     this.automaton = automaton;
     MinimizationOperations.minimize(automaton);
-  }
-
-  private synchronized void compileAutomaton() {
-    // this method must be synchronized, as setting the three transient fields is not atomic:
-    if (runAutomaton == null) {
-      runAutomaton = new ByteRunAutomaton(automaton);
-      isFinite = SpecialOperations.isFinite(automaton);
-      commonSuffixRef = isFinite ? null : SpecialOperations.getCommonSuffixBytesRef(runAutomaton.getAutomaton());
-    }
-  }
-
-  @Override
-  protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-    // matches nothing
-    if (BasicOperations.isEmpty(automaton)) {
-      return TermsEnum.EMPTY;
-    }
-    
-    TermsEnum tenum = terms.iterator();
     
-    // matches all possible strings
-    if (BasicOperations.isTotal(automaton)) {
-      return tenum;
-    }
-    
-    // matches a fixed string in singleton representation
-    String singleton = automaton.getSingleton();
-    if (singleton != null)
-      return new SingleTermsEnum(tenum, term.createTerm(singleton));
-
-    // matches a fixed string in expanded representation
-    final String commonPrefix = SpecialOperations.getCommonPrefix(automaton);
-
-    if (commonPrefix.length() > 0) {
-      if (BasicOperations.sameLanguage(automaton, BasicAutomata.makeString(commonPrefix))) {
-        return new SingleTermsEnum(tenum, term.createTerm(commonPrefix));
+    if (BasicOperations.isEmpty(automaton)) {
+      // matches nothing
+      factory = new TermsEnumFactory() {
+        @Override
+        protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
+          return TermsEnum.EMPTY;
+        }
+      };
+    } else if (BasicOperations.isTotal(automaton)) {
+      // matches all possible strings
+      factory = new TermsEnumFactory() {
+        @Override
+        protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
+          return terms.iterator();
+        }
+      };
+    } else {
+      final String singleton;
+      final String commonPrefix;
+      
+      if (automaton.getSingleton() == null) {
+        commonPrefix = SpecialOperations.getCommonPrefix(automaton);
+        if (commonPrefix.length() > 0 && BasicOperations.sameLanguage(automaton, BasicAutomata.makeString(commonPrefix))) {
+          singleton = commonPrefix;
+        } else {
+          singleton = null;
+        }
+      } else {
+        commonPrefix = null;
+        singleton = automaton.getSingleton();
       }
-    
-      // matches a constant prefix
-      Automaton prefixAutomaton = BasicOperations.concatenate(BasicAutomata
-                                                              .makeString(commonPrefix), BasicAutomata.makeAnyString());
-      if (BasicOperations.sameLanguage(automaton, prefixAutomaton)) {
-        return new PrefixTermsEnum(tenum, term.createTerm(commonPrefix));
+      
+      if (singleton != null) {
+        // matches a fixed string in singleton or expanded representation
+        factory = new TermsEnumFactory() {
+          @Override
+          protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
+            return new SingleTermsEnum(terms.iterator(), term.createTerm(singleton));
+          }
+        };
+      } else if (BasicOperations.sameLanguage(automaton, BasicOperations.concatenate(
+          BasicAutomata.makeString(commonPrefix), BasicAutomata.makeAnyString()))) {
+        // matches a constant prefix
+        factory = new TermsEnumFactory() {
+          @Override
+          protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
+            return new PrefixTermsEnum(terms.iterator(), term.createTerm(commonPrefix));
+          }
+        };
+      } else {
+        final AutomatonTermsEnum.CompiledAutomaton compiled = 
+          new CompiledAutomaton(automaton, SpecialOperations.isFinite(automaton));
+        factory = new TermsEnumFactory() {
+          @Override
+          protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
+            return new AutomatonTermsEnum(terms.iterator(), compiled);
+          }
+        };
       }
     }
+  }
 
-    compileAutomaton();
-    
-    return new AutomatonTermsEnum(runAutomaton, tenum, isFinite, commonSuffixRef);
+  @Override
+  protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
+    return factory.getTermsEnum(terms, atts);
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java?rev=1067720&r1=1067719&r2=1067720&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java Sun Feb  6 18:17:00 2011
@@ -27,6 +27,7 @@ import org.apache.lucene.util.automaton.
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.SpecialOperations;
 import org.apache.lucene.util.automaton.Transition;
+import org.apache.lucene.util.automaton.UTF32ToUTF8;
 
 /**
  * A FilteredTermsEnum that enumerates terms based upon what is accepted by a
@@ -46,8 +47,6 @@ import org.apache.lucene.util.automaton.
  * @lucene.experimental
  */
 public class AutomatonTermsEnum extends FilteredTermsEnum {
-  // the object-oriented form of the DFA
-  private final Automaton automaton;
   // a tableized array-based form of the DFA
   private final ByteRunAutomaton runAutomaton;
   // common suffix of the automaton
@@ -71,37 +70,20 @@ public class AutomatonTermsEnum extends 
   private final Comparator<BytesRef> termComp;
 
   /**
-   * Expert ctor:
    * Construct an enumerator based upon an automaton, enumerating the specified
    * field, working on a supplied TermsEnum
    * <p>
    * @lucene.experimental 
    * <p>
-   * @param runAutomaton pre-compiled ByteRunAutomaton
-   * @param finite true if the automaton accepts a finite language
+   * @param compiled CompiledAutomaton
    */
-  public AutomatonTermsEnum(ByteRunAutomaton runAutomaton,
-                     TermsEnum tenum,
-                     boolean finite, BytesRef commonSuffixRef)
-      throws IOException {
+  public AutomatonTermsEnum(TermsEnum tenum, CompiledAutomaton compiled) throws IOException {
     super(tenum);
-    this.automaton = runAutomaton.getAutomaton();
-    this.finite = finite;
+    this.finite = compiled.finite;
+    this.runAutomaton = compiled.runAutomaton;
+    this.commonSuffixRef = compiled.commonSuffixRef;
+    this.allTransitions = compiled.sortedTransitions;
 
-    this.runAutomaton = runAutomaton;
-    if (finite) {
-      // don't use suffix w/ finite DFAs
-      this.commonSuffixRef = null;
-    } else if (commonSuffixRef == null) {
-      // compute now
-      this.commonSuffixRef = SpecialOperations.getCommonSuffixBytesRef(automaton);
-    } else {
-      // precomputed
-      this.commonSuffixRef = commonSuffixRef;
-    }
-
-    // build a cache of sorted transitions for every state
-    allTransitions = this.automaton.getSortedTransitions();
     // used for path tracking, where each bit is a numbered state.
     visited = new long[runAutomaton.getSize()];
 
@@ -109,17 +91,6 @@ public class AutomatonTermsEnum extends 
   }
   
   /**
-   * Construct an enumerator based upon an automaton, enumerating the specified
-   * field, working on a supplied TermsEnum
-   * <p>
-   * It will automatically calculate whether or not the automaton is finite
-   */
-  public AutomatonTermsEnum(Automaton automaton, TermsEnum tenum)
-    throws IOException {
-    this(new ByteRunAutomaton(automaton), tenum, SpecialOperations.isFinite(automaton), null);
-  }
- 
-  /**
    * Returns true if the term matches the automaton. Also stashes away the term
    * to assist with smart enumeration.
    */
@@ -350,4 +321,26 @@ public class AutomatonTermsEnum extends 
     }
     return -1; /* all solutions exhausted */
   }
+  
+  /**
+   * immutable class with everything this enum needs.
+   */
+  public static class CompiledAutomaton {
+    public final ByteRunAutomaton runAutomaton;
+    public final Transition[][] sortedTransitions;
+    public final BytesRef commonSuffixRef;
+    public final boolean finite;
+    
+    public CompiledAutomaton(Automaton automaton, boolean finite) {
+      Automaton utf8 = new UTF32ToUTF8().convert(automaton);
+      runAutomaton = new ByteRunAutomaton(utf8, true);
+      sortedTransitions = utf8.getSortedTransitions();
+      this.finite = finite;
+      if (finite) {
+        commonSuffixRef = null;
+      } else {
+        commonSuffixRef = SpecialOperations.getCommonSuffixBytesRef(utf8);
+      }
+    }
+  }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=1067720&r1=1067719&r2=1067720&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Sun Feb  6 18:17:00 2011
@@ -22,6 +22,7 @@ import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.AutomatonTermsEnum.CompiledAutomaton;
 import org.apache.lucene.util.Attribute;
 import org.apache.lucene.util.AttributeImpl;
 import org.apache.lucene.util.AttributeSource;
@@ -140,18 +141,18 @@ public final class FuzzyTermsEnum extend
    */
   private TermsEnum getAutomatonEnum(int editDistance, BytesRef lastTerm)
       throws IOException {
-    final List<ByteRunAutomaton> runAutomata = initAutomata(editDistance);
+    final List<CompiledAutomaton> runAutomata = initAutomata(editDistance);
     if (editDistance < runAutomata.size()) {
       return new AutomatonFuzzyTermsEnum(runAutomata.subList(0, editDistance + 1)
-          .toArray(new ByteRunAutomaton[editDistance + 1]), lastTerm);
+          .toArray(new CompiledAutomaton[editDistance + 1]), lastTerm);
     } else {
       return null;
     }
   }
 
   /** initialize levenshtein DFAs up to maxDistance, if possible */
-  private List<ByteRunAutomaton> initAutomata(int maxDistance) {
-    final List<ByteRunAutomaton> runAutomata = dfaAtt.automata();
+  private List<CompiledAutomaton> initAutomata(int maxDistance) {
+    final List<CompiledAutomaton> runAutomata = dfaAtt.automata();
     if (runAutomata.size() <= maxDistance && 
         maxDistance <= LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE) {
       LevenshteinAutomata builder = 
@@ -165,7 +166,7 @@ public final class FuzzyTermsEnum extend
             UnicodeUtil.newString(termText, 0, realPrefixLength));
           a = BasicOperations.concatenate(prefix, a);
         }
-        runAutomata.add(new ByteRunAutomaton(a));
+        runAutomata.add(new CompiledAutomaton(a, true));
       }
     }
     return runAutomata;
@@ -312,10 +313,12 @@ public final class FuzzyTermsEnum extend
     private final BoostAttribute boostAtt =
       attributes().addAttribute(BoostAttribute.class);
     
-    public AutomatonFuzzyTermsEnum(ByteRunAutomaton matchers[], 
+    public AutomatonFuzzyTermsEnum(CompiledAutomaton compiled[], 
         BytesRef lastTerm) throws IOException {
-      super(matchers[matchers.length - 1], tenum, true, null);
-      this.matchers = matchers;
+      super(tenum, compiled[compiled.length - 1]);
+      this.matchers = new ByteRunAutomaton[compiled.length];
+      for (int i = 0; i < compiled.length; i++)
+        this.matchers[i] = compiled[i].runAutomaton;
       this.lastTerm = lastTerm;
       termRef = new BytesRef(term.text());
     }
@@ -563,14 +566,14 @@ public final class FuzzyTermsEnum extend
   
   /** @lucene.internal */
   public static interface LevenshteinAutomataAttribute extends Attribute {
-    public List<ByteRunAutomaton> automata();
+    public List<CompiledAutomaton> automata();
   }
     
   /** @lucene.internal */
   public static final class LevenshteinAutomataAttributeImpl extends AttributeImpl implements LevenshteinAutomataAttribute {
-    private final List<ByteRunAutomaton> automata = new ArrayList<ByteRunAutomaton>();
+    private final List<CompiledAutomaton> automata = new ArrayList<CompiledAutomaton>();
       
-    public List<ByteRunAutomaton> automata() {
+    public List<CompiledAutomaton> automata() {
       return automata;
     }
 
@@ -595,7 +598,7 @@ public final class FuzzyTermsEnum extend
 
     @Override
     public void copyTo(AttributeImpl target) {
-      final List<ByteRunAutomaton> targetAutomata =
+      final List<CompiledAutomaton> targetAutomata =
         ((LevenshteinAutomataAttribute) target).automata();
       targetAutomata.clear();
       targetAutomata.addAll(automata);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/Automaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/Automaton.java?rev=1067720&r1=1067719&r2=1067720&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/Automaton.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/Automaton.java Sun Feb  6 18:17:00 2011
@@ -66,6 +66,13 @@ import org.apache.lucene.util.RamUsageEs
  * assumed by the built-in automata operations.
  * 
  * <p>
+ * <p>
+ * Note: This class has internal mutable state and is not thread safe. It is 
+ * the caller's responsibility to ensure any necessary synchronization if you
+ * wish to use the same Automaton from multiple threads. In general it is instead
+ * recommended to use a {@link RunAutomaton} for multithreaded matching: it is immutable, 
+ * thread safe, and much faster.  
+ * </p>
  * @lucene.experimental
  */
 public class Automaton implements Serializable, Cloneable {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/ByteRunAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/ByteRunAutomaton.java?rev=1067720&r1=1067719&r2=1067720&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/ByteRunAutomaton.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/ByteRunAutomaton.java Sun Feb  6 18:17:00 2011
@@ -20,7 +20,12 @@ package org.apache.lucene.util.automaton
 public class ByteRunAutomaton extends RunAutomaton {
   
   public ByteRunAutomaton(Automaton a) {
-    super(new UTF32ToUTF8().convert(a), 256, true);
+    this(a, false);
+  }
+  
+  /** expert: if utf8 is true, the input is already byte-based */
+  public ByteRunAutomaton(Automaton a, boolean utf8) {
+    super(utf8 ? a : new UTF32ToUTF8().convert(a), 256, true);
   }
 
   /**

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/RunAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/RunAutomaton.java?rev=1067720&r1=1067719&r2=1067720&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/RunAutomaton.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/RunAutomaton.java Sun Feb  6 18:17:00 2011
@@ -45,7 +45,6 @@ public abstract class RunAutomaton imple
                      // getCharClass(c)]
   final int[] points; // char interval start points
   final int[] classmap; // map from char number to class class
-  final Automaton automaton;
   
   /**
    * Returns a string representation of this automaton.
@@ -113,13 +112,6 @@ public abstract class RunAutomaton imple
   final int getCharClass(int c) {
     return SpecialOperations.findIndex(c, points);
   }
-  
-  /**
-   * @return the automaton
-   */
-  public Automaton getAutomaton() {
-    return automaton;
-  }
 
   /**
    * Constructs a new <code>RunAutomaton</code> from a deterministic
@@ -160,7 +152,6 @@ public abstract class RunAutomaton imple
     } else {
       classmap = null;
     }
-    this.automaton = a;
   }
   
   /**

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java?rev=1067720&r1=1067719&r2=1067720&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java Sun Feb  6 18:17:00 2011
@@ -26,7 +26,8 @@ import java.util.ArrayList;
 // TODO
 //   - do we really need the .bits...?  if not we can make util in UnicodeUtil to convert 1 char into a BytesRef
 
-final class UTF32ToUTF8 {
+/** @lucene.internal */
+public final class UTF32ToUTF8 {
 
   // Unicode boundaries for UTF8 bytes 1,2,3,4
   private static final int[] startCodes = new int[] {0, 128, 2048, 65536};