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 [3/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/test/org/apache/lucene/util/automaton/TestCompiledAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestCompiledAutomaton.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestCompiledAutomaton.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestCompiledAutomaton.java Sat Nov  8 11:32:18 2014
@@ -31,14 +31,14 @@ import org.apache.lucene.util.TestUtil;
 
 public class TestCompiledAutomaton extends LuceneTestCase {
 
-  private CompiledAutomaton build(String... strings) {
+  private CompiledAutomaton build(int maxDeterminizedStates, String... strings) {
     final List<BytesRef> terms = new ArrayList<>();
     for(String s : strings) {
       terms.add(new BytesRef(s));
     }
     Collections.sort(terms);
     final Automaton a = DaciukMihovAutomatonBuilder.build(terms);
-    return new CompiledAutomaton(a, true, false);
+    return new CompiledAutomaton(a, true, false, maxDeterminizedStates);
   }
 
   private void testFloor(CompiledAutomaton c, String input, String expected) {
@@ -53,8 +53,8 @@ public class TestCompiledAutomaton exten
     }
   }
 
-  private void testTerms(String[] terms) throws Exception {
-    final CompiledAutomaton c = build(terms);
+  private void testTerms(int maxDeterminizedStates, String[] terms) throws Exception {
+    final CompiledAutomaton c = build(maxDeterminizedStates, terms);
     final BytesRef[] termBytes = new BytesRef[terms.length];
     for(int idx=0;idx<terms.length;idx++) {
       termBytes[idx] = new BytesRef(terms[idx]);
@@ -100,7 +100,7 @@ public class TestCompiledAutomaton exten
     while(terms.size() != numTerms) {
       terms.add(randomString());
     }
-    testTerms(terms.toArray(new String[terms.size()]));
+    testTerms(numTerms * 100, terms.toArray(new String[terms.size()]));
   }
 
   private String randomString() {
@@ -109,7 +109,8 @@ public class TestCompiledAutomaton exten
   }
 
   public void testBasic() throws Exception {
-    CompiledAutomaton c = build("fob", "foo", "goo");
+    CompiledAutomaton c = build(Operations.DEFAULT_MAX_DETERMINIZED_STATES,
+      "fob", "foo", "goo");
     testFloor(c, "goo", "goo");
     testFloor(c, "ga", "foo");
     testFloor(c, "g", "foo");

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestDeterminism.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestDeterminism.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestDeterminism.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestDeterminism.java Sat Nov  8 11:32:18 2014
@@ -19,6 +19,8 @@ package org.apache.lucene.util.automaton
 
 import org.apache.lucene.util.LuceneTestCase;
 
+import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
+
 /**
  * Not completely thorough, but tries to test determinism correctness
  * somewhat randomly.
@@ -39,29 +41,32 @@ public class TestDeterminism extends Luc
     for (int i = 0; i < num; i++) {
       Automaton a = AutomatonTestUtil.randomAutomaton(random());
       a = AutomatonTestUtil.determinizeSimple(a);
-      Automaton b = Operations.determinize(a);
+      Automaton b = Operations.determinize(a, Integer.MAX_VALUE);
       // TODO: more verifications possible?
       assertTrue(Operations.sameLanguage(a, b));
     }
   }
   
   private static void assertAutomaton(Automaton a) {
-    a = Operations.determinize(Operations.removeDeadStates(a));
+    a = Operations.determinize(Operations.removeDeadStates(a), DEFAULT_MAX_DETERMINIZED_STATES);
 
     // complement(complement(a)) = a
-    Automaton equivalent = Operations.complement(Operations.complement(a));
+    Automaton equivalent = Operations.complement(Operations.complement(a,
+      DEFAULT_MAX_DETERMINIZED_STATES), DEFAULT_MAX_DETERMINIZED_STATES);
     assertTrue(Operations.sameLanguage(a, equivalent));
     
     // a union a = a
-    equivalent = Operations.determinize(Operations.removeDeadStates(Operations.union(a, a)));
+    equivalent = Operations.determinize(Operations.removeDeadStates(Operations.union(a, a)),
+      DEFAULT_MAX_DETERMINIZED_STATES);
     assertTrue(Operations.sameLanguage(a, equivalent));
     
     // a intersect a = a
-    equivalent = Operations.determinize(Operations.removeDeadStates(Operations.intersection(a, a)));
+    equivalent = Operations.determinize(Operations.removeDeadStates(Operations.intersection(a, a)),
+      DEFAULT_MAX_DETERMINIZED_STATES);
     assertTrue(Operations.sameLanguage(a, equivalent));
     
     // a minus a = empty
-    Automaton empty = Operations.minus(a, a);
+    Automaton empty = Operations.minus(a, a, DEFAULT_MAX_DETERMINIZED_STATES);
     assertTrue(Operations.isEmpty(empty));
     
     // as long as don't accept the empty string
@@ -70,7 +75,8 @@ public class TestDeterminism extends Luc
       //System.out.println("test " + a);
       Automaton optional = Operations.optional(a);
       //System.out.println("optional " + optional);
-      equivalent = Operations.minus(optional, Automata.makeEmptyString());
+      equivalent = Operations.minus(optional, Automata.makeEmptyString(),
+        DEFAULT_MAX_DETERMINIZED_STATES);
       //System.out.println("equiv " + equivalent);
       assertTrue(Operations.sameLanguage(a, equivalent));
     }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestDeterminizeLexicon.java Sat Nov  8 11:32:18 2014
@@ -50,12 +50,12 @@ public class TestDeterminizeLexicon exte
   public void assertLexicon() throws Exception {
     Collections.shuffle(automata, random());
     Automaton lex = Operations.union(automata);
-    lex = Operations.determinize(lex);
+    lex = Operations.determinize(lex, 1000000);
     assertTrue(Operations.isFinite(lex));
     for (String s : terms) {
       assertTrue(Operations.run(lex, s));
     }
-    final ByteRunAutomaton lexByte = new ByteRunAutomaton(lex);
+    final ByteRunAutomaton lexByte = new ByteRunAutomaton(lex, false, 1000000);
     for (String s : terms) {
       byte bytes[] = s.getBytes(StandardCharsets.UTF_8);
       assertTrue(lexByte.run(bytes, 0, bytes.length));

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestLevenshteinAutomata.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestLevenshteinAutomata.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestLevenshteinAutomata.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestLevenshteinAutomata.java Sat Nov  8 11:32:18 2014
@@ -22,6 +22,8 @@ import java.util.List;
 
 import org.apache.lucene.util.LuceneTestCase;
 
+import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
+
 public class TestLevenshteinAutomata extends LuceneTestCase {
  
   public void testLev0() throws Exception {
@@ -121,11 +123,11 @@ public class TestLevenshteinAutomata ext
   private Automaton naiveLev1(String s) {
     Automaton a = Automata.makeString(s);
     a = Operations.union(a, insertionsOf(s));
-    a = MinimizationOperations.minimize(a);
+    a = MinimizationOperations.minimize(a, DEFAULT_MAX_DETERMINIZED_STATES);
     a = Operations.union(a, deletionsOf(s));
-    a = MinimizationOperations.minimize(a);
+    a = MinimizationOperations.minimize(a, DEFAULT_MAX_DETERMINIZED_STATES);
     a = Operations.union(a, substitutionsOf(s));
-    a = MinimizationOperations.minimize(a);
+    a = MinimizationOperations.minimize(a, DEFAULT_MAX_DETERMINIZED_STATES);
     
     return a;
   }
@@ -137,7 +139,7 @@ public class TestLevenshteinAutomata ext
   private Automaton naiveLev1T(String s) {
     Automaton a = naiveLev1(s);
     a = Operations.union(a, transpositionsOf(s));
-    a = MinimizationOperations.minimize(a);
+    a = MinimizationOperations.minimize(a, DEFAULT_MAX_DETERMINIZED_STATES);
     return a;
   }
   
@@ -156,7 +158,7 @@ public class TestLevenshteinAutomata ext
     }
     
     Automaton a = Operations.union(list);
-    a = MinimizationOperations.minimize(a);
+    a = MinimizationOperations.minimize(a, DEFAULT_MAX_DETERMINIZED_STATES);
     return a;
   }
   
@@ -174,7 +176,7 @@ public class TestLevenshteinAutomata ext
     }
     
     Automaton a = Operations.union(list);
-    a = MinimizationOperations.minimize(a);
+    a = MinimizationOperations.minimize(a, DEFAULT_MAX_DETERMINIZED_STATES);
     return a;
   }
   
@@ -193,7 +195,7 @@ public class TestLevenshteinAutomata ext
     }
     
     Automaton a = Operations.union(list);
-    a = MinimizationOperations.minimize(a);
+    a = MinimizationOperations.minimize(a, DEFAULT_MAX_DETERMINIZED_STATES);
     return a;
   }
   
@@ -218,7 +220,7 @@ public class TestLevenshteinAutomata ext
       }
     }
     Automaton a = Operations.union(list);
-    a = MinimizationOperations.minimize(a);
+    a = MinimizationOperations.minimize(a, DEFAULT_MAX_DETERMINIZED_STATES);
     return a;
   }
   

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestMinimize.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestMinimize.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestMinimize.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestMinimize.java Sat Nov  8 11:32:18 2014
@@ -28,8 +28,10 @@ public class TestMinimize extends Lucene
     int num = atLeast(200);
     for (int i = 0; i < num; i++) {
       Automaton a = AutomatonTestUtil.randomAutomaton(random());
-      Automaton la = Operations.determinize(Operations.removeDeadStates(a));
-      Automaton lb = MinimizationOperations.minimize(a);
+      Automaton la = Operations.determinize(Operations.removeDeadStates(a),
+        Integer.MAX_VALUE);
+      Automaton lb = MinimizationOperations.minimize(a,
+        Integer.MAX_VALUE);
       assertTrue(Operations.sameLanguage(la, lb));
     }
   }
@@ -42,7 +44,8 @@ public class TestMinimize extends Lucene
     for (int i = 0; i < num; i++) {
       Automaton a = AutomatonTestUtil.randomAutomaton(random());
       a = AutomatonTestUtil.minimizeSimple(a);
-      Automaton b = MinimizationOperations.minimize(a);
+      Automaton b = MinimizationOperations.minimize(a,
+        Integer.MAX_VALUE);
       assertTrue(Operations.sameLanguage(a, b));
       assertEquals(a.getNumStates(), b.getNumStates());
       int numStates = a.getNumStates();
@@ -62,6 +65,6 @@ public class TestMinimize extends Lucene
   
   /** n^2 space usage in Hopcroft minimization? */
   public void testMinimizeHuge() {
-    new RegExp("+-*(A|.....|BC)*]", RegExp.NONE).toAutomaton();
+    new RegExp("+-*(A|.....|BC)*]", RegExp.NONE).toAutomaton(1000000);
   }
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestOperations.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestOperations.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/automaton/TestOperations.java Sat Nov  8 11:32:18 2014
@@ -24,6 +24,8 @@ import org.apache.lucene.util.fst.Util;
 
 import com.carrotsearch.randomizedtesting.generators.RandomInts;
 
+import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
+
 public class TestOperations extends LuceneTestCase {
   /** Test string union. */
   public void testStringUnion() {
@@ -51,7 +53,8 @@ public class TestOperations extends Luce
     for (BytesRef bref : strings) {
       eachIndividual[i++] = Automata.makeString(bref.utf8ToString());
     }
-    return Operations.determinize(Operations.union(Arrays.asList(eachIndividual)));
+    return Operations.determinize(Operations.union(Arrays.asList(eachIndividual)),
+      DEFAULT_MAX_DETERMINIZED_STATES);
   }
 
   /** Test concatenation with empty language returns empty */
@@ -71,12 +74,12 @@ public class TestOperations extends Luce
     Automaton concat1 = Operations.concatenate(expandedSingleton, nfa);
     Automaton concat2 = Operations.concatenate(singleton, nfa);
     assertFalse(concat2.isDeterministic());
-    assertTrue(Operations.sameLanguage(Operations.determinize(concat1),
-                                       Operations.determinize(concat2)));
-    assertTrue(Operations.sameLanguage(Operations.determinize(nfa),
-                                       Operations.determinize(concat1)));
-    assertTrue(Operations.sameLanguage(Operations.determinize(nfa),
-                                       Operations.determinize(concat2)));
+    assertTrue(Operations.sameLanguage(Operations.determinize(concat1, 100),
+                                       Operations.determinize(concat2, 100)));
+    assertTrue(Operations.sameLanguage(Operations.determinize(nfa, 100),
+                                       Operations.determinize(concat1, 100)));
+    assertTrue(Operations.sameLanguage(Operations.determinize(nfa, 100),
+                                       Operations.determinize(concat2, 100)));
   }
 
   public void testGetRandomAcceptedString() throws Throwable {
@@ -86,7 +89,7 @@ public class TestOperations extends Luce
 
       final RegExp re = new RegExp(AutomatonTestUtil.randomRegexp(random()), RegExp.NONE);
       //System.out.println("TEST i=" + i + " re=" + re);
-      final Automaton a = Operations.determinize(re.toAutomaton());
+      final Automaton a = Operations.determinize(re.toAutomaton(), DEFAULT_MAX_DETERMINIZED_STATES);
       assertFalse(Operations.isEmpty(a));
 
       final AutomatonTestUtil.RandomAcceptedStrings rx = new AutomatonTestUtil.RandomAcceptedStrings(a);
@@ -137,7 +140,7 @@ public class TestOperations extends Luce
    */
   public void testFiniteStringsBasic() {
     Automaton a = Operations.union(Automata.makeString("dog"), Automata.makeString("duck"));
-    a = MinimizationOperations.minimize(a);
+    a = MinimizationOperations.minimize(a, DEFAULT_MAX_DETERMINIZED_STATES);
     Set<IntsRef> strings = getFiniteStrings(a, -1, true);
     assertEquals(2, strings.size());
     IntsRefBuilder dog = new IntsRefBuilder();
@@ -190,7 +193,7 @@ public class TestOperations extends Luce
     // TODO: what other random things can we do here...
     Automaton a = Operations.union(automata);
     if (random().nextBoolean()) {
-      a = MinimizationOperations.minimize(a);
+      a = MinimizationOperations.minimize(a, 1000000);
       if (VERBOSE) {
         System.out.println("TEST: a.minimize numStates=" + a.getNumStates());
       }
@@ -198,7 +201,7 @@ public class TestOperations extends Luce
       if (VERBOSE) {
         System.out.println("TEST: a.determinize");
       }
-      a = Operations.determinize(a);
+      a = Operations.determinize(a, 1000000);
     } else if (random().nextBoolean()) {
       if (VERBOSE) {
         System.out.println("TEST: a.removeDeadStates");

Modified: lucene/dev/branches/lucene6005/lucene/facet/src/test/org/apache/lucene/facet/TestRandomSamplingFacetsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/facet/src/test/org/apache/lucene/facet/TestRandomSamplingFacetsCollector.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/facet/src/test/org/apache/lucene/facet/TestRandomSamplingFacetsCollector.java (original)
+++ lucene/dev/branches/lucene6005/lucene/facet/src/test/org/apache/lucene/facet/TestRandomSamplingFacetsCollector.java Sat Nov  8 11:32:18 2014
@@ -1,5 +1,6 @@
 package org.apache.lucene.facet;
 
+import java.util.List;
 import java.util.Random;
 
 import org.apache.lucene.document.Document;
@@ -12,6 +13,7 @@ import org.apache.lucene.facet.taxonomy.
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MultiCollector;
 import org.apache.lucene.search.TermQuery;
@@ -37,29 +39,40 @@ import org.apache.lucene.util.IOUtils;
 
 public class TestRandomSamplingFacetsCollector extends FacetTestCase {
   
+  // The first 50 chi-square value for p-value=0.05, taken from:
+  // http://en.wikibooks.org/wiki/Engineering_Tables/Chi-Squared_Distibution
+  private static final float[] CHI_SQUARE_VALUES = new float[] {0.0f, 3.841f,
+      5.991f, 7.815f, 9.488f, 11.07f, 12.592f, 14.067f, 15.507f, 16.919f,
+      18.307f, 19.675f, 21.026f, 22.362f, 23.685f, 24.996f, 26.296f, 27.587f,
+      28.869f, 30.144f, 31.41f, 32.671f, 33.924f, 35.172f, 36.415f, 37.652f,
+      38.885f, 40.113f, 41.337f, 42.557f, 43.773f, 44.985f, 46.194f, 47.4f,
+      48.602f, 49.802f, 50.998f, 52.192f, 53.384f, 54.572f, 55.758f, 56.942f,
+      58.124f, 59.304f, 60.481f, 61.656f, 62.83f, 64.001f, 65.171f, 66.339f,
+      67.505f};
+  
   public void testRandomSampling() throws Exception {
     Directory dir = newDirectory();
     Directory taxoDir = newDirectory();
     
+    Random random = random();
     DirectoryTaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir);
-    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+    RandomIndexWriter writer = new RandomIndexWriter(random, dir);
     
     FacetsConfig config = new FacetsConfig();
     
+    final int numCategories = 10;
     int numDocs = atLeast(10000);
     for (int i = 0; i < numDocs; i++) {
       Document doc = new Document();
       doc.add(new StringField("EvenOdd", (i % 2 == 0) ? "even" : "odd", Store.NO));
-      doc.add(new FacetField("iMod10", String.valueOf(i % 10)));
+      doc.add(new FacetField("iMod10", Integer.toString(i % numCategories)));
       writer.addDocument(config.build(taxoWriter, doc));
     }
-    Random random = random();
     
     // NRT open
     IndexSearcher searcher = newSearcher(writer.getReader());
     TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter);
-    writer.close();
-    IOUtils.close(taxoWriter);
+    IOUtils.close(writer, taxoWriter);
     
     // Test empty results
     RandomSamplingFacetsCollector collectRandomZeroResults = new RandomSamplingFacetsCollector(numDocs / 10, random.nextLong());
@@ -80,61 +93,55 @@ public class TestRandomSamplingFacetsCol
     // Use a query to select half of the documents.
     TermQuery query = new TermQuery(new Term("EvenOdd", "even"));
     
-    // there will be 5 facet values (0, 2, 4, 6 and 8), as only the even (i %
-    // 10) are hits.
-    // there is a REAL small chance that one of the 5 values will be missed when
-    // sampling.
-    // but is that 0.8 (chance not to take a value) ^ 2000 * 5 (any can be
-    // missing) ~ 10^-193
-    // so that is probably not going to happen.
-    int maxNumChildren = 5;
-    
-    RandomSamplingFacetsCollector random100Percent = new RandomSamplingFacetsCollector(numDocs, random.nextLong()); // no sampling
-    RandomSamplingFacetsCollector random10Percent = new RandomSamplingFacetsCollector(numDocs / 10, random.nextLong()); // 10 % of total docs, 20% of the hits
+    RandomSamplingFacetsCollector random10Percent = new RandomSamplingFacetsCollector(numDocs / 10, random.nextLong()); // 10% of total docs, 20% of the hits
 
     FacetsCollector fc = new FacetsCollector();
     
-    searcher.search(query, MultiCollector.wrap(fc, random100Percent, random10Percent));
+    searcher.search(query, MultiCollector.wrap(fc, random10Percent));
     
-    FastTaxonomyFacetCounts random10FacetCounts = new FastTaxonomyFacetCounts(taxoReader, config, random10Percent);
-    FastTaxonomyFacetCounts random100FacetCounts = new FastTaxonomyFacetCounts(taxoReader, config, random100Percent);
-    FastTaxonomyFacetCounts exactFacetCounts = new FastTaxonomyFacetCounts(taxoReader, config, fc);
-    
-    FacetResult random10Result = random10Percent.amortizeFacetCounts(random10FacetCounts.getTopChildren(10, "iMod10"), config, searcher);
-    FacetResult random100Result = random100FacetCounts.getTopChildren(10, "iMod10");
-    FacetResult exactResult = exactFacetCounts.getTopChildren(10, "iMod10");
-    
-    assertEquals(random100Result, exactResult);
-    
-    // we should have five children, but there is a small chance we have less.
-    // (see above).
-    assertTrue(random10Result.childCount <= maxNumChildren);
-    // there should be one child at least.
-    assertTrue(random10Result.childCount >= 1);
-    
-    // now calculate some statistics to determine if the sampled result is 'ok'.
-    // because random sampling is used, the results will vary each time.
-    int sum = 0;
-    for (LabelAndValue lav : random10Result.labelValues) {
-      sum += lav.value.intValue();
+    final List<MatchingDocs> matchingDocs = random10Percent.getMatchingDocs();
+
+    // count the total hits and sampled docs, also store the number of sampled
+    // docs per segment
+    int totalSampledDocs = 0, totalHits = 0;
+    int[] numSampledDocs = new int[matchingDocs.size()];
+//    System.out.println("numSegments=" + numSampledDocs.length);
+    for (int i = 0; i < numSampledDocs.length; i++) {
+      MatchingDocs md = matchingDocs.get(i);
+      final DocIdSetIterator iter = md.bits.iterator();
+      while (iter.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) ++numSampledDocs[i];
+      totalSampledDocs += numSampledDocs[i];
+      totalHits += md.totalHits;
     }
-    float mu = (float) sum / (float) maxNumChildren;
     
-    float variance = 0;
-    for (LabelAndValue lav : random10Result.labelValues) {
-      variance += Math.pow((mu - lav.value.intValue()), 2);
+    // compute the chi-square value for the sampled documents' distribution
+    float chi_square = 0;
+    for (int i = 0; i < numSampledDocs.length; i++) {
+      MatchingDocs md = matchingDocs.get(i);
+      float ei = (float) md.totalHits / totalHits;
+      if (ei > 0.0f) {
+        float oi = (float) numSampledDocs[i] / totalSampledDocs;
+        chi_square += (Math.pow(ei - oi, 2) / ei);
+      }
     }
-    variance = variance / maxNumChildren;
-    float sigma = (float) Math.sqrt(variance);
     
-    // we query only half the documents and have 5 categories. The average
-    // number of docs in a category will thus be the total divided by 5*2
-    float targetMu = numDocs / (5.0f * 2.0f);
-    
-    // the average should be in the range and the standard deviation should not
-    // be too great
-    assertTrue(sigma < 200);
-    assertTrue(targetMu - 3 * sigma < mu && mu < targetMu + 3 * sigma);
+    // Verify that the chi-square value isn't too big. According to
+    // http://en.wikipedia.org/wiki/Chi-squared_distribution#Table_of_.CF.872_value_vs_p-value,
+    // we basically verify that there is a really small chance of hitting a very
+    // bad sample (p-value < 0.05), for n-degrees of freedom. The number 'n' depends
+    // on the number of segments.
+    assertTrue("chisquare not statistically significant enough: " + chi_square, chi_square < CHI_SQUARE_VALUES[numSampledDocs.length]);
+    
+    // Test amortized counts - should be 5X the sampled count, but maximum numDocs/10
+    final FastTaxonomyFacetCounts random10FacetCounts = new FastTaxonomyFacetCounts(taxoReader, config, random10Percent);
+    final FacetResult random10Result = random10FacetCounts.getTopChildren(10, "iMod10");
+    final FacetResult amortized10Result = random10Percent.amortizeFacetCounts(random10Result, config, searcher);
+    for (int i = 0; i < amortized10Result.labelValues.length; i++) {
+      LabelAndValue amortized = amortized10Result.labelValues[i];
+      LabelAndValue sampled = random10Result.labelValues[i];
+      // since numDocs may not divide by 10 exactly, allow for some slack in the amortized count 
+      assertEquals(amortized.value.floatValue(), Math.min(5 * sampled.value.floatValue(), numDocs / 10.f), 1.0);
+    }
     
     IOUtils.close(searcher.getIndexReader(), taxoReader, dir, taxoDir);
   }

Modified: lucene/dev/branches/lucene6005/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldQueryTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldQueryTest.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldQueryTest.java (original)
+++ lucene/dev/branches/lucene6005/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldQueryTest.java Sat Nov  8 11:32:18 2014
@@ -25,6 +25,7 @@ import java.util.Set;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.ConstantScoreQuery;
 import org.apache.lucene.search.DocIdSet;
@@ -32,7 +33,6 @@ import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.FilteredQuery;
 import org.apache.lucene.search.PrefixQuery;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.RegexpQuery;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TermRangeQuery;
@@ -903,7 +903,7 @@ public class FieldQueryTest extends Abst
   public void testRegexpQuery() throws Exception {
     makeIndexStrMV();
     Term term = new Term(F, "d[a-z].g");
-    defgMultiTermQueryTest(new RegexpQuery (term));
+    defgMultiTermQueryTest(new RegexpQuery(term));
   }
 
   public void testRangeQuery() throws Exception {

Modified: lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/uninverting/DocTermOrds.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/uninverting/DocTermOrds.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/uninverting/DocTermOrds.java (original)
+++ lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/uninverting/DocTermOrds.java Sat Nov  8 11:32:18 2014
@@ -24,12 +24,13 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.lucene.codecs.PostingsFormat; // javadocs
-import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
@@ -282,7 +283,7 @@ public class DocTermOrds implements Acco
   /** Call this only once (if you subclass!) */
   protected void uninvert(final LeafReader reader, Bits liveDocs, final BytesRef termPrefix) throws IOException {
     final FieldInfo info = reader.getFieldInfos().fieldInfo(field);
-    if (info != null && info.hasDocValues()) {
+    if (info != null && info.getDocValuesType() != DocValuesType.NONE) {
       throw new IllegalStateException("Type mismatch: " + field + " was indexed as " + info.getDocValuesType());
     }
     //System.out.println("DTO uninvert field=" + field + " prefix=" + termPrefix);

Modified: lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCacheImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCacheImpl.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCacheImpl.java (original)
+++ lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCacheImpl.java Sat Nov  8 11:32:18 2014
@@ -26,11 +26,13 @@ import java.util.List;
 import java.util.Map;
 import java.util.WeakHashMap;
 
-import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SegmentReader;
 import org.apache.lucene.index.SortedDocValues;
@@ -356,9 +358,9 @@ class FieldCacheImpl implements FieldCac
     if (fieldInfo == null) {
       // field does not exist or has no value
       return new Bits.MatchNoBits(reader.maxDoc());
-    } else if (fieldInfo.hasDocValues()) {
+    } else if (fieldInfo.getDocValuesType() != DocValuesType.NONE) {
       return reader.getDocsWithField(field);
-    } else if (!fieldInfo.isIndexed()) {
+    } else if (fieldInfo.getIndexOptions() == IndexOptions.NONE) {
       return new Bits.MatchNoBits(reader.maxDoc());
     }
     BitsEntry bitsEntry = (BitsEntry) caches.get(DocsWithFieldCache.class).get(reader, new CacheKey(field, null), false);
@@ -459,9 +461,9 @@ class FieldCacheImpl implements FieldCac
       final FieldInfo info = reader.getFieldInfos().fieldInfo(field);
       if (info == null) {
         return DocValues.emptyNumeric();
-      } else if (info.hasDocValues()) {
+      } else if (info.getDocValuesType() != DocValuesType.NONE) {
         throw new IllegalStateException("Type mismatch: " + field + " was indexed as " + info.getDocValuesType());
-      } else if (!info.isIndexed()) {
+      } else if (info.getIndexOptions() == IndexOptions.NONE) {
         return DocValues.emptyNumeric();
       }
       return (NumericDocValues) caches.get(Long.TYPE).get(reader, new CacheKey(field, parser), setDocsWithField);
@@ -634,11 +636,11 @@ class FieldCacheImpl implements FieldCac
       final FieldInfo info = reader.getFieldInfos().fieldInfo(field);
       if (info == null) {
         return DocValues.emptySorted();
-      } else if (info.hasDocValues()) {
+      } else if (info.getDocValuesType() != DocValuesType.NONE) {
         // we don't try to build a sorted instance from numeric/binary doc
         // values because dedup can be very costly
         throw new IllegalStateException("Type mismatch: " + field + " was indexed as " + info.getDocValuesType());
-      } else if (!info.isIndexed()) {
+      } else if (info.getIndexOptions() == IndexOptions.NONE) {
         return DocValues.emptySorted();
       }
       SortedDocValuesImpl impl = (SortedDocValuesImpl) caches.get(SortedDocValues.class).get(reader, new CacheKey(field, acceptableOverheadRatio), false);
@@ -783,9 +785,9 @@ class FieldCacheImpl implements FieldCac
     final FieldInfo info = reader.getFieldInfos().fieldInfo(field);
     if (info == null) {
       return DocValues.emptyBinary();
-    } else if (info.hasDocValues()) {
+    } else if (info.getDocValuesType() != DocValuesType.NONE) {
       throw new IllegalStateException("Type mismatch: " + field + " was indexed as " + info.getDocValuesType());
-    } else if (!info.isIndexed()) {
+    } else if (info.getIndexOptions() == IndexOptions.NONE) {
       return DocValues.emptyBinary();
     }
 
@@ -906,9 +908,9 @@ class FieldCacheImpl implements FieldCac
     final FieldInfo info = reader.getFieldInfos().fieldInfo(field);
     if (info == null) {
       return DocValues.emptySortedSet();
-    } else if (info.hasDocValues()) {
+    } else if (info.getDocValuesType() != DocValuesType.NONE) {
       throw new IllegalStateException("Type mismatch: " + field + " was indexed as " + info.getDocValuesType());
-    } else if (!info.isIndexed()) {
+    } else if (info.getIndexOptions() == IndexOptions.NONE) {
       return DocValues.emptySortedSet();
     }
     

Modified: lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java Sat Nov  8 11:32:18 2014
@@ -37,6 +37,7 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.FilterDirectoryReader;
 import org.apache.lucene.index.FilterLeafReader;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
@@ -185,7 +186,7 @@ public class UninvertingReader extends F
     ArrayList<FieldInfo> filteredInfos = new ArrayList<>();
     for (FieldInfo fi : in.getFieldInfos()) {
       DocValuesType type = fi.getDocValuesType();
-      if (fi.isIndexed() && !fi.hasDocValues()) {
+      if (fi.getIndexOptions() != IndexOptions.NONE && fi.getDocValuesType() == DocValuesType.NONE) {
         Type t = mapping.get(fi.name);
         if (t != null) {
           switch(t) {
@@ -291,7 +292,7 @@ public class UninvertingReader extends F
    */
   private Type getType(String field) {
     FieldInfo info = fieldInfos.fieldInfo(field);
-    if (info == null || info.hasDocValues() == false) {
+    if (info == null || info.getDocValuesType() == DocValuesType.NONE) {
       return null;
     }
     return mapping.get(field);

Modified: lucene/dev/branches/lucene6005/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java (original)
+++ lucene/dev/branches/lucene6005/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java Sat Nov  8 11:32:18 2014
@@ -263,11 +263,11 @@ public class CustomScoreQuery extends Qu
         valSrcExpls[i] = valSrcWeights[i].explain(info, doc);
       }
       Explanation customExp = CustomScoreQuery.this.getCustomScoreProvider(info).customExplain(doc,subQueryExpl,valSrcExpls);
-      float sc = getBoost() * customExp.getValue();
+      float sc = queryWeight * customExp.getValue();
       Explanation res = new ComplexExplanation(
         true, sc, CustomScoreQuery.this.toString() + ", product of:");
       res.addDetail(customExp);
-      res.addDetail(new Explanation(getBoost(), "queryBoost")); // actually using the q boost as q weight (== weight value)
+      res.addDetail(new Explanation(queryWeight, "queryWeight"));
       return res;
     }
 

Modified: lucene/dev/branches/lucene6005/lucene/queries/src/java/org/apache/lucene/queries/mlt/MoreLikeThis.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/queries/src/java/org/apache/lucene/queries/mlt/MoreLikeThis.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/queries/src/java/org/apache/lucene/queries/mlt/MoreLikeThis.java (original)
+++ lucene/dev/branches/lucene6005/lucene/queries/src/java/org/apache/lucene/queries/mlt/MoreLikeThis.java Sat Nov  8 11:32:18 2014
@@ -591,7 +591,7 @@ public final class MoreLikeThis {
    * @param filteredDocument Document with field values extracted for selected fields.
    * @return More Like This query for the passed document.
    */
-  public Query like(Map<String, ArrayList<String>> filteredDocument) throws IOException {
+  public Query like(Map<String, Collection<Object>> filteredDocument) throws IOException {
     if (fieldNames == null) {
       // gather list of valid fields from lucene
       Collection<String> fields = MultiFields.getIndexedFields(ir);
@@ -753,16 +753,16 @@ public final class MoreLikeThis {
   }
 
 
-  private PriorityQueue<ScoreTerm> retrieveTerms(Map<String, ArrayList<String>> fields) throws 
+  private PriorityQueue<ScoreTerm> retrieveTerms(Map<String, Collection<Object>> fields) throws 
       IOException {
     HashMap<String,Int> termFreqMap = new HashMap();
     for (String fieldName : fieldNames) {
 
       for (String field : fields.keySet()) {
-        ArrayList<String> fieldValues = fields.get(field);
-        for(String fieldValue:fieldValues) {
+        Collection<Object> fieldValues = fields.get(field);
+        for(Object fieldValue:fieldValues) {
           if (fieldValue != null) {
-            addTermFrequencies(new StringReader(fieldValue), termFreqMap,
+            addTermFrequencies(new StringReader(String.valueOf(fieldValue)), termFreqMap,
                 fieldName);
           }
         }

Modified: lucene/dev/branches/lucene6005/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java (original)
+++ lucene/dev/branches/lucene6005/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java Sat Nov  8 11:32:18 2014
@@ -33,6 +33,9 @@ import org.apache.lucene.search.*;
 import org.apache.lucene.search.BooleanQuery.TooManyClauses;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.QueryBuilder;
+import org.apache.lucene.util.automaton.RegExp;
+
+import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
 
 /** This class is overridden by QueryParser in QueryParser.jj
  * and acts to separate the majority of the Java code from the .jj grammar file. 
@@ -81,6 +84,7 @@ public abstract class QueryParserBase ex
   boolean analyzeRangeTerms = false;
 
   boolean autoGeneratePhraseQueries;
+  int maxDeterminizedStates = DEFAULT_MAX_DETERMINIZED_STATES;
 
   // So the generated QueryParser(CharStream) won't error out
   protected QueryParserBase() {
@@ -398,6 +402,24 @@ public abstract class QueryParserBase ex
     return analyzeRangeTerms;
   }
 
+  /**
+   * @param maxDeterminizedStates the maximum number of states that
+   *   determinizing a regexp query can result in.  If the query results in any
+   *   more states a TooComplexToDeterminizeException is thrown.
+   */
+  public void setMaxDeterminizedStates(int maxDeterminizedStates) {
+    this.maxDeterminizedStates = maxDeterminizedStates;
+  }
+
+  /**
+   * @return the maximum number of states that determinizing a regexp query
+   *   can result in.  If the query results in any more states a
+   *   TooComplexToDeterminizeException is thrown.
+   */
+  public int getMaxDeterminizedStates() {
+    return maxDeterminizedStates;
+  }
+
   protected void addClause(List<BooleanClause> clauses, int conj, int mods, Query q) {
     boolean required, prohibited;
 
@@ -553,7 +575,8 @@ public abstract class QueryParserBase ex
    * @return new RegexpQuery instance
    */
   protected Query newRegexpQuery(Term regexp) {
-    RegexpQuery query = new RegexpQuery(regexp);
+    RegexpQuery query = new RegexpQuery(regexp, RegExp.ALL,
+      maxDeterminizedStates);
     query.setRewriteMethod(multiTermRewriteMethod);
     return query;
   }

Modified: lucene/dev/branches/lucene6005/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/RegexpQueryNodeBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/RegexpQueryNodeBuilder.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/RegexpQueryNodeBuilder.java (original)
+++ lucene/dev/branches/lucene6005/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/RegexpQueryNodeBuilder.java Sat Nov  8 11:32:18 2014
@@ -38,6 +38,7 @@ public class RegexpQueryNodeBuilder impl
   public RegexpQuery build(QueryNode queryNode) throws QueryNodeException {
     RegexpQueryNode regexpNode = (RegexpQueryNode) queryNode;
 
+    // TODO: make the maxStates configurable w/ a reasonable default (QueryParserBase uses 10000)
     RegexpQuery q = new RegexpQuery(new Term(regexpNode.getFieldAsString(),
         regexpNode.textToBytesRef()));
 

Modified: lucene/dev/branches/lucene6005/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java (original)
+++ lucene/dev/branches/lucene6005/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java Sat Nov  8 11:32:18 2014
@@ -24,9 +24,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.IndexReaderContext;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.ReaderUtil;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermContext;
@@ -40,6 +40,8 @@ import org.apache.lucene.util.automaton.
 import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.Transition;
 
+import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
+
 // TODO
 //    - compare perf to PhraseQuery exact and sloppy
 //    - optimize: find terms that are in fact MUST (because all paths
@@ -108,6 +110,16 @@ public class TermAutomatonQuery extends 
 
   /** Call this once you are done adding states/transitions. */
   public void finish() {
+    finish(DEFAULT_MAX_DETERMINIZED_STATES);
+  }
+
+  /**
+   * Call this once you are done adding states/transitions.
+   * @param maxDeterminizedStates Maximum number of states created when
+   *   determinizing the automaton.  Higher numbers allow this operation to
+   *   consume more memory but allow more complex automatons.
+   */
+  public void finish(int maxDeterminizedStates) {
     Automaton automaton = builder.finish();
 
     // System.out.println("before det:\n" + automaton.toDot());
@@ -171,7 +183,8 @@ public class TermAutomatonQuery extends 
       automaton = newAutomaton;
     }
 
-    det = Operations.removeDeadStates(Operations.determinize(automaton));
+    det = Operations.removeDeadStates(Operations.determinize(automaton,
+      maxDeterminizedStates));
   }
 
   @Override

Modified: lucene/dev/branches/lucene6005/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java (original)
+++ lucene/dev/branches/lucene6005/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java Sat Nov  8 11:32:18 2014
@@ -106,7 +106,7 @@ public class BBoxStrategy extends Spatia
 
     FieldType fieldType = new FieldType(DoubleField.TYPE_NOT_STORED);
     fieldType.setNumericPrecisionStep(8);//Solr's default
-    fieldType.setDocValueType(DocValuesType.NUMERIC);
+    fieldType.setDocValuesType(DocValuesType.NUMERIC);
     setFieldType(fieldType);
   }
 

Modified: lucene/dev/branches/lucene6005/lucene/spatial/src/test/org/apache/lucene/spatial/bbox/TestBBoxStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/spatial/src/test/org/apache/lucene/spatial/bbox/TestBBoxStrategy.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/spatial/src/test/org/apache/lucene/spatial/bbox/TestBBoxStrategy.java (original)
+++ lucene/dev/branches/lucene6005/lucene/spatial/src/test/org/apache/lucene/spatial/bbox/TestBBoxStrategy.java Sat Nov  8 11:32:18 2014
@@ -112,7 +112,7 @@ public class TestBBoxStrategy extends Ra
     if (random().nextBoolean()) {
       BBoxStrategy bboxStrategy = (BBoxStrategy) strategy;
       FieldType fieldType = new FieldType(bboxStrategy.getFieldType());
-      fieldType.setDocValueType(DocValuesType.NONE);
+      fieldType.setDocValuesType(DocValuesType.NONE);
       bboxStrategy.setFieldType(fieldType);
     }
     for (SpatialOperation operation : SpatialOperation.values()) {

Modified: lucene/dev/branches/lucene6005/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java (original)
+++ lucene/dev/branches/lucene6005/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java Sat Nov  8 11:32:18 2014
@@ -62,6 +62,8 @@ import org.apache.lucene.util.fst.Util;
 import org.apache.lucene.util.fst.Util.Result;
 import org.apache.lucene.util.fst.Util.TopResults;
 
+import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
+
 /**
  * Suggester that first analyzes the surface form, adds the
  * analyzed form to a weighted FST, and then does the same
@@ -898,7 +900,7 @@ public class AnalyzingSuggester extends 
 
     // TODO: we can optimize this somewhat by determinizing
     // while we convert
-    automaton = Operations.determinize(automaton);
+    automaton = Operations.determinize(automaton, DEFAULT_MAX_DETERMINIZED_STATES);
     return automaton;
   }
 

Modified: lucene/dev/branches/lucene6005/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java (original)
+++ lucene/dev/branches/lucene6005/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java Sat Nov  8 11:32:18 2014
@@ -17,7 +17,6 @@ package org.apache.lucene.search.suggest
  */
 
 import java.io.IOException;
-import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Set;
@@ -30,13 +29,15 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.automaton.Automata;
-import org.apache.lucene.util.automaton.Operations;
-import org.apache.lucene.util.automaton.LevenshteinAutomata;
 import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.LevenshteinAutomata;
+import org.apache.lucene.util.automaton.Operations;
 import org.apache.lucene.util.automaton.UTF32ToUTF8;
 import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.PairOutputs.Pair;
 
+import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
+
 /**
  * Implements a fuzzy {@link AnalyzingSuggester}. The similarity measurement is
  * based on the Damerau-Levenshtein (optimal string alignment) algorithm, though
@@ -205,7 +206,7 @@ public final class FuzzySuggester extend
   protected Automaton convertAutomaton(Automaton a) {
     if (unicodeAware) {
       Automaton utf8automaton = new UTF32ToUTF8().convert(a);
-      utf8automaton = Operations.determinize(utf8automaton);
+      utf8automaton = Operations.determinize(utf8automaton, DEFAULT_MAX_DETERMINIZED_STATES);
       return utf8automaton;
     } else {
       return a;
@@ -253,7 +254,7 @@ public final class FuzzySuggester extend
       Automaton a = Operations.union(Arrays.asList(subs));
       // TODO: we could call toLevenshteinAutomata() before det? 
       // this only happens if you have multiple paths anyway (e.g. synonyms)
-      return Operations.determinize(a);
+      return Operations.determinize(a, DEFAULT_MAX_DETERMINIZED_STATES);
     }
   }
 }

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java Sat Nov  8 11:32:18 2014
@@ -800,11 +800,11 @@ public class AssertingLeafReader extends
     FieldInfo fi = getFieldInfos().fieldInfo(field);
     if (docsWithField != null) {
       assert fi != null;
-      assert fi.hasDocValues();
+      assert fi.getDocValuesType() != DocValuesType.NONE;
       assert maxDoc() == docsWithField.length();
       docsWithField = new AssertingBits(docsWithField);
     } else {
-      assert fi == null || fi.hasDocValues() == false;
+      assert fi == null || fi.getDocValuesType() == DocValuesType.NONE;
     }
     return docsWithField;
   }

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java Sat Nov  8 11:32:18 2014
@@ -49,15 +49,16 @@ public abstract class BaseFieldInfoForma
     Codec codec = getCodec();
     SegmentInfo segmentInfo = newSegmentInfo(dir, "_123");
     FieldInfos.Builder builder = new FieldInfos.Builder();
-    FieldInfo fi = builder.addOrUpdate("field", TextField.TYPE_STORED);
+    FieldInfo fi = builder.getOrAdd("field");
+    fi.setIndexOptions(TextField.TYPE_STORED.indexOptions());
     addAttributes(fi);
     FieldInfos infos = builder.finish();
     codec.fieldInfosFormat().write(dir, segmentInfo, "", infos, IOContext.DEFAULT);
     FieldInfos infos2 = codec.fieldInfosFormat().read(dir, segmentInfo, "", IOContext.DEFAULT);
     assertEquals(1, infos2.size());
     assertNotNull(infos2.fieldInfo("field"));
-    assertTrue(infos2.fieldInfo("field").isIndexed());
-    assertFalse(infos2.fieldInfo("field").hasDocValues());
+    assertTrue(infos2.fieldInfo("field").getIndexOptions() != IndexOptions.NONE);
+    assertFalse(infos2.fieldInfo("field").getDocValuesType() != DocValuesType.NONE);
     assertFalse(infos2.fieldInfo("field").omitsNorms());
     assertFalse(infos2.fieldInfo("field").hasPayloads());
     assertFalse(infos2.fieldInfo("field").hasVectors());
@@ -81,7 +82,15 @@ public abstract class BaseFieldInfoForma
     FieldInfos.Builder builder = new FieldInfos.Builder();
     for (String field : fieldNames) {
       IndexableFieldType fieldType = randomFieldType(random());
-      FieldInfo fi = builder.addOrUpdate(field, fieldType);
+      FieldInfo fi = builder.getOrAdd(field);
+      IndexOptions indexOptions = fieldType.indexOptions();
+      if (indexOptions != IndexOptions.NONE) {
+        fi.setIndexOptions(indexOptions);
+        if (fieldType.omitNorms()) {      
+          fi.setOmitsNorms();
+        }
+      }
+      fi.setDocValuesType(fieldType.docValuesType());
       if (fieldType.indexOptions() != IndexOptions.NONE && fieldType.indexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0) {
         if (random().nextBoolean()) {
           fi.setStorePayloads();
@@ -118,7 +127,7 @@ public abstract class BaseFieldInfoForma
     
     if (r.nextBoolean()) {
       DocValuesType values[] = getDocValuesTypes();
-      type.setDocValueType(values[r.nextInt(values.length)]);
+      type.setDocValuesType(values[r.nextInt(values.length)]);
     }
         
     return type;
@@ -157,11 +166,9 @@ public abstract class BaseFieldInfoForma
     assertEquals(expected.name, actual.name);
     assertEquals(expected.getDocValuesType(), actual.getDocValuesType());
     assertEquals(expected.getIndexOptions(), actual.getIndexOptions());
-    assertEquals(expected.hasDocValues(), actual.hasDocValues());
     assertEquals(expected.hasNorms(), actual.hasNorms());
     assertEquals(expected.hasPayloads(), actual.hasPayloads());
     assertEquals(expected.hasVectors(), actual.hasVectors());
-    assertEquals(expected.isIndexed(), actual.isIndexed());
     assertEquals(expected.omitsNorms(), actual.omitsNorms());
     assertEquals(expected.getDocValuesGen(), actual.getDocValuesGen());
   }

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java Sat Nov  8 11:32:18 2014
@@ -60,8 +60,8 @@ import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.Version;
 import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.AutomatonTestUtil.RandomAcceptedStrings;
 import org.apache.lucene.util.automaton.AutomatonTestUtil;
+import org.apache.lucene.util.automaton.AutomatonTestUtil.RandomAcceptedStrings;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -1247,7 +1247,7 @@ public abstract class BasePostingsFormat
     for(String field : fields.keySet()) {
       while (true) {
         Automaton a = AutomatonTestUtil.randomAutomaton(random());
-        CompiledAutomaton ca = new CompiledAutomaton(a);
+        CompiledAutomaton ca = new CompiledAutomaton(a, null, true, Integer.MAX_VALUE);
         if (ca.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
           // Keep retrying until we get an A that will really "use" the PF's intersect code:
           continue;

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Sat Nov  8 11:32:18 2014
@@ -58,8 +58,8 @@ import java.util.logging.Logger;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document2;
-import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
@@ -70,6 +70,7 @@ import org.apache.lucene.index.BinaryDoc
 import org.apache.lucene.index.CompositeReader;
 import org.apache.lucene.index.ConcurrentMergeScheduler;
 import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldFilterLeafReader;
@@ -77,8 +78,8 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexOptions;
-import org.apache.lucene.index.IndexReader.ReaderClosedListener;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.ReaderClosedListener;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.IndexableField;
@@ -104,8 +105,8 @@ import org.apache.lucene.index.SortedDoc
 import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum.SeekStatus;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.index.TermsEnum.SeekStatus;
 import org.apache.lucene.index.TieredMergePolicy;
 import org.apache.lucene.search.AssertingIndexSearcher;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -115,12 +116,12 @@ import org.apache.lucene.store.BaseDirec
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.store.FlushInfo;
-import org.apache.lucene.store.IOContext.Context;
 import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IOContext.Context;
 import org.apache.lucene.store.LockFactory;
 import org.apache.lucene.store.MergeInfo;
-import org.apache.lucene.store.MockDirectoryWrapper.Throttling;
 import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.store.MockDirectoryWrapper.Throttling;
 import org.apache.lucene.store.NRTCachingDirectory;
 import org.apache.lucene.store.RateLimitedDirectoryWrapper;
 import org.apache.lucene.util.automaton.AutomatonTestUtil;
@@ -137,6 +138,7 @@ import org.junit.Test;
 import org.junit.rules.RuleChain;
 import org.junit.rules.TestRule;
 import org.junit.runner.RunWith;
+
 import com.carrotsearch.randomizedtesting.JUnit4MethodProvider;
 import com.carrotsearch.randomizedtesting.LifecycleScope;
 import com.carrotsearch.randomizedtesting.MixWithSuiteName;
@@ -147,16 +149,16 @@ import com.carrotsearch.randomizedtestin
 import com.carrotsearch.randomizedtesting.annotations.SeedDecorators;
 import com.carrotsearch.randomizedtesting.annotations.TestGroup;
 import com.carrotsearch.randomizedtesting.annotations.TestMethodProviders;
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakAction.Action;
 import com.carrotsearch.randomizedtesting.annotations.ThreadLeakAction;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakAction.Action;
 import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakGroup.Group;
 import com.carrotsearch.randomizedtesting.annotations.ThreadLeakGroup;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakGroup.Group;
 import com.carrotsearch.randomizedtesting.annotations.ThreadLeakLingering;
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope.Scope;
 import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakZombies.Consequence;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope.Scope;
 import com.carrotsearch.randomizedtesting.annotations.ThreadLeakZombies;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakZombies.Consequence;
 import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 import com.carrotsearch.randomizedtesting.rules.NoClassHooksShadowingRule;
@@ -2226,7 +2228,7 @@ public abstract class LuceneTestCase ext
   private static Set<String> getDVFields(IndexReader reader) {
     Set<String> fields = new HashSet<>();
     for(FieldInfo fi : MultiFields.getMergedFieldInfos(reader)) {
-      if (fi.hasDocValues()) {
+      if (fi.getDocValuesType() != DocValuesType.NONE) {
         fields.add(fi.name);
       }
     }

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java Sat Nov  8 11:32:18 2014
@@ -895,7 +895,7 @@ public final class TestUtil {
     for(IndexableField f : doc1.getFields()) {
       final Field field1 = (Field) f;
       final Field field2;
-      final DocValuesType dvType = field1.fieldType().docValueType();
+      final DocValuesType dvType = field1.fieldType().docValuesType();
       final NumericType numType = field1.fieldType().numericType();
       if (dvType != DocValuesType.NONE) {
         switch(dvType) {

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/automaton/AutomatonTestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/automaton/AutomatonTestUtil.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/automaton/AutomatonTestUtil.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/automaton/AutomatonTestUtil.java Sat Nov  8 11:32:18 2014
@@ -40,6 +40,11 @@ import org.apache.lucene.util.UnicodeUti
  * basic unoptimized implementations (*slow) for testing.
  */
 public class AutomatonTestUtil {
+  /**
+   * Default maximum number of states that {@link Operations#determinize} should create.
+   */
+  public static final int DEFAULT_MAX_DETERMINIZED_STATES = 1000000;
+
   /** Returns random string, including full unicode range. */
   public static String randomRegexp(Random r) {
     while (true) {
@@ -257,12 +262,12 @@ public class AutomatonTestUtil {
     // get two random Automata from regexps
     Automaton a1 = new RegExp(AutomatonTestUtil.randomRegexp(random), RegExp.NONE).toAutomaton();
     if (random.nextBoolean()) {
-      a1 = Operations.complement(a1);
+      a1 = Operations.complement(a1, DEFAULT_MAX_DETERMINIZED_STATES);
     }
     
     Automaton a2 = new RegExp(AutomatonTestUtil.randomRegexp(random), RegExp.NONE).toAutomaton();
     if (random.nextBoolean()) {
-      a2 = Operations.complement(a2);
+      a2 = Operations.complement(a2, DEFAULT_MAX_DETERMINIZED_STATES);
     }
 
     // combine them in random ways
@@ -270,7 +275,7 @@ public class AutomatonTestUtil {
       case 0: return Operations.concatenate(a1, a2);
       case 1: return Operations.union(a1, a2);
       case 2: return Operations.intersection(a1, a2);
-      default: return Operations.minus(a1, a2);
+      default: return Operations.minus(a1, a2, DEFAULT_MAX_DETERMINIZED_STATES);
     }
   }
   

Modified: lucene/dev/branches/lucene6005/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/CHANGES.txt?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/CHANGES.txt (original)
+++ lucene/dev/branches/lucene6005/solr/CHANGES.txt Sat Nov  8 11:32:18 2014
@@ -198,6 +198,15 @@ New Features
   have similar content. It works in standalone/cloud mode and shares logic with the
   Lucene MoreLikeThis class (Anshum Gupta).
 
+* SOLR-6670: change BALANCESLICEUNIQUE to BALANCESHARDUNIQUE. Also, the parameter
+  for ADDREPLICAPROP that used to be sliceUnique is now shardUnique. (Erick Erickson)
+
+* SOLR-6351: Stats can now be nested under pivot values by adding a 'stats' local param to 
+  facet.pivot which refers to a 'tag' local param in one or more stats.field params.
+  (hossman, Vitaliy Zhovtyuk, Steve Molloy)
+
+SOLR-6533: Support editing common solrconfig.xml values (Noble Paul)
+
 Bug Fixes
 ----------------------
 
@@ -228,9 +237,6 @@ Bug Fixes
 * SOLR-6457: LBHttpSolrServer: ArrayIndexOutOfBoundsException risk if counter overflows
   (longkey via Noble Paul)
 
-* SOLR-6452: StatsComponent's stat 'missing' will work on fields with docValues=true and
-  indexed=false (Xu Zhang via Tomás Fernández Löbbe)
-
 * SOLR-6499: Log warning about multiple update request handlers
   (Noble Paul, Andreas Hubold, hossman)
 
@@ -242,19 +248,6 @@ Bug Fixes
 * SOLR-6484: SolrCLI's healthcheck action needs to check live nodes as part of reporting
   the status of a replica (Timothy Potter)
 
-* SOLR-6509: Solr start scripts interactive mode doesn't honor -z argument (Timothy Potter)
-
-* SOLR-6511: Fencepost error in LeaderInitiatedRecoveryThread (Timothy Potter)
-
-* SOLR-6530: Commits under network partitions can put any node in down state.
-  (Ramkumar Aiyengar, Alan Woodward, Mark Miller, shalin)
-
-* SOLR-6587: Misleading exception when creating collections in SolrCloud with bad configuration.
-  (Tomás Fernández Löbbe)
-
-* SOLR-6545: Query field list with wild card on dynamic field fails.
-  (Burke Webster, Xu Zhang, shalin)
-
 * SOLR-6540 Fix NPE from strdist() func when doc value source does not exist in a doc (hossman)
 
 * SOLR-6624 Spelling mistakes in the Java source (Hrishikesh Gadre)
@@ -262,25 +255,22 @@ Bug Fixes
 * SOLR-6307: Atomic update remove does not work for int array or date array
   (Anurag Sharma , noble)
 
-* SOLR-6573: QueryElevationComponent now works with localParams in the query (janhoy)
-
-* SOLR-6524: Collections left in recovery state after node restart because recovery sleep time
-  increases exponentially between retries. (Mark Miller, shalin)
-
 * SOLR-6224: Post soft-commit callbacks are called before soft commit actually happens.
   (shalin)
 
-* SOLR-6646: bin/solr start script fails to detect solr on non-default port and then after
-  30s tails wrong log file (janhoy)
-
-* SOLR-6647: Bad error message when missing resource from ZK when parsing Schema (janhoy)
-
 * SOLR-6591: Overseer can use stale cluster state and lose updates for collections
   with stateFormat > 1. (shalin)
 
 * SOLR-6631: DistributedQueue spinning on calling zookeeper getChildren()
   (Jessica Cheng Mallet, Mark Miller, Timothy Potter)
 
+* SOLR-2927: Solr does not unregister all mbeans upon exception in constructor
+  causing memory leaks. (tom liu, Sharath Babu, Cyrille Roy, shalin)
+
+* SOLR-6685: ConcurrentModificationException in Overseer Status API. (shalin)
+
+* SOLR-6706: /update/json/docs throws RuntimeException if a nested structure
+  contains a non-leaf float field (Noble Paul, shalin)
 
 Optimizations
 ----------------------
@@ -344,9 +334,6 @@ Other Changes
 * SOLR-6115: Cleanup enum/string action types in Overseer, OverseerCollectionProcessor and
   CollectionHandler. (Erick Erickson, shalin)
 
-* SOLR-6486: solr start script can have a debug flag option; use -a to set arbitrary options
-  (Noble Paul, Timothy Potter)
-
 * SOLR-6453: Stop throwing an error message from Overseer when node exits (Ramkumar Aiyengar, Noble Paul)
 
 * SOLR-6249: Schema API changes return success before all cores are updated; client application
@@ -354,18 +341,9 @@ Other Changes
   managed schema update to block until all replicas of the same collection have processed the
   update or until the specified timeout is reached (Timothy Potter)
 
-* SOLR-6550: Provide simple mechanism for passing additional metadata / context about a server-side
-   SolrException back to the client-side (Timothy Potter)
-
 * SOLR-6597: SolrIndexConfig parameter in one of the SolrIndexSearcher constructor has been removed.
   It was just passed and never used via that constructor. (Anshum Gupta)
 
-* SOLR-6549: bin/solr script should support a -s option to set the -Dsolr.solr.home property.
-  (Timothy Potter)
-
-* SOLR-6529: Stop command in the start scripts should only stop the instance that it had started.
-  (Varun Thacker, Timothy Potter)
-
 * SOLR-5852: Add CloudSolrServer helper method to connect to a ZK ensemble. (Varun Thacker, Furkan KAMACI,
   Shawn Heisey, Mark Miller, Erick Erickson via shalin)
 
@@ -387,6 +365,68 @@ Other Changes
   relative paths for solrconfig.xml <lib> references with solr.install.dir 
   system property; bin/solr scripts will set it appropriately. (ehatcher)
 
+* SOLR-6698: Solr is not consistent wrt ZkCredentialsProvider / ZkCredentialProvider.
+  References to zkCredentialProvider in System properties or configurations should be
+  changed to zkCredentialsProvider.  (Gregory Chanan)
+
+* SOLR-6715: ZkSolrResourceLoader constructors accept a parameter called 'collection'
+  but it should be 'configName'. (shalin)
+
+==================  4.10.3 ==================
+
+Bug Fixes
+----------------------
+
+* SOLR-6696: bin/solr start script should not enable autoSoftCommit by default (janhoy)
+
+* SOLR-6704: TrieDateField type drops schema properties in branch 4.10 (Tomás Fernández Löbbe)
+
+==================  4.10.2 ==================
+
+Bug Fixes
+----------------------
+
+* SOLR-6509: Solr start scripts interactive mode doesn't honor -z argument (Timothy Potter)
+
+* SOLR-6511: Fencepost error in LeaderInitiatedRecoveryThread (Timothy Potter)
+
+* SOLR-6530: Commits under network partitions can put any node in down state.
+  (Ramkumar Aiyengar, Alan Woodward, Mark Miller, shalin)
+
+* SOLR-6573: QueryElevationComponent now works with localParams in the query (janhoy)
+
+* SOLR-6524: Collections left in recovery state after node restart because recovery sleep time
+  increases exponentially between retries. (Mark Miller, shalin)
+
+* SOLR-6587: Misleading exception when creating collections in SolrCloud with bad configuration.
+  (Tomás Fernández Löbbe)
+
+* SOLR-6452: StatsComponent's stat 'missing' will work on fields with docValues=true and
+  indexed=false (Xu Zhang via Tomás Fernández Löbbe)
+
+* SOLR-6646: bin/solr start script fails to detect solr on non-default port and then after
+  30s tails wrong log file (janhoy)
+
+* SOLR-6647: Bad error message when missing resource from ZK when parsing Schema (janhoy)
+
+* SOLR-6545: Query field list with wild card on dynamic field fails.
+  (Burke Webster, Xu Zhang, shalin)
+
+Other Changes
+----------------------
+
+* SOLR-6550: Provide simple mechanism for passing additional metadata / context about a server-side
+   SolrException back to the client-side (Timothy Potter)
+
+* SOLR-6486: solr start script can have a debug flag option; use -a to set arbitrary options
+  (Noble Paul, Timothy Potter)
+
+* SOLR-6549: bin/solr script should support a -s option to set the -Dsolr.solr.home property.
+  (Timothy Potter)
+
+* SOLR-6529: Stop command in the start scripts should only stop the instance that it had started.
+  (Varun Thacker, Timothy Potter)
+
 ==================  4.10.1 ==================
 
 Bug Fixes

Modified: lucene/dev/branches/lucene6005/solr/bin/solr
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/bin/solr?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/bin/solr (original)
+++ lucene/dev/branches/lucene6005/solr/bin/solr Sat Nov  8 11:32:18 2014
@@ -1023,7 +1023,7 @@ $SOLR_HOST_ARG -Djetty.port=$SOLR_PORT \
 -Dsolr.solr.home=$SOLR_HOME \
 -Dsolr.install.dir=$SOLR_TIP \
 -Duser.timezone=$SOLR_TIMEZONE \
--Djava.net.preferIPv4Stack=true -Dsolr.autoSoftCommit.maxTime=3000"
+-Djava.net.preferIPv4Stack=true"
   
   if [ "$SOLR_MODE" == "solrcloud" ]; then
     IN_CLOUD_MODE=" in SolrCloud mode"

Modified: lucene/dev/branches/lucene6005/solr/bin/solr.cmd
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/bin/solr.cmd?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/bin/solr.cmd (original)
+++ lucene/dev/branches/lucene6005/solr/bin/solr.cmd Sat Nov  8 11:32:18 2014
@@ -619,7 +619,7 @@ IF "%verbose%"=="1" (
     @echo     SOLR_TIMEZONE   = %SOLR_TIMEZONE%
 )
 
-set START_OPTS=-Duser.timezone=%SOLR_TIMEZONE% -Djava.net.preferIPv4Stack=true -Dsolr.autoSoftCommit.maxTime=3000
+set START_OPTS=-Duser.timezone=%SOLR_TIMEZONE% -Djava.net.preferIPv4Stack=true
 set START_OPTS=%START_OPTS% %GC_TUNE% %GC_LOG_OPTS%
 IF NOT "!CLOUD_MODE_OPTS!"=="" set START_OPTS=%START_OPTS% !CLOUD_MODE_OPTS!
 IF NOT "%REMOTE_JMX_OPTS%"=="" set START_OPTS=%START_OPTS% %REMOTE_JMX_OPTS%

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java Sat Nov  8 11:32:18 2014
@@ -88,7 +88,7 @@ public class CloudUtil {
 
   /**
    * Returns a displayable unified path to the given resource. For non-solrCloud that will be the
-   * same as getConfigDir, but for Cloud it will be getCollectionZkPath ending in a /
+   * same as getConfigDir, but for Cloud it will be getConfigSetZkPath ending in a /
    * <p/>
    * <b>Note:</b> Do not use this to generate a valid file path, but for debug printing etc
    * @param loader Resource loader instance
@@ -96,7 +96,7 @@ public class CloudUtil {
    */
   public static String unifiedResourcePath(SolrResourceLoader loader) {
     return (loader instanceof ZkSolrResourceLoader) ?
-            ((ZkSolrResourceLoader) loader).getCollectionZkPath() + "/" :
+            ((ZkSolrResourceLoader) loader).getConfigSetZkPath() + "/" :
             loader.getConfigDir();
   }
 }

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/cloud/Overseer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/cloud/Overseer.java?rev=1637544&r1=1637543&r2=1637544&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/cloud/Overseer.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/cloud/Overseer.java Sat Nov  8 11:32:18 2014
@@ -18,11 +18,11 @@ package org.apache.solr.cloud;
  */
 
 import static java.util.Collections.singletonMap;
-import static org.apache.solr.cloud.OverseerCollectionProcessor.SLICE_UNIQUE;
+import static org.apache.solr.cloud.OverseerCollectionProcessor.SHARD_UNIQUE;
 import static org.apache.solr.common.cloud.ZkNodeProps.makeMap;
 import static org.apache.solr.cloud.OverseerCollectionProcessor.ONLY_ACTIVE_NODES;
 import static org.apache.solr.cloud.OverseerCollectionProcessor.COLL_PROP_PREFIX;
-import static org.apache.solr.common.params.CollectionParams.CollectionAction.BALANCESLICEUNIQUE;
+import static org.apache.solr.common.params.CollectionParams.CollectionAction.BALANCESHARDUNIQUE;
 
 import java.io.Closeable;
 import java.io.IOException;
@@ -40,6 +40,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Random;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -382,6 +383,20 @@ public class Overseer implements Closeab
                 zkClient.setData(e.getKey(), data, true);
               } else {
                 log.info("going to create_collection {}", e.getKey());
+                String parentPath = e.getKey().substring(0, e.getKey().lastIndexOf('/'));
+                if (!zkClient.exists(parentPath, true)) {
+                  // if the /collections/collection_name path doesn't exist then it means that
+                  // 1) the user invoked a DELETE collection API and the OverseerCollectionProcessor has deleted
+                  // this zk path.
+                  // 2) these are most likely old "state" messages which are only being processed now because
+                  // if they were new "state" messages then in legacy mode, a new collection would have been
+                  // created with stateFormat = 1 (which is the default state format)
+                  // 3) these can't be new "state" messages created for a new collection because
+                  // otherwise the OverseerCollectionProcessor would have already created this path
+                  // as part of the create collection API call -- which is the only way in which a collection
+                  // with stateFormat > 1 can possibly be created
+                  continue;
+                }
                 zkClient.create(e.getKey(), data, CreateMode.PERSISTENT, true);
               }
             }
@@ -475,7 +490,7 @@ public class Overseer implements Closeab
           case DELETEREPLICAPROP:
             clusterState = deleteReplicaProp(clusterState, message);
             break;
-          case BALANCESLICEUNIQUE:
+          case BALANCESHARDUNIQUE:
             ExclusiveSliceProperty dProp = new ExclusiveSliceProperty(this, clusterState, message);
             if (dProp.balanceProperty()) {
               String collName = message.getStr(ZkStateReader.COLLECTION_PROP);
@@ -571,19 +586,19 @@ public class Overseer implements Closeab
       }
       property = property.toLowerCase(Locale.ROOT);
       String propVal = message.getStr(ZkStateReader.PROPERTY_VALUE_PROP);
-      String sliceUnique = message.getStr(OverseerCollectionProcessor.SLICE_UNIQUE);
+      String shardUnique = message.getStr(OverseerCollectionProcessor.SHARD_UNIQUE);
 
       boolean isUnique = false;
 
       if (sliceUniqueBooleanProperties.contains(property)) {
-        if (StringUtils.isNotBlank(sliceUnique) && Boolean.parseBoolean(sliceUnique) == false) {
+        if (StringUtils.isNotBlank(shardUnique) && Boolean.parseBoolean(shardUnique) == false) {
           throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Overseer SETREPLICAPROPERTY for " +
-              property + " cannot have " + OverseerCollectionProcessor.SLICE_UNIQUE + " set to anything other than" +
+              property + " cannot have " + OverseerCollectionProcessor.SHARD_UNIQUE + " set to anything other than" +
               "'true'. No action taken");
         }
         isUnique = true;
       } else {
-        isUnique = Boolean.parseBoolean(sliceUnique);
+        isUnique = Boolean.parseBoolean(shardUnique);
       }
 
       Replica replica = clusterState.getReplica(collectionName, replicaName);
@@ -1456,12 +1471,12 @@ public class Overseer implements Closeab
                 ZkStateReader.PROPERTY_PROP + "' parameters. No action taken ");
       }
 
-      Boolean sliceUnique = Boolean.parseBoolean(message.getStr(SLICE_UNIQUE));
-      if (sliceUnique == false &&
+      Boolean shardUnique = Boolean.parseBoolean(message.getStr(SHARD_UNIQUE));
+      if (shardUnique == false &&
           Overseer.sliceUniqueBooleanProperties.contains(this.property) == false) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Balancing properties amongst replicas in a slice requires that"
-            + " the property be a pre-defined property (e.g. 'preferredLeader') or that 'sliceUnique' be set to 'true' " +
-            " Property: " + this.property + " sliceUnique: " + Boolean.toString(sliceUnique));
+            + " the property be a pre-defined property (e.g. 'preferredLeader') or that 'shardUnique' be set to 'true' " +
+            " Property: " + this.property + " shardUnique: " + Boolean.toString(shardUnique));
       }
 
       collection = clusterState.getCollection(collectionName);
@@ -1508,7 +1523,7 @@ public class Overseer implements Closeab
           if (StringUtils.isNotBlank(replica.getStr(property))) {
             if (sliceHasProp) {
               throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                  "'" + BALANCESLICEUNIQUE + "' should only be called for properties that have at most one member " +
+                  "'" + BALANCESHARDUNIQUE + "' should only be called for properties that have at most one member " +
                       "in any slice with the property set. No action taken.");
             }
             if (nodesHostingProp.containsKey(nodeName) == false) {
@@ -1948,7 +1963,7 @@ public class Overseer implements Closeab
   public static class Stats {
     static final int MAX_STORED_FAILURES = 10;
 
-    final Map<String, Stat> stats = Collections.synchronizedMap(new HashMap<String, Stat>());
+    final Map<String, Stat> stats = new ConcurrentHashMap<>();
 
     public Map<String, Stat> getStats() {
       return stats;
@@ -1966,19 +1981,16 @@ public class Overseer implements Closeab
 
     public void success(String operation) {
       String op = operation.toLowerCase(Locale.ROOT);
-      synchronized (stats) {
-        Stat stat = stats.get(op);
-        if (stat == null) {
-          stat = new Stat();
-          stats.put(op, stat);
-        }
-        stat.success.incrementAndGet();
+      Stat stat = stats.get(op);
+      if (stat == null) {
+        stat = new Stat();
+        stats.put(op, stat);
       }
+      stat.success.incrementAndGet();
     }
 
     public void error(String operation) {
       String op = operation.toLowerCase(Locale.ROOT);
-      synchronized (stats) {
       Stat stat = stats.get(op);
       if (stat == null) {
         stat = new Stat();
@@ -1986,26 +1998,20 @@ public class Overseer implements Closeab
       }
       stat.errors.incrementAndGet();
     }
-    }
 
     public TimerContext time(String operation) {
       String op = operation.toLowerCase(Locale.ROOT);
-      Stat stat;
-      synchronized (stats) {
-        stat = stats.get(op);
+      Stat stat = stats.get(op);
       if (stat == null) {
         stat = new Stat();
         stats.put(op, stat);
       }
-      }
       return stat.requestTime.time();
     }
 
     public void storeFailureDetails(String operation, ZkNodeProps request, SolrResponse resp) {
       String op = operation.toLowerCase(Locale.ROOT);
-      Stat stat ;
-      synchronized (stats) {
-        stat = stats.get(op);
+      Stat stat = stats.get(op);
       if (stat == null) {
         stat = new Stat();
         stats.put(op, stat);
@@ -2018,7 +2024,6 @@ public class Overseer implements Closeab
         failedOps.addLast(new FailedOp(request, resp));
       }
     }
-    }
 
     public List<FailedOp> getFailureDetails(String operation) {
       Stat stat = stats.get(operation.toLowerCase(Locale.ROOT));