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 2014/11/08 12:32:23 UTC

svn commit: r1637544 [2/6] - in /lucene/dev/branches/lucene6005: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/codecs/perfield/ lucene/core/src/java/org/apache/lucene/document/ lucene/core/src/java/org/apache/lucene/index/ lucene/core/...

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java Sat Nov  8 11:32:18 2014
@@ -29,6 +29,13 @@
 
 package org.apache.lucene.util.automaton;
 
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
+import org.apache.lucene.util.RamUsageEstimator;
+
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.BitSet;
@@ -40,20 +47,17 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.IntsRef;
-import org.apache.lucene.util.IntsRefBuilder;
-import org.apache.lucene.util.RamUsageEstimator;
-
 /**
  * Automata operations.
  * 
  * @lucene.experimental
  */
 final public class Operations {
-  
+  /**
+   * Default maximum number of states that {@link Operations#determinize} should create.
+   */
+  public static final int DEFAULT_MAX_DETERMINIZED_STATES = 10000;
+
   private Operations() {}
 
   /**
@@ -170,6 +174,10 @@ final public class Operations {
    * Complexity: linear in number of states.
    */
   static public Automaton repeat(Automaton a) {
+    if (a.getNumStates() == 0) {
+      // Repeating the empty automata will still only accept the empty automata.
+      return a;
+    }
     Automaton.Builder builder = new Automaton.Builder();
     builder.createState();
     builder.setAccept(0, true);
@@ -202,12 +210,12 @@ final public class Operations {
    * <p>
    * Complexity: linear in number of states and in <code>min</code>.
    */
-  static public Automaton repeat(Automaton a, int min) {
-    if (min == 0) {
+  static public Automaton repeat(Automaton a, int count) {
+    if (count == 0) {
       return repeat(a);
     }
     List<Automaton> as = new ArrayList<>();
-    while (min-- > 0) {
+    while (count-- > 0) {
       as.add(a);
     }
     as.add(repeat(a));
@@ -242,19 +250,18 @@ final public class Operations {
     }
 
     Set<Integer> prevAcceptStates = toSet(b, 0);
-
+    Automaton.Builder builder = new Automaton.Builder();
+    builder.copy(b);
     for(int i=min;i<max;i++) {
-      int numStates = b.getNumStates();
-      b.copy(a);
+      int numStates = builder.getNumStates();
+      builder.copy(a);
       for(int s : prevAcceptStates) {
-        b.addEpsilon(s, numStates);
+        builder.addEpsilon(s, numStates);
       }
       prevAcceptStates = toSet(a, numStates);
     }
 
-    b.finishState();
-
-    return b;
+    return builder.finish();
   }
 
   private static Set<Integer> toSet(Automaton a, int offset) {
@@ -274,10 +281,14 @@ final public class Operations {
    * Returns a (deterministic) automaton that accepts the complement of the
    * language of the given automaton.
    * <p>
-   * Complexity: linear in number of states (if already deterministic).
+   * Complexity: linear in number of states if already deterministic and
+   *  exponential otherwise.
+   * @param maxDeterminizedStates maximum number of states determinizing the
+   *  automaton can result in.  Set higher to allow more complex queries and
+   *  lower to prevent memory exhaustion.
    */
-  static public Automaton complement(Automaton a) {
-    a = totalize(determinize(a));
+  static public Automaton complement(Automaton a, int maxDeterminizedStates) {
+    a = totalize(determinize(a, maxDeterminizedStates));
     int numStates = a.getNumStates();
     for (int p=0;p<numStates;p++) {
       a.setAccept(p, !a.isAccept(p));
@@ -291,16 +302,17 @@ final public class Operations {
    * <code>a2</code>. As a side-effect, the automata may be determinized, if not
    * already deterministic.
    * <p>
-   * Complexity: quadratic in number of states (if already deterministic).
+   * Complexity: quadratic in number of states if a2 already deterministic and
+   *  exponential in number of a2's states otherwise.
    */
-  static public Automaton minus(Automaton a1, Automaton a2) {
+  static public Automaton minus(Automaton a1, Automaton a2, int maxDeterminizedStates) {
     if (Operations.isEmpty(a1) || a1 == a2) {
       return Automata.makeEmpty();
     }
     if (Operations.isEmpty(a2)) {
       return a1;
     }
-    return intersection(a1, complement(a2));
+    return intersection(a1, complement(a2, maxDeterminizedStates));
   }
   
   /**
@@ -490,7 +502,6 @@ final public class Operations {
     result.createState();
 
     // Copy over all automata
-    Transition t = new Transition();
     for(Automaton a : l) {
       result.copy(a);
     }
@@ -644,8 +655,15 @@ final public class Operations {
    * Determinizes the given automaton.
    * <p>
    * Worst case complexity: exponential in number of states.
+   * @param maxDeterminizedStates Maximum number of states created when
+   *   determinizing.  Higher numbers allow this operation to consume more
+   *   memory but allow more complex automatons.  Use
+   *   DEFAULT_MAX_DETERMINIZED_STATES as a decent default if you don't know
+   *   how many to allow.
+   * @throws TooComplexToDeterminizeException if determinizing a creates an
+   *   automaton with more than maxDeterminizedStates
    */
-  public static Automaton determinize(Automaton a) {
+  public static Automaton determinize(Automaton a, int maxDeterminizedStates) {
     if (a.isDeterministic()) {
       // Already determinized
       return a;
@@ -674,11 +692,6 @@ final public class Operations {
     b.setAccept(0, a.isAccept(0));
     newstate.put(initialset, 0);
 
-    int newStateUpto = 0;
-    int[] newStatesArray = new int[5];
-    newStatesArray[newStateUpto] = 0;
-    newStateUpto++;
-
     // like Set<Integer,PointTransitions>
     final PointTransitionSet points = new PointTransitionSet();
 
@@ -726,6 +739,9 @@ final public class Operations {
           Integer q = newstate.get(statesSet);
           if (q == null) {
             q = b.createState();
+            if (q >= maxDeterminizedStates) {
+              throw new TooComplexToDeterminizeException(a, maxDeterminizedStates);
+            }
             final SortedIntSet.FrozenIntSet p = statesSet.freeze(q);
             //System.out.println("  make new state=" + q + " -> " + p + " accCount=" + accCount);
             worklist.add(p);
@@ -1100,12 +1116,14 @@ final public class Operations {
    * Returns the longest BytesRef that is a suffix of all accepted strings.
    * Worst case complexity: exponential in number of states (this calls
    * determinize).
-   *
+   * @param maxDeterminizedStates maximum number of states determinizing the
+   *  automaton can result in.  Set higher to allow more complex queries and
+   *  lower to prevent memory exhaustion.
    * @return common suffix
    */
-  public static BytesRef getCommonSuffixBytesRef(Automaton a) {
+  public static BytesRef getCommonSuffixBytesRef(Automaton a, int maxDeterminizedStates) {
     // reverse the language of the automaton, then reverse its common prefix.
-    Automaton r = Operations.determinize(reverse(a));
+    Automaton r = Operations.determinize(reverse(a), maxDeterminizedStates);
     BytesRef ref = getCommonPrefixBytesRef(r);
     reverseBytes(ref);
     return ref;

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java Sat Nov  8 11:32:18 2014
@@ -360,7 +360,8 @@ public class RegExp {
    * Syntax flag, enables no optional regexp syntax.
    */
   public static final int NONE = 0x0000;
-  
+
+  private final String originalString;
   Kind kind;
   RegExp exp1, exp2;
   String s;
@@ -368,11 +369,12 @@ public class RegExp {
   int min, max, digits;
   int from, to;
   
-  String b;
   int flags;
   int pos;
   
-  RegExp() {}
+  RegExp() {
+    this.originalString = null;
+  }
   
   /**
    * Constructs new <code>RegExp</code> from a string. Same as
@@ -396,13 +398,13 @@ public class RegExp {
    *              regular expression
    */
   public RegExp(String s, int syntax_flags) throws IllegalArgumentException {
-    b = s;
+    originalString = s;
     flags = syntax_flags;
     RegExp e;
     if (s.length() == 0) e = makeString("");
     else {
       e = parseUnionExp();
-      if (pos < b.length()) throw new IllegalArgumentException(
+      if (pos < originalString.length()) throw new IllegalArgumentException(
           "end-of-string expected at position " + pos);
     }
     kind = e.kind;
@@ -415,7 +417,6 @@ public class RegExp {
     digits = e.digits;
     from = e.from;
     to = e.to;
-    b = null;
   }
 
   /**
@@ -423,21 +424,47 @@ public class RegExp {
    * as <code>toAutomaton(null)</code> (empty automaton map).
    */
   public Automaton toAutomaton() {
-    return toAutomaton(null, null);
+    return toAutomaton(null, null, Operations.DEFAULT_MAX_DETERMINIZED_STATES);
   }
-  
+
   /**
    * Constructs new <code>Automaton</code> from this <code>RegExp</code>. The
    * constructed automaton is minimal and deterministic and has no transitions
    * to dead states.
    * 
-   * @param automaton_provider provider of automata for named identifiers
+   * @param maxDeterminizedStates maximum number of states in the resulting
+   *   automata.  If the automata would need more than this many states
+   *   TooComplextToDeterminizeException is thrown.  Higher number require more
+   *   space but can process more complex regexes.
    * @exception IllegalArgumentException if this regular expression uses a named
    *              identifier that is not available from the automaton provider
+   * @exception TooComplexToDeterminizeException if determinizing this regexp
+   *   requires more than maxDeterminizedStates states
    */
-  public Automaton toAutomaton(AutomatonProvider automaton_provider)
-      throws IllegalArgumentException {
-    return toAutomaton(null, automaton_provider);
+  public Automaton toAutomaton(int maxDeterminizedStates)
+      throws IllegalArgumentException, TooComplexToDeterminizeException {
+    return toAutomaton(null, null, maxDeterminizedStates);
+  }
+
+  /**
+   * Constructs new <code>Automaton</code> from this <code>RegExp</code>. The
+   * constructed automaton is minimal and deterministic and has no transitions
+   * to dead states.
+   * 
+   * @param automaton_provider provider of automata for named identifiers
+   * @param maxDeterminizedStates maximum number of states in the resulting
+   *   automata.  If the automata would need more than this many states
+   *   TooComplextToDeterminizeException is thrown.  Higher number require more
+   *   space but can process more complex regexes.
+   * @exception IllegalArgumentException if this regular expression uses a named
+   *   identifier that is not available from the automaton provider
+   * @exception TooComplexToDeterminizeException if determinizing this regexp
+   *   requires more than maxDeterminizedStates states
+   */
+  public Automaton toAutomaton(AutomatonProvider automaton_provider,
+      int maxDeterminizedStates) throws IllegalArgumentException,
+      TooComplexToDeterminizeException {
+    return toAutomaton(null, automaton_provider, maxDeterminizedStates);
   }
   
   /**
@@ -447,60 +474,95 @@ public class RegExp {
    * 
    * @param automata a map from automaton identifiers to automata (of type
    *          <code>Automaton</code>).
+   * @param maxDeterminizedStates maximum number of states in the resulting
+   *   automata.  If the automata would need more than this many states
+   *   TooComplexToDeterminizeException is thrown.  Higher number require more
+   *   space but can process more complex regexes.
    * @exception IllegalArgumentException if this regular expression uses a named
-   *              identifier that does not occur in the automaton map
+   *   identifier that does not occur in the automaton map
+   * @exception TooComplexToDeterminizeException if determinizing this regexp
+   *   requires more than maxDeterminizedStates states
    */
-  public Automaton toAutomaton(Map<String,Automaton> automata)
-      throws IllegalArgumentException {
-    return toAutomaton(automata, null);
+  public Automaton toAutomaton(Map<String,Automaton> automata,
+      int maxDeterminizedStates) throws IllegalArgumentException,
+      TooComplexToDeterminizeException {
+    return toAutomaton(automata, null, maxDeterminizedStates);
   }
 
   private Automaton toAutomaton(Map<String,Automaton> automata,
-      AutomatonProvider automaton_provider) throws IllegalArgumentException {
+      AutomatonProvider automaton_provider, int maxDeterminizedStates)
+      throws IllegalArgumentException, TooComplexToDeterminizeException {
+    try {
+      return toAutomatonInternal(automata, automaton_provider,
+        maxDeterminizedStates);
+    } catch (TooComplexToDeterminizeException e) {
+      throw new TooComplexToDeterminizeException(this, e);
+    }
+  }
+
+  private Automaton toAutomatonInternal(Map<String,Automaton> automata,
+      AutomatonProvider automaton_provider, int maxDeterminizedStates)
+      throws IllegalArgumentException {
     List<Automaton> list;
     Automaton a = null;
     switch (kind) {
       case REGEXP_UNION:
         list = new ArrayList<>();
-        findLeaves(exp1, Kind.REGEXP_UNION, list, automata, automaton_provider);
-        findLeaves(exp2, Kind.REGEXP_UNION, list, automata, automaton_provider);
+        findLeaves(exp1, Kind.REGEXP_UNION, list, automata, automaton_provider,
+          maxDeterminizedStates);
+        findLeaves(exp2, Kind.REGEXP_UNION, list, automata, automaton_provider,
+          maxDeterminizedStates);
         a = Operations.union(list);
-        a = MinimizationOperations.minimize(a);
+        a = MinimizationOperations.minimize(a, maxDeterminizedStates);
         break;
       case REGEXP_CONCATENATION:
         list = new ArrayList<>();
         findLeaves(exp1, Kind.REGEXP_CONCATENATION, list, automata,
-            automaton_provider);
+            automaton_provider, maxDeterminizedStates);
         findLeaves(exp2, Kind.REGEXP_CONCATENATION, list, automata,
-            automaton_provider);
+            automaton_provider, maxDeterminizedStates);
         a = Operations.concatenate(list);
-        a = MinimizationOperations.minimize(a);
+        a = MinimizationOperations.minimize(a, maxDeterminizedStates);
         break;
       case REGEXP_INTERSECTION:
         a = Operations.intersection(
-            exp1.toAutomaton(automata, automaton_provider),
-            exp2.toAutomaton(automata, automaton_provider));
-        a = MinimizationOperations.minimize(a);
+            exp1.toAutomatonInternal(
+              automata, automaton_provider, maxDeterminizedStates),
+            exp2.toAutomatonInternal(
+              automata, automaton_provider, maxDeterminizedStates));
+        a = MinimizationOperations.minimize(a, maxDeterminizedStates);
         break;
       case REGEXP_OPTIONAL:
-        a = Operations.optional(exp1.toAutomaton(automata, automaton_provider));
-        a = MinimizationOperations.minimize(a);
+        a = Operations.optional(exp1.toAutomatonInternal(automata,
+          automaton_provider, maxDeterminizedStates));
+        a = MinimizationOperations.minimize(a, maxDeterminizedStates);
         break;
       case REGEXP_REPEAT:
-        a = Operations.repeat(exp1.toAutomaton(automata, automaton_provider));
-        a = MinimizationOperations.minimize(a);
+        a = Operations.repeat(exp1.toAutomatonInternal(
+          automata, automaton_provider, maxDeterminizedStates));
+        a = MinimizationOperations.minimize(a, maxDeterminizedStates);
         break;
       case REGEXP_REPEAT_MIN:
-        a = Operations.repeat(exp1.toAutomaton(automata, automaton_provider), min);
-        a = MinimizationOperations.minimize(a);
+        a = Operations.repeat(
+          exp1.toAutomatonInternal(automata, automaton_provider,
+            maxDeterminizedStates),
+          min);
+        a = MinimizationOperations.minimize(a, maxDeterminizedStates);
         break;
       case REGEXP_REPEAT_MINMAX:
-        a = Operations.repeat(exp1.toAutomaton(automata, automaton_provider), min, max);
-        a = MinimizationOperations.minimize(a);
+        a = Operations.repeat(
+          exp1.toAutomatonInternal(automata, automaton_provider,
+            maxDeterminizedStates),
+          min,
+          max);
+        a = MinimizationOperations.minimize(a, maxDeterminizedStates);
         break;
       case REGEXP_COMPLEMENT:
-        a = Operations.complement(exp1.toAutomaton(automata, automaton_provider));
-        a = MinimizationOperations.minimize(a);
+        a = Operations.complement(
+          exp1.toAutomatonInternal(automata, automaton_provider,
+            maxDeterminizedStates),
+          maxDeterminizedStates);
+        a = MinimizationOperations.minimize(a, maxDeterminizedStates);
         break;
       case REGEXP_CHAR:
         a = Automata.makeChar(c);
@@ -545,24 +607,37 @@ public class RegExp {
   }
   
   private void findLeaves(RegExp exp, Kind kind, List<Automaton> list,
-      Map<String,Automaton> automata, AutomatonProvider automaton_provider) {
+      Map<String,Automaton> automata, AutomatonProvider automaton_provider,
+      int maxDeterminizedStates) {
     if (exp.kind == kind) {
-      findLeaves(exp.exp1, kind, list, automata, automaton_provider);
-      findLeaves(exp.exp2, kind, list, automata, automaton_provider);
+      findLeaves(exp.exp1, kind, list, automata, automaton_provider,
+        maxDeterminizedStates);
+      findLeaves(exp.exp2, kind, list, automata, automaton_provider,
+        maxDeterminizedStates);
     } else {
-      list.add(exp.toAutomaton(automata, automaton_provider));
+      list.add(exp.toAutomatonInternal(automata, automaton_provider, 
+        maxDeterminizedStates));
     }
   }
-  
+
+  /**
+   * The string that was used to construct the regex.  Compare to toString.
+   */
+  public String getOriginalString() {
+    return originalString;
+  }
+
   /**
    * Constructs string from parsed regular expression.
    */
   @Override
   public String toString() {
-    return toStringBuilder(new StringBuilder()).toString();
+    StringBuilder b = new StringBuilder();
+    toStringBuilder(b);
+    return b.toString();
   }
   
-  StringBuilder toStringBuilder(StringBuilder b) {
+  void toStringBuilder(StringBuilder b) {
     switch (kind) {
       case REGEXP_UNION:
         b.append("(");
@@ -640,9 +715,112 @@ public class RegExp {
         b.append(s2).append(">");
         break;
     }
-    return b;
   }
-  
+
+  /**
+   * Like to string, but more verbose (shows the higherchy more clearly).
+   */
+  public String toStringTree() {
+    StringBuilder b = new StringBuilder();
+    toStringTree(b, "");
+    return b.toString();
+  }
+
+  void toStringTree(StringBuilder b, String indent) {
+    switch (kind) {
+      // binary
+      case REGEXP_UNION:
+      case REGEXP_CONCATENATION:
+      case REGEXP_INTERSECTION:
+        b.append(indent);
+        b.append(kind);
+        b.append('\n');
+        exp1.toStringTree(b, indent + "  ");
+        exp2.toStringTree(b, indent + "  ");
+        break;
+      // unary
+      case REGEXP_OPTIONAL:
+      case REGEXP_REPEAT:
+      case REGEXP_COMPLEMENT:
+        b.append(indent);
+        b.append(kind);
+        b.append('\n');
+        exp1.toStringTree(b, indent + "  ");
+        break;
+      case REGEXP_REPEAT_MIN:
+        b.append(indent);
+        b.append(kind);
+        b.append(" min=");
+        b.append(min);
+        b.append('\n');
+        exp1.toStringTree(b, indent + "  ");
+        break;
+      case REGEXP_REPEAT_MINMAX:
+        b.append(indent);
+        b.append(kind);
+        b.append(" min=");
+        b.append(min);
+        b.append(" max=");
+        b.append(max);
+        b.append('\n');
+        exp1.toStringTree(b, indent + "  ");
+        break;
+      case REGEXP_CHAR:
+        b.append(indent);
+        b.append(kind);
+        b.append(" char=");
+        b.appendCodePoint(c);
+        b.append('\n');
+        break;
+      case REGEXP_CHAR_RANGE:
+        b.append(indent);
+        b.append(kind);
+        b.append(" from=");
+        b.appendCodePoint(from);
+        b.append(" to=");
+        b.appendCodePoint(to);
+        b.append('\n');
+        break;
+      case REGEXP_ANYCHAR:
+      case REGEXP_EMPTY:
+        b.append(indent);
+        b.append(kind);
+        b.append('\n');
+        break;
+      case REGEXP_STRING:
+        b.append(indent);
+        b.append(kind);
+        b.append(" string=");
+        b.append(s);
+        b.append('\n');
+        break;
+      case REGEXP_ANYSTRING:
+        b.append(indent);
+        b.append(kind);
+        b.append('\n');
+        break;
+      case REGEXP_AUTOMATON:
+        b.append(indent);
+        b.append(kind);
+        b.append('\n');
+        break;
+      case REGEXP_INTERVAL:
+        b.append(indent);
+        b.append(kind);
+        String s1 = Integer.toString(min);
+        String s2 = Integer.toString(max);
+        b.append("<");
+        if (digits > 0) for (int i = s1.length(); i < digits; i++)
+          b.append('0');
+        b.append(s1).append("-");
+        if (digits > 0) for (int i = s2.length(); i < digits; i++)
+          b.append('0');
+        b.append(s2).append(">");
+        b.append('\n');
+        break;
+    }
+  }
+
   /**
    * Returns set of automaton identifiers that occur in this regular expression.
    */
@@ -819,12 +997,12 @@ public class RegExp {
   }
   
   private boolean peek(String s) {
-    return more() && s.indexOf(b.codePointAt(pos)) != -1;
+    return more() && s.indexOf(originalString.codePointAt(pos)) != -1;
   }
   
   private boolean match(int c) {
-    if (pos >= b.length()) return false;
-    if (b.codePointAt(pos) == c) {
+    if (pos >= originalString.length()) return false;
+    if (originalString.codePointAt(pos) == c) {
       pos += Character.charCount(c);
       return true;
     }
@@ -832,12 +1010,12 @@ public class RegExp {
   }
   
   private boolean more() {
-    return pos < b.length();
+    return pos < originalString.length();
   }
   
   private int next() throws IllegalArgumentException {
     if (!more()) throw new IllegalArgumentException("unexpected end-of-string");
-    int ch = b.codePointAt(pos);
+    int ch = originalString.codePointAt(pos);
     pos += Character.charCount(ch);
     return ch;
   }
@@ -878,13 +1056,14 @@ public class RegExp {
           next();
         if (start == pos) throw new IllegalArgumentException(
             "integer expected at position " + pos);
-        int n = Integer.parseInt(b.substring(start, pos));
+        int n = Integer.parseInt(originalString.substring(start, pos));
         int m = -1;
         if (match(',')) {
           start = pos;
           while (peek("0123456789"))
             next();
-          if (start != pos) m = Integer.parseInt(b.substring(start, pos));
+          if (start != pos) m = Integer.parseInt(
+            originalString.substring(start, pos));
         } else m = n;
         if (!match('}')) throw new IllegalArgumentException(
             "expected '}' at position " + pos);
@@ -935,7 +1114,7 @@ public class RegExp {
         next();
       if (!match('"')) throw new IllegalArgumentException(
           "expected '\"' at position " + pos);
-      return makeString(b.substring(start, pos - 1));
+      return makeString(originalString.substring(start, pos - 1));
     } else if (match('(')) {
       if (match(')')) return makeString("");
       RegExp e = parseUnionExp();
@@ -948,7 +1127,7 @@ public class RegExp {
         next();
       if (!match('>')) throw new IllegalArgumentException(
           "expected '>' at position " + pos);
-      String s = b.substring(start, pos - 1);
+      String s = originalString.substring(start, pos - 1);
       int i = s.indexOf('-');
       if (i == -1) {
         if (!check(AUTOMATON)) throw new IllegalArgumentException(

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/automaton/RunAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/automaton/RunAutomaton.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/automaton/RunAutomaton.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/automaton/RunAutomaton.java Sat Nov  8 11:32:18 2014
@@ -121,8 +121,21 @@ public abstract class RunAutomaton {
    * @param a an automaton
    */
   public RunAutomaton(Automaton a, int maxInterval, boolean tableize) {
+    this(a, maxInterval, tableize, Operations.DEFAULT_MAX_DETERMINIZED_STATES);
+  }
+
+  /**
+   * Constructs a new <code>RunAutomaton</code> from a deterministic
+   * <code>Automaton</code>.
+   * 
+   * @param a an automaton
+   * @param maxDeterminizedStates maximum number of states that can be created
+   *   while determinizing a
+   */
+  public RunAutomaton(Automaton a, int maxInterval, boolean tableize,
+      int maxDeterminizedStates) {
     this.maxInterval = maxInterval;
-    a = Operations.determinize(a);
+    a = Operations.determinize(a, maxDeterminizedStates);
     this.automaton = a;
     points = a.getStartPoints();
     initial = 0;

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDecoder.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDecoder.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/util/packed/EliasFanoDecoder.java Sat Nov  8 11:32:18 2014
@@ -17,7 +17,7 @@
 
 package org.apache.lucene.util.packed;
 
-import org.apache.lucene.util.BroadWord; // bit selection in long
+import org.apache.lucene.util.BitUtil; // bit selection in long
 
 
 /** A decoder for an {@link EliasFanoEncoder}.
@@ -312,9 +312,10 @@ public class EliasFanoDecoder {
     if (rank >= 1) {
       long invCurHighLong = ~curHighLong;
       int clearBitForValue = (rank <= 8)
-                              ? BroadWord.selectNaive(invCurHighLong, rank)
-                              : BroadWord.select(invCurHighLong, rank);
-      assert clearBitForValue <= (Long.SIZE-1);
+                              ? BitUtil.selectNaive(invCurHighLong, rank)
+                              : BitUtil.select(invCurHighLong, rank);
+      assert clearBitForValue >= 0;
+      assert clearBitForValue <= Long.SIZE-1;
       setBitForIndex += clearBitForValue + 1; // the high bit just before setBitForIndex is zero
       int oneBitsBeforeClearBit = clearBitForValue - rank + 1;
       efIndex += oneBitsBeforeClearBit; // the high bit at setBitForIndex and belongs to the unary code for efIndex

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java Sat Nov  8 11:32:18 2014
@@ -30,8 +30,10 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
 import org.apache.lucene.util.automaton.Automata;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.Operations;
+
+import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
 
 public class TestGraphTokenizers extends BaseTokenStreamTestCase {
 
@@ -404,15 +406,11 @@ public class TestGraphTokenizers extends
   }
 
   public void testSingleToken() throws Exception {
-
     final TokenStream ts = new CannedTokenStream(
       new Token[] {
         token("abc", 1, 1),
       });
-    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
-    final Automaton expected = s2a("abc");
-    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
-                                       Operations.determinize(Operations.removeDeadStates(actual))));
+    assertSameLanguage(s2a("abc"), ts);
   }
 
   public void testMultipleHoles() throws Exception {
@@ -421,10 +419,7 @@ public class TestGraphTokenizers extends
         token("a", 1, 1),
         token("b", 3, 1),
       });
-    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
-    final Automaton expected = join(s2a("a"), SEP_A, HOLE_A, SEP_A, HOLE_A, SEP_A, s2a("b")); 
-    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
-                                       Operations.determinize(Operations.removeDeadStates(actual))));
+    assertSameLanguage(join(s2a("a"), SEP_A, HOLE_A, SEP_A, HOLE_A, SEP_A, s2a("b")), ts);
   }
 
   public void testSynOverMultipleHoles() throws Exception {
@@ -434,12 +429,9 @@ public class TestGraphTokenizers extends
         token("x", 0, 3),
         token("b", 3, 1),
       });
-    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
     final Automaton a1 = join(s2a("a"), SEP_A, HOLE_A, SEP_A, HOLE_A, SEP_A, s2a("b")); 
     final Automaton a2 = join(s2a("x"), SEP_A, s2a("b")); 
-    final Automaton expected = Operations.union(a1, a2);
-    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
-                                       Operations.determinize(Operations.removeDeadStates(actual))));
+    assertSameLanguage(Operations.union(a1, a2), ts);
   }
 
   // for debugging!
@@ -475,18 +467,12 @@ public class TestGraphTokenizers extends
   }
 
   public void testTwoTokens() throws Exception {
-
     final TokenStream ts = new CannedTokenStream(
       new Token[] {
         token("abc", 1, 1),
         token("def", 1, 1),
       });
-    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
-    final Automaton expected =  join("abc", "def");
-
-    //toDot(actual);
-    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
-                                       Operations.determinize(Operations.removeDeadStates(actual))));
+    assertSameLanguage(join("abc", "def"), ts);
   }
 
   public void testHole() throws Exception {
@@ -496,13 +482,7 @@ public class TestGraphTokenizers extends
         token("abc", 1, 1),
         token("def", 2, 1),
       });
-    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
-
-    final Automaton expected = join(s2a("abc"), SEP_A, HOLE_A, SEP_A, s2a("def"));
-
-    //toDot(actual);
-    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
-                                       Operations.determinize(Operations.removeDeadStates(actual))));
+    assertSameLanguage(join(s2a("abc"), SEP_A, HOLE_A, SEP_A, s2a("def")), ts);
   }
 
   public void testOverlappedTokensSausage() throws Exception {
@@ -513,12 +493,9 @@ public class TestGraphTokenizers extends
         token("abc", 1, 1),
         token("xyz", 0, 1)
       });
-    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
     final Automaton a1 = s2a("abc");
     final Automaton a2 = s2a("xyz");
-    final Automaton expected = Operations.union(a1, a2);
-    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
-                                       Operations.determinize(Operations.removeDeadStates(actual))));
+    assertSameLanguage(Operations.union(a1, a2), ts);
   }
 
   public void testOverlappedTokensLattice() throws Exception {
@@ -529,14 +506,9 @@ public class TestGraphTokenizers extends
         token("xyz", 0, 2),
         token("def", 1, 1),
       });
-    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
     final Automaton a1 = s2a("xyz");
     final Automaton a2 = join("abc", "def");
-                                                                   
-    final Automaton expected = Operations.union(a1, a2);
-    //toDot(actual);
-    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
-                                       Operations.determinize(Operations.removeDeadStates(actual))));
+    assertSameLanguage(Operations.union(a1, a2), ts);
   }
 
   public void testSynOverHole() throws Exception {
@@ -547,15 +519,9 @@ public class TestGraphTokenizers extends
         token("X", 0, 2),
         token("b", 2, 1),
       });
-    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
-    final Automaton a1 = Operations.union(
-                                               join(s2a("a"), SEP_A, HOLE_A),
-                                               s2a("X"));
-    final Automaton expected = Operations.concatenate(a1,
-                                                           join(SEP_A, s2a("b")));
-    //toDot(actual);
-    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
-                                       Operations.determinize(Operations.removeDeadStates(actual))));
+    final Automaton a1 = Operations.union(join(s2a("a"), SEP_A, HOLE_A), s2a("X"));
+    final Automaton expected = Operations.concatenate(a1, join(SEP_A, s2a("b")));
+    assertSameLanguage(expected, ts);
   }
 
   public void testSynOverHole2() throws Exception {
@@ -566,12 +532,9 @@ public class TestGraphTokenizers extends
         token("abc", 0, 3),
         token("def", 2, 1),
       });
-    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
     final Automaton expected = Operations.union(
-                                                     join(s2a("xyz"), SEP_A, HOLE_A, SEP_A, s2a("def")),
-                                                     s2a("abc"));
-    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
-                                       Operations.determinize(Operations.removeDeadStates(actual))));
+      join(s2a("xyz"), SEP_A, HOLE_A, SEP_A, s2a("def")), s2a("abc"));
+    assertSameLanguage(expected, ts);
   }
 
   public void testOverlappedTokensLattice2() throws Exception {
@@ -583,13 +546,9 @@ public class TestGraphTokenizers extends
         token("def", 1, 1),
         token("ghi", 1, 1),
       });
-    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
     final Automaton a1 = s2a("xyz");
     final Automaton a2 = join("abc", "def", "ghi");
-    final Automaton expected = Operations.union(a1, a2);
-    //toDot(actual);
-    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
-                                       Operations.determinize(Operations.removeDeadStates(actual))));
+    assertSameLanguage(Operations.union(a1, a2), ts);
   }
 
   public void testToDot() throws Exception {
@@ -604,11 +563,7 @@ public class TestGraphTokenizers extends
       new Token[] {
         token("abc", 2, 1),
       });
-    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
-    final Automaton expected = join(HOLE_A, SEP_A, s2a("abc"));
-    //toDot(actual);
-    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
-                                       Operations.determinize(Operations.removeDeadStates(actual))));
+    assertSameLanguage(join(HOLE_A, SEP_A, s2a("abc")), ts);
   }
 
   // TODO: testEndsWithHole... but we need posInc to set in TS.end()
@@ -619,10 +574,16 @@ public class TestGraphTokenizers extends
         token("a", 1, 1),
         token("X", 0, 10),
       });
-    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
-    final Automaton expected = Operations.union(s2a("a"),
-                                                               s2a("X"));
-    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(expected)),
-                                       Operations.determinize(Operations.removeDeadStates(actual))));
+    assertSameLanguage(Operations.union(s2a("a"), s2a("X")), ts);
+  }
+
+  private void assertSameLanguage(Automaton expected, TokenStream ts) throws IOException {
+    assertSameLanguage(expected, new TokenStreamToAutomaton().toAutomaton(ts));
+  }
+
+  private void assertSameLanguage(Automaton expected, Automaton actual) {
+    assertTrue(Operations.sameLanguage(
+      Operations.determinize(Operations.removeDeadStates(expected), DEFAULT_MAX_DETERMINIZED_STATES),
+      Operations.determinize(Operations.removeDeadStates(actual), DEFAULT_MAX_DETERMINIZED_STATES)));
   }
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java Sat Nov  8 11:32:18 2014
@@ -42,6 +42,8 @@ import org.apache.lucene.util.automaton.
 import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.RegExp;
 
+import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
+
 public class TestMockAnalyzer extends BaseTokenStreamTestCase {
 
   /** Test a configuration that behaves a lot like WhitespaceAnalyzer */
@@ -168,7 +170,8 @@ public class TestMockAnalyzer extends Ba
       new CharacterRunAutomaton(
           Operations.complement(
               Operations.union(
-                  Arrays.asList(Automata.makeString("foo"), Automata.makeString("bar")))));
+                  Arrays.asList(Automata.makeString("foo"), Automata.makeString("bar"))),
+              DEFAULT_MAX_DETERMINIZED_STATES));
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true, keepWords);
     assertAnalyzesTo(a, "quick foo brown bar bar fox foo",
         new String[] { "foo", "bar", "bar", "foo" },
@@ -231,7 +234,7 @@ public class TestMockAnalyzer extends Ba
   public void testRandomRegexps() throws Exception {
     int iters = atLeast(30);
     for (int i = 0; i < iters; i++) {
-      final CharacterRunAutomaton dfa = new CharacterRunAutomaton(AutomatonTestUtil.randomAutomaton(random()));
+      final CharacterRunAutomaton dfa = new CharacterRunAutomaton(AutomatonTestUtil.randomAutomaton(random()), Integer.MAX_VALUE);
       final boolean lowercase = random().nextBoolean();
       final int limit = TestUtil.nextInt(random(), 0, 500);
       Analyzer a = new Analyzer() {

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/document/TestFieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/document/TestFieldType.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/document/TestFieldType.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/document/TestFieldType.java Sat Nov  8 11:32:18 2014
@@ -39,7 +39,7 @@ public class TestFieldType extends Lucen
     assertFalse(ft3.equals(ft));
     
     FieldType ft4 = new FieldType();
-    ft4.setDocValueType(DocValuesType.BINARY);
+    ft4.setDocValuesType(DocValuesType.BINARY);
     assertFalse(ft4.equals(ft));
     
     FieldType ft5 = new FieldType();

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java Sat Nov  8 11:32:18 2014
@@ -833,7 +833,7 @@ public class TestBinaryDocValuesUpdates 
   
   public void testUpdateBinaryDVFieldWithSameNameAsPostingField() throws Exception {
     // this used to fail because FieldInfos.Builder neglected to update
-    // globalFieldMaps.docValueTypes map
+    // globalFieldMaps.docValuesTypes map
     Directory dir = newDirectory();
     IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
     IndexWriter writer = new IndexWriter(dir, conf);

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java Sat Nov  8 11:32:18 2014
@@ -89,32 +89,12 @@ public class TestCodecs extends LuceneTe
       this.omitTF = omitTF;
       this.storePayloads = storePayloads;
       // TODO: change this test to use all three
-      fieldInfo = fieldInfos.addOrUpdate(name, new IndexableFieldType() {
-
-        @Override
-        public boolean stored() { return false; }
-
-        @Override
-        public boolean storeTermVectors() { return false; }
-
-        @Override
-        public boolean storeTermVectorOffsets() { return false; }
-
-        @Override
-        public boolean storeTermVectorPositions() { return false; }
-
-        @Override
-        public boolean storeTermVectorPayloads() { return false; }
-
-        @Override
-        public boolean omitNorms() { return false; }
-
-        @Override
-        public IndexOptions indexOptions() { return omitTF ? IndexOptions.DOCS : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS; }
-
-        @Override
-        public DocValuesType docValueType() { return DocValuesType.NONE; }
-      });
+      fieldInfo = fieldInfos.getOrAdd(name);
+      if (omitTF) {
+        fieldInfo.setIndexOptions(IndexOptions.DOCS);
+      } else {
+        fieldInfo.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+      }
       if (storePayloads) {
         fieldInfo.setStorePayloads();
       }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java Sat Nov  8 11:32:18 2014
@@ -246,7 +246,7 @@ public class TestDirectoryReader extends
     for(FieldInfo fieldInfo : fieldInfos) {
       final String name = fieldInfo.name;
       allFieldNames.add(name);
-      if (fieldInfo.isIndexed()) {
+      if (fieldInfo.getIndexOptions() != IndexOptions.NONE) {
         indexedFieldNames.add(name);
       } else {
         notIndexedFieldNames.add(name);
@@ -254,6 +254,9 @@ public class TestDirectoryReader extends
       if (fieldInfo.hasVectors()) {
         tvFieldNames.add(name);
       }
+      if (fieldInfo.hasVectors()) {
+        tvFieldNames.add(name);
+      }
     }
 
     assertTrue(allFieldNames.contains("keyword"));

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java Sat Nov  8 11:32:18 2014
@@ -209,7 +209,7 @@ public class TestDocValuesIndexing exten
     LeafReader slow = SlowCompositeReaderWrapper.wrap(r);
     FieldInfos fi = slow.getFieldInfos();
     FieldInfo dvInfo = fi.fieldInfo("dv");
-    assertTrue(dvInfo.hasDocValues());
+    assertTrue(dvInfo.getDocValuesType() != DocValuesType.NONE);
     NumericDocValues dv = slow.getNumericDocValues("dv");
     for (int i = 0; i < 50; i++) {
       assertEquals(i, dv.get(i));
@@ -902,13 +902,14 @@ public class TestDocValuesIndexing exten
     dir.close();
   }
 
+  // LUCENE-6049
   public void testExcIndexingDocBeforeDocValues() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
     IndexWriter w = new IndexWriter(dir, iwc);
     Document doc = new Document();
     FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
-    ft.setDocValueType(DocValuesType.SORTED);
+    ft.setDocValuesType(DocValuesType.SORTED);
     ft.freeze();
     Field field = new Field("test", "value", ft);
     field.setTokenStream(new TokenStream() {

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java Sat Nov  8 11:32:18 2014
@@ -92,7 +92,7 @@ public class TestDocumentWriter extends 
     // test that the norms are not present in the segment if
     // omitNorms is true
     for (FieldInfo fi : reader.getFieldInfos()) {
-      if (fi.isIndexed()) {
+      if (fi.getIndexOptions() != IndexOptions.NONE) {
         assertTrue(fi.omitsNorms() == (reader.getNormValues(fi.name) == null));
       }
     }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java Sat Nov  8 11:32:18 2014
@@ -46,6 +46,8 @@ import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.ThreadInterruptedException;
 import org.junit.Test;
 
+// nocommit sometimes fails? ant test  -Dtestcase=TestIndexWriterReader -Dtests.method=testAddIndexesAndDoDeletesThreads -Dtests.seed=964B420DA6617A73 -Dtests.locale=fr -Dtests.timezone=Africa/Maseru -Dtests.asserts=true -Dtests.file.encoding=UTF-8
+
 public class TestIndexWriterReader extends LuceneTestCase {
   
   private final int numThreads = TEST_NIGHTLY ? 5 : 3;

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java Sat Nov  8 11:32:18 2014
@@ -87,7 +87,7 @@ public class TestIndexableField extends 
       }
 
       @Override
-      public DocValuesType docValueType() {
+      public DocValuesType docValuesType() {
         return DocValuesType.NONE;
       }
     };

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java Sat Nov  8 11:32:18 2014
@@ -826,7 +826,7 @@ public class TestNumericDocValuesUpdates
   @Test
   public void testUpdateNumericDVFieldWithSameNameAsPostingField() throws Exception {
     // this used to fail because FieldInfos.Builder neglected to update
-    // globalFieldMaps.docValueTypes map
+    // globalFieldMaps.docValuesTypes map
     Directory dir = newDirectory();
     IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
     IndexWriter writer = new IndexWriter(dir, conf);

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java Sat Nov  8 11:32:18 2014
@@ -88,14 +88,14 @@ public class TestSegmentReader extends L
     for(FieldInfo fieldInfo : reader.getFieldInfos()) {
       final String name = fieldInfo.name;
       allFieldNames.add(name);
-      if (fieldInfo.isIndexed()) {
+      if (fieldInfo.getIndexOptions() != IndexOptions.NONE) {
         indexedFieldNames.add(name);
       } else {
         notIndexedFieldNames.add(name);
       }
       if (fieldInfo.hasVectors()) {
         tvFieldNames.add(name);
-      } else if (fieldInfo.isIndexed()) {
+      } else if (fieldInfo.getIndexOptions() != IndexOptions.NONE) {
         noTVFieldNames.add(name);
       }
     }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java Sat Nov  8 11:32:18 2014
@@ -30,8 +30,8 @@ import org.apache.lucene.search.DocIdSet
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LineFileDocs;
-import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.automaton.Automata;
 import org.apache.lucene.util.automaton.Automaton;
@@ -183,7 +183,6 @@ public class TestTermsEnum extends Lucen
 
   // Tests Terms.intersect
   public void testIntersectRandom() throws IOException {
-
     final Directory dir = newDirectory();
     final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
 
@@ -262,7 +261,7 @@ public class TestTermsEnum extends Lucen
         a = Automata.makeStringUnion(sortedAcceptTerms);
       }
       
-      final CompiledAutomaton c = new CompiledAutomaton(a, true, false);
+      final CompiledAutomaton c = new CompiledAutomaton(a, true, false, 1000000);
 
       final BytesRef[] acceptTermsArray = new BytesRef[acceptTerms.size()];
       final Set<BytesRef> acceptTermsSet = new HashSet<>();

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum2.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum2.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum2.java Sat Nov  8 11:32:18 2014
@@ -38,6 +38,8 @@ import org.apache.lucene.util.LuceneTest
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.automaton.*;
 
+import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
+
 public class TestTermsEnum2 extends LuceneTestCase {
   private Directory dir;
   private IndexReader reader;
@@ -86,7 +88,8 @@ public class TestTermsEnum2 extends Luce
 
     for (int i = 0; i < numIterations; i++) {
       String reg = AutomatonTestUtil.randomRegexp(random());
-      Automaton automaton = Operations.determinize(new RegExp(reg, RegExp.NONE).toAutomaton());
+      Automaton automaton = Operations.determinize(new RegExp(reg, RegExp.NONE).toAutomaton(),
+        DEFAULT_MAX_DETERMINIZED_STATES);
       final List<BytesRef> matchedTerms = new ArrayList<>();
       for(BytesRef t : terms) {
         if (Operations.run(automaton, t.utf8ToString())) {
@@ -97,9 +100,9 @@ public class TestTermsEnum2 extends Luce
       Automaton alternate = Automata.makeStringUnion(matchedTerms);
       //System.out.println("match " + matchedTerms.size() + " " + alternate.getNumberOfStates() + " states, sigma=" + alternate.getStartPoints().length);
       //AutomatonTestUtil.minimizeSimple(alternate);
-      //System.out.println("minmize done");
+      //System.out.println("minimize done");
       AutomatonQuery a1 = new AutomatonQuery(new Term("field", ""), automaton);
-      AutomatonQuery a2 = new AutomatonQuery(new Term("field", ""), alternate);
+      AutomatonQuery a2 = new AutomatonQuery(new Term("field", ""), alternate, Integer.MAX_VALUE);
 
       ScoreDoc[] origHits = searcher.search(a1, 25).scoreDocs;
       ScoreDoc[] newHits = searcher.search(a2, 25).scoreDocs;
@@ -111,7 +114,8 @@ public class TestTermsEnum2 extends Luce
   public void testSeeking() throws Exception {
     for (int i = 0; i < numIterations; i++) {
       String reg = AutomatonTestUtil.randomRegexp(random());
-      Automaton automaton = Operations.determinize(new RegExp(reg, RegExp.NONE).toAutomaton());
+      Automaton automaton = Operations.determinize(new RegExp(reg, RegExp.NONE).toAutomaton(),
+        DEFAULT_MAX_DETERMINIZED_STATES);
       TermsEnum te = MultiFields.getTerms(reader, "field").iterator(null);
       ArrayList<BytesRef> unsortedTerms = new ArrayList<>(terms);
       Collections.shuffle(unsortedTerms, random());
@@ -158,13 +162,15 @@ public class TestTermsEnum2 extends Luce
       Automaton automaton = new RegExp(reg, RegExp.NONE).toAutomaton();
       CompiledAutomaton ca = new CompiledAutomaton(automaton, Operations.isFinite(automaton), false);
       TermsEnum te = MultiFields.getTerms(reader, "field").intersect(ca, null);
-      Automaton expected = Operations.determinize(Operations.intersection(termsAutomaton, automaton));
+      Automaton expected = Operations.determinize(Operations.intersection(termsAutomaton, automaton),
+        DEFAULT_MAX_DETERMINIZED_STATES);
       TreeSet<BytesRef> found = new TreeSet<>();
       while (te.next() != null) {
         found.add(BytesRef.deepCopyOf(te.term()));
       }
       
-      Automaton actual = Operations.determinize(Automata.makeStringUnion(found));
+      Automaton actual = Operations.determinize(Automata.makeStringUnion(found),
+        DEFAULT_MAX_DETERMINIZED_STATES);
       assertTrue(Operations.sameLanguage(expected, actual));
     }
   }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java Sat Nov  8 11:32:18 2014
@@ -18,6 +18,9 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
 import java.util.concurrent.CountDownLatch;
 
 import org.apache.lucene.document.Document;
@@ -30,13 +33,16 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.Rethrow;
 import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.automaton.AutomatonTestUtil;
 import org.apache.lucene.util.automaton.Automata;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.AutomatonTestUtil;
+import org.apache.lucene.util.automaton.Operations;
+
+import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
 
 public class TestAutomatonQuery extends LuceneTestCase {
   private Directory directory;
@@ -118,7 +124,7 @@ public class TestAutomatonQuery extends 
     assertAutomatonHits(0, Operations.intersection(Automata
         .makeChar('a'), Automata.makeChar('b')));
     assertAutomatonHits(1, Operations.minus(Automata.makeCharRange('a', 'b'), 
-        Automata.makeChar('a')));
+        Automata.makeChar('a'), DEFAULT_MAX_DETERMINIZED_STATES));
   }
 
   /**
@@ -209,7 +215,7 @@ public class TestAutomatonQuery extends 
   public void testHashCodeWithThreads() throws Exception {
     final AutomatonQuery queries[] = new AutomatonQuery[1000];
     for (int i = 0; i < queries.length; i++) {
-      queries[i] = new AutomatonQuery(new Term("bogus", "bogus"), AutomatonTestUtil.randomAutomaton(random()));
+      queries[i] = new AutomatonQuery(new Term("bogus", "bogus"), AutomatonTestUtil.randomAutomaton(random()), Integer.MAX_VALUE);
     }
     final CountDownLatch startingGun = new CountDownLatch(1);
     int numThreads = TestUtil.nextInt(random(), 2, 5);
@@ -236,4 +242,13 @@ public class TestAutomatonQuery extends 
       thread.join();
     }
   }
+
+  public void testBiggishAutomaton() {
+    List<BytesRef> terms = new ArrayList<>();
+    while (terms.size() < 3000) {
+      terms.add(new BytesRef(TestUtil.randomUnicodeString(random())));
+    }
+    Collections.sort(terms);
+    new AutomatonQuery(new Term("foo", "bar"), Automata.makeStringUnion(terms), Integer.MAX_VALUE);
+  }
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestComplexExplanations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestComplexExplanations.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestComplexExplanations.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestComplexExplanations.java Sat Nov  8 11:32:18 2014
@@ -27,7 +27,7 @@ import org.apache.lucene.search.spans.*;
  * on the assumption that if the explanations work out right for them,
  * they should work for anything.
  */
-public class TestComplexExplanations extends TestExplanations {
+public class TestComplexExplanations extends BaseExplanationTestCase {
 
   /**
    * Override the Similarity used in our searcher with one that plays

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestFieldValueFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestFieldValueFilter.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestFieldValueFilter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestFieldValueFilter.java Sat Nov  8 11:32:18 2014
@@ -23,12 +23,20 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.FilterLeafReader;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BitDocIdSet;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.SparseFixedBitSet;
 
 /**
  * 
@@ -77,8 +85,8 @@ public class TestFieldValueFilter extend
     }
     IndexReader reader = DirectoryReader.open(directory);
     IndexSearcher searcher = newSearcher(reader);
-    TopDocs search = searcher.search(new TermQuery(new Term("all", "test")),
-        new FieldValueFilter("some"), docs);
+    Filter filter = new FieldValueFilter("some");
+    TopDocs search = searcher.search(new TermQuery(new Term("all", "test")), filter, docs);
     assertEquals(search.totalHits, numDocsWithValue);
     
     ScoreDoc[] scoreDocs = search.scoreDocs;
@@ -90,6 +98,48 @@ public class TestFieldValueFilter extend
     directory.close();
   }
 
+  public void testOptimizations() throws IOException {
+    Directory directory = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), directory,
+        newIndexWriterConfig(new MockAnalyzer(random())));
+    final int docs = atLeast(10);
+    buildIndex(writer, docs);
+    IndexReader reader = DirectoryReader.open(directory);
+    LeafReader leafReader = reader.leaves().get(0).reader();
+    
+    FilterLeafReader filterReader = new FilterLeafReader(leafReader) {
+      @Override
+      public Bits getDocsWithField(String field) throws IOException {
+        switch (field) {
+          case "with_matchall":
+            return new Bits.MatchAllBits(maxDoc());
+          case "with_matchno":
+            return new Bits.MatchNoBits(maxDoc());
+          case "with_bitset":
+            BitSet b = random().nextBoolean() ? new SparseFixedBitSet(maxDoc()) : new FixedBitSet(maxDoc());
+            b.set(random().nextInt(maxDoc()));
+            return b;
+        }
+        return super.getDocsWithField(field);
+      }
+    };
+
+    Filter filter = new FieldValueFilter("with_matchall", true);
+    DocIdSet set = filter.getDocIdSet(filterReader.getContext(), null);
+    assertNull(set);
+
+    filter = new FieldValueFilter("with_matchno");
+    set = filter.getDocIdSet(filterReader.getContext(), null);
+    assertNull(set);
+
+    filter = new FieldValueFilter("with_bitset");
+    set = filter.getDocIdSet(filterReader.getContext(), null);
+    assertTrue(set instanceof BitDocIdSet);
+
+    reader.close();
+    directory.close();
+  }
+
   private int[] buildIndex(RandomIndexWriter writer, int docs)
       throws IOException {
     int[] docStates = new int[docs];

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestRegexpQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestRegexpQuery.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestRegexpQuery.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestRegexpQuery.java Sat Nov  8 11:32:18 2014
@@ -28,11 +28,13 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.automaton.Automata;
-import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.AutomatonProvider;
+import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.RegExp;
 
+import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
+
 /**
  * Some simple regex tests, mostly converted from contrib's TestRegexQuery.
  */
@@ -108,7 +110,8 @@ public class TestRegexpQuery extends Luc
         else return null;
       }
     };
-    RegexpQuery query = new RegexpQuery(newTerm("<quickBrown>"), RegExp.ALL, myProvider);
+    RegexpQuery query = new RegexpQuery(newTerm("<quickBrown>"), RegExp.ALL,
+      myProvider, DEFAULT_MAX_DETERMINIZED_STATES);
     assertEquals(1, searcher.search(query, 5).totalHits);
   }
   

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestSimpleExplanations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestSimpleExplanations.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestSimpleExplanations.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/TestSimpleExplanations.java Sat Nov  8 11:32:18 2014
@@ -22,7 +22,7 @@ import org.apache.lucene.index.Term;
 /**
  * TestExplanations subclass focusing on basic query types
  */
-public class TestSimpleExplanations extends TestExplanations {
+public class TestSimpleExplanations extends BaseExplanationTestCase {
 
   // we focus on queries that don't rewrite to other queries.
   // if we get those covered well, then the ones that rewrite should

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadExplanations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadExplanations.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadExplanations.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadExplanations.java Sat Nov  8 11:32:18 2014
@@ -20,14 +20,14 @@ package org.apache.lucene.search.payload
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.similarities.DefaultSimilarity;
 import org.apache.lucene.search.similarities.Similarity;
-import org.apache.lucene.search.TestExplanations;
+import org.apache.lucene.search.BaseExplanationTestCase;
 import org.apache.lucene.search.spans.SpanQuery;
 import org.apache.lucene.util.BytesRef;
 
 /**
  * TestExplanations subclass focusing on payload queries
  */
-public class TestPayloadExplanations extends TestExplanations {
+public class TestPayloadExplanations extends BaseExplanationTestCase {
   private PayloadFunction functions[] = new PayloadFunction[] { 
       new AveragePayloadFunction(),
       new MinPayloadFunction(),

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/spans/TestSpanExplanations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/spans/TestSpanExplanations.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/spans/TestSpanExplanations.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/search/spans/TestSpanExplanations.java Sat Nov  8 11:32:18 2014
@@ -23,7 +23,7 @@ import org.apache.lucene.search.*;
 /**
  * TestExplanations subclass focusing on span queries
  */
-public class TestSpanExplanations extends TestExplanations {
+public class TestSpanExplanations extends BaseExplanationTestCase {
 
   /* simple SpanTermQueries */
   

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java Sat Nov  8 11:32:18 2014
@@ -36,6 +36,8 @@ import org.apache.lucene.util.UnicodeUti
 import org.apache.lucene.util.automaton.AutomatonTestUtil.RandomAcceptedStrings;
 import org.apache.lucene.util.fst.Util;
 
+import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
+
 public class TestAutomaton extends LuceneTestCase {
 
   public void testBasic() throws Exception {
@@ -111,7 +113,7 @@ public class TestAutomaton extends Lucen
                             Automata.makeAnyString(),
                             Automata.makeString("n"),
                             Automata.makeAnyString()));
-    a = Operations.determinize(a);
+    a = Operations.determinize(a, DEFAULT_MAX_DETERMINIZED_STATES);
     assertTrue(Operations.run(a, "mn"));
     assertTrue(Operations.run(a, "mone"));
     assertFalse(Operations.run(a, "m"));
@@ -122,7 +124,7 @@ public class TestAutomaton extends Lucen
     Automaton a = Operations.union(Arrays.asList(
                             Automata.makeString("foobar"),
                             Automata.makeString("barbaz")));
-    a = Operations.determinize(a);
+    a = Operations.determinize(a, DEFAULT_MAX_DETERMINIZED_STATES);
     assertTrue(Operations.run(a, "foobar"));
     assertTrue(Operations.run(a, "barbaz"));
 
@@ -134,7 +136,7 @@ public class TestAutomaton extends Lucen
                             Automata.makeString("foobar"),
                             Automata.makeString(""),
                             Automata.makeString("barbaz")));
-    a = Operations.determinize(a);
+    a = Operations.determinize(a, DEFAULT_MAX_DETERMINIZED_STATES);
     assertTrue(Operations.run(a, "foobar"));
     assertTrue(Operations.run(a, "barbaz"));
     assertTrue(Operations.run(a, ""));
@@ -144,7 +146,7 @@ public class TestAutomaton extends Lucen
 
   public void testMinimizeSimple() throws Exception {
     Automaton a = Automata.makeString("foobar");
-    Automaton aMin = MinimizationOperations.minimize(a);
+    Automaton aMin = MinimizationOperations.minimize(a, DEFAULT_MAX_DETERMINIZED_STATES);
 
     assertTrue(Operations.sameLanguage(a, aMin));
   }
@@ -152,14 +154,16 @@ public class TestAutomaton extends Lucen
   public void testMinimize2() throws Exception {
     Automaton a = Operations.union(Arrays.asList(Automata.makeString("foobar"),
                                                            Automata.makeString("boobar")));
-    Automaton aMin = MinimizationOperations.minimize(a);
-    assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(a)), aMin));
+    Automaton aMin = MinimizationOperations.minimize(a, DEFAULT_MAX_DETERMINIZED_STATES);
+    assertTrue(Operations.sameLanguage(Operations.determinize(
+      Operations.removeDeadStates(a), DEFAULT_MAX_DETERMINIZED_STATES), aMin));
   }
 
   public void testReverse() throws Exception {
     Automaton a = Automata.makeString("foobar");
     Automaton ra = Operations.reverse(a);
-    Automaton a2 = Operations.determinize(Operations.reverse(ra));
+    Automaton a2 = Operations.determinize(Operations.reverse(ra),
+      DEFAULT_MAX_DETERMINIZED_STATES);
     
     assertTrue(Operations.sameLanguage(a, a2));
   }
@@ -167,7 +171,7 @@ public class TestAutomaton extends Lucen
   public void testOptional() throws Exception {
     Automaton a = Automata.makeString("foobar");
     Automaton a2 = Operations.optional(a);
-    a2 = Operations.determinize(a2);
+    a2 = Operations.determinize(a2, DEFAULT_MAX_DETERMINIZED_STATES);
     
     assertTrue(Operations.run(a, "foobar"));
     assertFalse(Operations.run(a, ""));
@@ -177,7 +181,8 @@ public class TestAutomaton extends Lucen
 
   public void testRepeatAny() throws Exception {
     Automaton a = Automata.makeString("zee");
-    Automaton a2 = Operations.determinize(Operations.repeat(a));
+    Automaton a2 = Operations.determinize(Operations.repeat(a),
+      DEFAULT_MAX_DETERMINIZED_STATES);
     assertTrue(Operations.run(a2, ""));
     assertTrue(Operations.run(a2, "zee"));    
     assertTrue(Operations.run(a2, "zeezee"));
@@ -186,7 +191,8 @@ public class TestAutomaton extends Lucen
 
   public void testRepeatMin() throws Exception {
     Automaton a = Automata.makeString("zee");
-    Automaton a2 = Operations.determinize(Operations.repeat(a, 2));
+    Automaton a2 = Operations.determinize(Operations.repeat(a, 2),
+      DEFAULT_MAX_DETERMINIZED_STATES);
     assertFalse(Operations.run(a2, ""));
     assertFalse(Operations.run(a2, "zee"));    
     assertTrue(Operations.run(a2, "zeezee"));
@@ -195,7 +201,8 @@ public class TestAutomaton extends Lucen
 
   public void testRepeatMinMax1() throws Exception {
     Automaton a = Automata.makeString("zee");
-    Automaton a2 = Operations.determinize(Operations.repeat(a, 0, 2));
+    Automaton a2 = Operations.determinize(Operations.repeat(a, 0, 2),
+      DEFAULT_MAX_DETERMINIZED_STATES);
     assertTrue(Operations.run(a2, ""));
     assertTrue(Operations.run(a2, "zee"));    
     assertTrue(Operations.run(a2, "zeezee"));
@@ -204,7 +211,8 @@ public class TestAutomaton extends Lucen
 
   public void testRepeatMinMax2() throws Exception {
     Automaton a = Automata.makeString("zee");
-    Automaton a2 = Operations.determinize(Operations.repeat(a, 2, 4));
+    Automaton a2 = Operations.determinize(Operations.repeat(a, 2, 4),
+      DEFAULT_MAX_DETERMINIZED_STATES);
     assertFalse(Operations.run(a2, ""));
     assertFalse(Operations.run(a2, "zee"));    
     assertTrue(Operations.run(a2, "zeezee"));
@@ -215,7 +223,8 @@ public class TestAutomaton extends Lucen
 
   public void testComplement() throws Exception {
     Automaton a = Automata.makeString("zee");
-    Automaton a2 = Operations.determinize(Operations.complement(a));
+    Automaton a2 = Operations.determinize(Operations.complement(a,
+      DEFAULT_MAX_DETERMINIZED_STATES), DEFAULT_MAX_DETERMINIZED_STATES);
     assertTrue(Operations.run(a2, ""));
     assertFalse(Operations.run(a2, "zee"));    
     assertTrue(Operations.run(a2, "zeezee"));
@@ -223,7 +232,8 @@ public class TestAutomaton extends Lucen
   }
 
   public void testInterval() throws Exception {
-    Automaton a = Operations.determinize(Automata.makeDecimalInterval(17, 100, 3));
+    Automaton a = Operations.determinize(Automata.makeDecimalInterval(17, 100, 3),
+      DEFAULT_MAX_DETERMINIZED_STATES);
     assertFalse(Operations.run(a, ""));
     assertTrue(Operations.run(a, "017"));
     assertTrue(Operations.run(a, "100"));
@@ -239,7 +249,8 @@ public class TestAutomaton extends Lucen
     a.addTransition(init, fini, 'm');
     a.addTransition(fini, fini, 'm');
     a.finishState();
-    assertEquals(0, Operations.getCommonSuffixBytesRef(a).length);
+    assertEquals(0, Operations.getCommonSuffixBytesRef(a,
+      DEFAULT_MAX_DETERMINIZED_STATES).length);
   }
 
   public void testReverseRandom1() throws Exception {
@@ -248,8 +259,9 @@ public class TestAutomaton extends Lucen
       Automaton a = AutomatonTestUtil.randomAutomaton(random());
       Automaton ra = Operations.reverse(a);
       Automaton rra = Operations.reverse(ra);
-      assertTrue(Operations.sameLanguage(Operations.determinize(Operations.removeDeadStates(a)),
-                                              Operations.determinize(Operations.removeDeadStates(rra))));
+      assertTrue(Operations.sameLanguage(
+        Operations.determinize(Operations.removeDeadStates(a), Integer.MAX_VALUE),
+        Operations.determinize(Operations.removeDeadStates(rra), Integer.MAX_VALUE)));
     }
   }
 
@@ -262,7 +274,7 @@ public class TestAutomaton extends Lucen
         a = Operations.removeDeadStates(a);
       }
       Automaton ra = Operations.reverse(a);
-      Automaton rda = Operations.determinize(ra);
+      Automaton rda = Operations.determinize(ra, Integer.MAX_VALUE);
 
       if (Operations.isEmpty(a)) {
         assertTrue(Operations.isEmpty(rda));
@@ -290,7 +302,8 @@ public class TestAutomaton extends Lucen
   }
 
   public void testAnyStringEmptyString() throws Exception {
-    Automaton a = Operations.determinize(Automata.makeAnyString());
+    Automaton a = Operations.determinize(Automata.makeAnyString(),
+      DEFAULT_MAX_DETERMINIZED_STATES);
     assertTrue(Operations.run(a, ""));
   }
 
@@ -349,9 +362,9 @@ public class TestAutomaton extends Lucen
       }
 
       assertTrue(Operations.sameLanguage(
-                    Operations.determinize(Operations.removeDeadStates(a)),
-                    Operations.determinize(Operations.removeDeadStates(builder.finish()))));
-      
+        Operations.determinize(Operations.removeDeadStates(a), Integer.MAX_VALUE),
+        Operations.determinize(Operations.removeDeadStates(builder.finish()),
+          Integer.MAX_VALUE)));
     }
   }
 
@@ -368,7 +381,8 @@ public class TestAutomaton extends Lucen
     a.finishState();
     assertFalse(Operations.isTotal(a));
     a.setAccept(init, true);
-    assertTrue(Operations.isTotal(MinimizationOperations.minimize(a)));
+    assertTrue(Operations.isTotal(MinimizationOperations.minimize(a,
+      DEFAULT_MAX_DETERMINIZED_STATES)));
   }
 
   public void testMinimizeEmpty() throws Exception {
@@ -377,7 +391,7 @@ public class TestAutomaton extends Lucen
     int fini = a.createState();
     a.addTransition(init, fini, 'a');
     a.finishState();
-    a = MinimizationOperations.minimize(a);
+    a = MinimizationOperations.minimize(a, DEFAULT_MAX_DETERMINIZED_STATES);
     assertEquals(0, a.getNumStates());
   }
 
@@ -387,26 +401,29 @@ public class TestAutomaton extends Lucen
     Automaton a3 = Automata.makeString("beebar");
     Automaton a = Operations.union(Arrays.asList(a1, a2, a3));
     if (random().nextBoolean()) {
-      a = Operations.determinize(a);
+      a = Operations.determinize(a, DEFAULT_MAX_DETERMINIZED_STATES);
     } else if (random().nextBoolean()) {
-      a = MinimizationOperations.minimize(a);
+      a = MinimizationOperations.minimize(a, DEFAULT_MAX_DETERMINIZED_STATES);
     }
     assertMatches(a, "foobar", "beebar", "boobar");
 
-    Automaton a4 = Operations.determinize(Operations.minus(a, a2));
+    Automaton a4 = Operations.determinize(Operations.minus(a, a2,
+      DEFAULT_MAX_DETERMINIZED_STATES), DEFAULT_MAX_DETERMINIZED_STATES);
     
     assertTrue(Operations.run(a4, "foobar"));
     assertFalse(Operations.run(a4, "boobar"));
     assertTrue(Operations.run(a4, "beebar"));
     assertMatches(a4, "foobar", "beebar");
 
-    a4 = Operations.determinize(Operations.minus(a4, a1));
+    a4 = Operations.determinize(Operations.minus(a4, a1,
+      DEFAULT_MAX_DETERMINIZED_STATES), DEFAULT_MAX_DETERMINIZED_STATES);
     assertFalse(Operations.run(a4, "foobar"));
     assertFalse(Operations.run(a4, "boobar"));
     assertTrue(Operations.run(a4, "beebar"));
     assertMatches(a4, "beebar");
 
-    a4 = Operations.determinize(Operations.minus(a4, a3));
+    a4 = Operations.determinize(Operations.minus(a4, a3,
+      DEFAULT_MAX_DETERMINIZED_STATES), DEFAULT_MAX_DETERMINIZED_STATES);
     assertFalse(Operations.run(a4, "foobar"));
     assertFalse(Operations.run(a4, "boobar"));
     assertFalse(Operations.run(a4, "beebar"));
@@ -415,7 +432,7 @@ public class TestAutomaton extends Lucen
 
   public void testOneInterval() throws Exception {
     Automaton a = Automata.makeDecimalInterval(999, 1032, 0);
-    a = Operations.determinize(a);
+    a = Operations.determinize(a, DEFAULT_MAX_DETERMINIZED_STATES);
     assertTrue(Operations.run(a, "0999"));
     assertTrue(Operations.run(a, "00999"));
     assertTrue(Operations.run(a, "000999"));
@@ -423,7 +440,7 @@ public class TestAutomaton extends Lucen
 
   public void testAnotherInterval() throws Exception {
     Automaton a = Automata.makeDecimalInterval(1, 2, 0);
-    a = Operations.determinize(a);
+    a = Operations.determinize(a, DEFAULT_MAX_DETERMINIZED_STATES);
     assertTrue(Operations.run(a, "01"));
   }
 
@@ -445,9 +462,10 @@ public class TestAutomaton extends Lucen
       }
       String prefix = b.toString();
 
-      Automaton a = Operations.determinize(Automata.makeDecimalInterval(min, max, digits));
+      Automaton a = Operations.determinize(Automata.makeDecimalInterval(min, max, digits),
+        DEFAULT_MAX_DETERMINIZED_STATES);
       if (random().nextBoolean()) {
-        a = MinimizationOperations.minimize(a);
+        a = MinimizationOperations.minimize(a, DEFAULT_MAX_DETERMINIZED_STATES);
       }
       String mins = Integer.toString(min);
       String maxs = Integer.toString(max);
@@ -487,7 +505,8 @@ public class TestAutomaton extends Lucen
       expected.add(Util.toUTF32(s, ints));
     }
 
-    assertEquals(expected, Operations.getFiniteStrings(Operations.determinize(a), -1)); 
+    assertEquals(expected, Operations.getFiniteStrings(Operations.determinize(a,
+      DEFAULT_MAX_DETERMINIZED_STATES), -1)); 
   }
 
   public void testConcatenatePreservesDet() throws Exception {
@@ -578,13 +597,13 @@ public class TestAutomaton extends Lucen
       if (VERBOSE) {
         System.out.println("  randomNoOp: determinize");
       }
-      return Operations.determinize(a);
+      return Operations.determinize(a, Integer.MAX_VALUE);
     case 1:
       if (a.getNumStates() < 100) {
         if (VERBOSE) {
           System.out.println("  randomNoOp: minimize");
         }
-        return MinimizationOperations.minimize(a);
+        return MinimizationOperations.minimize(a, DEFAULT_MAX_DETERMINIZED_STATES);
       } else {
         if (VERBOSE) {
           System.out.println("  randomNoOp: skip op=minimize: too many states (" + a.getNumStates() + ")");
@@ -725,7 +744,7 @@ public class TestAutomaton extends Lucen
         if (VERBOSE) {
           System.out.println("  op=determinize");
         }
-        a = Operations.determinize(a);
+        a = Operations.determinize(a, Integer.MAX_VALUE);
         assertTrue(a.isDeterministic());
         break;
 
@@ -735,7 +754,7 @@ public class TestAutomaton extends Lucen
             System.out.println("  op=minimize");
           }
           // minimize
-          a = MinimizationOperations.minimize(a);
+          a = MinimizationOperations.minimize(a, DEFAULT_MAX_DETERMINIZED_STATES);
         } else if (VERBOSE) {
           System.out.println("  skip op=minimize: too many states (" + a.getNumStates() + ")");
         }
@@ -791,7 +810,7 @@ public class TestAutomaton extends Lucen
               assertTrue(removed);
             }
             Automaton a2 = unionTerms(toRemove);
-            a = Operations.minus(a, a2);
+            a = Operations.minus(a, a2, DEFAULT_MAX_DETERMINIZED_STATES);
           }
         }
         break;
@@ -831,7 +850,7 @@ public class TestAutomaton extends Lucen
             }
           }
           Automaton a2 = randomNoOp(Operations.union(as));
-          a = Operations.minus(a, a2);
+          a = Operations.minus(a, a2, DEFAULT_MAX_DETERMINIZED_STATES);
         }
         break;
 
@@ -868,9 +887,9 @@ public class TestAutomaton extends Lucen
 
           Automaton a2 = Operations.union(as);
           if (random().nextBoolean()) {
-            a2 = Operations.determinize(a2);
+            a2 = Operations.determinize(a2, DEFAULT_MAX_DETERMINIZED_STATES);
           } else if (random().nextBoolean()) {
-            a2 = MinimizationOperations.minimize(a2);
+            a2 = MinimizationOperations.minimize(a2, DEFAULT_MAX_DETERMINIZED_STATES);
           }
           a = Operations.intersection(a, a2);
 
@@ -944,7 +963,7 @@ public class TestAutomaton extends Lucen
         if (VERBOSE) {
           System.out.println("  op=remove the empty string");
         }
-        a = Operations.minus(a, Automata.makeEmptyString());
+        a = Operations.minus(a, Automata.makeEmptyString(), DEFAULT_MAX_DETERMINIZED_STATES);
         terms.remove(new BytesRef());
         break;
 
@@ -1024,7 +1043,7 @@ public class TestAutomaton extends Lucen
       assertTrue(Operations.isFinite(a));
       assertFalse(Operations.isTotal(a));
 
-      Automaton detA = Operations.determinize(a);
+      Automaton detA = Operations.determinize(a, DEFAULT_MAX_DETERMINIZED_STATES);
 
       // Make sure all terms are accepted:
       IntsRefBuilder scratch = new IntsRefBuilder();
@@ -1058,8 +1077,10 @@ public class TestAutomaton extends Lucen
       }
 
       // Use sameLanguage:
-      Automaton a2 = Operations.removeDeadStates(Operations.determinize(unionTerms(terms)));
-      assertTrue(Operations.sameLanguage(a2, Operations.removeDeadStates(Operations.determinize(a))));
+      Automaton a2 = Operations.removeDeadStates(Operations.determinize(unionTerms(terms),
+        Integer.MAX_VALUE));
+      assertTrue(Operations.sameLanguage(a2, Operations.removeDeadStates(Operations.determinize(a,
+        Integer.MAX_VALUE))));
 
       // Do same check, in UTF8 space
       Automaton utf8 = randomNoOp(new UTF32ToUTF8().convert(a));
@@ -1098,7 +1119,7 @@ public class TestAutomaton extends Lucen
 
       Automaton a = Automata.makeBinaryInterval(minTerm, minInclusive, maxTerm, maxInclusive);
 
-      Automaton minA = MinimizationOperations.minimize(a);
+      Automaton minA = MinimizationOperations.minimize(a, Integer.MAX_VALUE);
       if (minA.getNumStates() != a.getNumStates()) {
         assertTrue(minA.getNumStates() < a.getNumStates());
         System.out.println("Original was not minimal:");