You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rj...@apache.org on 2015/03/31 07:22:50 UTC

svn commit: r1670257 [16/39] - in /lucene/dev/branches/lucene6271: ./ dev-tools/ dev-tools/idea/.idea/libraries/ dev-tools/scripts/ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneou...

Modified: lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/index/MismatchedDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/index/MismatchedDirectoryReader.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/index/MismatchedDirectoryReader.java (original)
+++ lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/index/MismatchedDirectoryReader.java Tue Mar 31 05:22:40 2015
@@ -17,6 +17,7 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.util.Random;
 
 /**
@@ -38,12 +39,12 @@ public class MismatchedDirectoryReader e
     }
   }
 
-  public MismatchedDirectoryReader(DirectoryReader in, Random random) {
+  public MismatchedDirectoryReader(DirectoryReader in, Random random) throws IOException {
     super(in, new MismatchedSubReaderWrapper(random));
   }
 
   @Override
-  protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
+  protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
     return new AssertingDirectoryReader(in);
   }
 }

Modified: lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/index/MockRandomMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/index/MockRandomMergePolicy.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/index/MockRandomMergePolicy.java (original)
+++ lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/index/MockRandomMergePolicy.java Tue Mar 31 05:22:40 2015
@@ -97,7 +97,7 @@ public class MockRandomMergePolicy exten
 
     //System.out.println("MRMP: findMerges sis=" + segmentInfos + " eligible=" + eligibleSegments);
     MergeSpecification mergeSpec = null;
-    if (eligibleSegments.size() > 1 || (eligibleSegments.size() == 1 && eligibleSegments.get(0).hasDeletions())) {
+    if (eligibleSegments.size() > 1 || (eligibleSegments.size() == 1 && isMerged(segmentInfos, eligibleSegments.get(0), writer) == false)) {
       mergeSpec = new MergeSpecification();
       // Already shuffled having come out of a set but
       // shuffle again for good measure:

Modified: lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java Tue Mar 31 05:22:40 2015
@@ -29,6 +29,7 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.NullInfoStream;
@@ -49,6 +50,7 @@ public class RandomIndexWriter implement
   private double flushAtFactor = 1.0;
   private boolean getReaderCalled;
   private final Codec codec; // sugar
+  private final Analyzer analyzer; // only if WE created it (then we close it)
 
   /** Returns an indexwriter that randomly mixes up thread scheduling (by yielding at test points) */
   public static IndexWriter mockIndexWriter(Directory dir, IndexWriterConfig conf, Random r) throws IOException {
@@ -73,7 +75,7 @@ public class RandomIndexWriter implement
 
   /** create a RandomIndexWriter with a random config: Uses MockAnalyzer */
   public RandomIndexWriter(Random r, Directory dir) throws IOException {
-    this(r, dir, LuceneTestCase.newIndexWriterConfig(r, new MockAnalyzer(r)));
+    this(r, dir, LuceneTestCase.newIndexWriterConfig(r, new MockAnalyzer(r)), true);
   }
   
   /** create a RandomIndexWriter with a random config */
@@ -83,10 +85,19 @@ public class RandomIndexWriter implement
   
   /** create a RandomIndexWriter with the provided config */
   public RandomIndexWriter(Random r, Directory dir, IndexWriterConfig c) throws IOException {
+    this(r, dir, c, false);
+  }
+      
+  private RandomIndexWriter(Random r, Directory dir, IndexWriterConfig c, boolean closeAnalyzer) throws IOException {
     // TODO: this should be solved in a different way; Random should not be shared (!).
     this.r = new Random(r.nextLong());
     w = mockIndexWriter(dir, c, r);
     flushAt = TestUtil.nextInt(r, 10, 1000);
+    if (closeAnalyzer) {
+      analyzer = w.getAnalyzer();
+    } else {
+      analyzer = null;
+    }
     codec = w.getConfig().getCodec();
     if (LuceneTestCase.VERBOSE) {
       System.out.println("RIW dir=" + dir + " config=" + w.getConfig());
@@ -361,7 +372,7 @@ public class RandomIndexWriter implement
         w.commit();
       }
     }
-    w.close();
+    IOUtils.close(w, analyzer);
   }
 
   /**

Modified: lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java (original)
+++ lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java Tue Mar 31 05:22:40 2015
@@ -380,7 +380,7 @@ public abstract class ThreadedIndexingAn
                         //if (VERBOSE) {
                         //System.out.println(Thread.currentThread().getName() + " now search body:" + term.utf8ToString());
                         //}
-                        totHits.addAndGet(runQuery(s, new TermQuery(new Term("body", term))));
+                        totHits.addAndGet(runQuery(s, new TermQuery(new Term("body", BytesRef.deepCopyOf(term)))));
                       }
                     }
                     //if (VERBOSE) {

Modified: lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorer.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorer.java (original)
+++ lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorer.java Tue Mar 31 05:22:40 2015
@@ -27,6 +27,8 @@ import java.util.WeakHashMap;
 
 /** Wraps a Scorer with additional checks */
 public class AssertingScorer extends Scorer {
+  
+  // TODO: add asserts for two-phase intersection
 
   static enum IteratorState { START, ITERATING, FINISHED };
 
@@ -91,8 +93,7 @@ public class AssertingScorer extends Sco
   public float score() throws IOException {
     assert iterating();
     final float score = in.score();
-    assert !Float.isNaN(score);
-    assert !Float.isNaN(score);
+    assert !Float.isNaN(score) : "NaN score for in="+in;
     return score;
   }
 

Modified: lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/search/BaseExplanationTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/search/BaseExplanationTestCase.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/search/BaseExplanationTestCase.java (original)
+++ lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/search/BaseExplanationTestCase.java Tue Mar 31 05:22:40 2015
@@ -17,6 +17,7 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -51,6 +52,7 @@ public abstract class BaseExplanationTes
   protected static IndexSearcher searcher;
   protected static IndexReader reader;
   protected static Directory directory;
+  protected static Analyzer analyzer;
   
   public static final String KEY = "KEY";
   // boost on this field is the same as the iterator for the doc
@@ -65,12 +67,15 @@ public abstract class BaseExplanationTes
     reader = null;
     directory.close();
     directory = null;
+    analyzer.close();
+    analyzer = null;
   }
   
   @BeforeClass
   public static void beforeClassTestExplanations() throws Exception {
     directory = newDirectory();
-    RandomIndexWriter writer= new RandomIndexWriter(random(), directory, newIndexWriterConfig(new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
+    analyzer = new MockAnalyzer(random());
+    RandomIndexWriter writer= new RandomIndexWriter(random(), directory, newIndexWriterConfig(analyzer).setMergePolicy(newLogMergePolicy()));
     for (int i = 0; i < docFields.length; i++) {
       Document doc = new Document();
       doc.add(newStringField(KEY, ""+i, Field.Store.NO));
@@ -112,7 +117,7 @@ public abstract class BaseExplanationTes
   /** 
    * Convenience subclass of FieldCacheTermsFilter
    */
-  public static class ItemizedFilter extends DocValuesTermsFilter {
+  public static class ItemizedQuery extends DocValuesTermsQuery {
     private static String[] int2str(int [] terms) {
       String [] out = new String[terms.length];
       for (int i = 0; i < terms.length; i++) {
@@ -120,7 +125,7 @@ public abstract class BaseExplanationTes
       }
       return out;
     }
-    public ItemizedFilter(int [] keys) {
+    public ItemizedQuery(int [] keys) {
       super(KEY, int2str(keys));
     }
   }

Modified: lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java (original)
+++ lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java Tue Mar 31 05:22:40 2015
@@ -22,6 +22,7 @@ import java.util.List;
 import java.util.Random;
 
 import junit.framework.Assert;
+
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.AllDeletedFilterReader;
@@ -133,7 +134,7 @@ public class QueryUtils {
   public static class FCInvisibleMultiReader extends MultiReader {
     private final Object cacheKey = new Object();
   
-    public FCInvisibleMultiReader(IndexReader... readers) {
+    public FCInvisibleMultiReader(IndexReader... readers) throws IOException {
       super(readers);
     }
     
@@ -213,7 +214,7 @@ public class QueryUtils {
     return new AllDeletedFilterReader(LuceneTestCase.getOnlySegmentReader(reader));
   }
 
-  /** alternate scorer skipTo(),skipTo(),next(),next(),skipTo(),skipTo(), etc
+  /** alternate scorer advance(),advance(),next(),next(),advance(),advance(), etc
    * and ensure a hitcollector receives same docs and scores
    */
   public static void checkSkipTo(final Query q, final IndexSearcher s) throws IOException {
@@ -318,7 +319,7 @@ public class QueryUtils {
               Scorer scorer = w.scorer(ctx, ctx.reader().getLiveDocs());
               if (scorer != null) {
                 boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;
-                Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
+                Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but advance("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
               }
               leafPtr++;
             }
@@ -340,7 +341,7 @@ public class QueryUtils {
           Scorer scorer = w.scorer(ctx, ctx.reader().getLiveDocs());
           if (scorer != null) {
             boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;
-            Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
+            Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but advance("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
           }
         }
       }
@@ -369,11 +370,11 @@ public class QueryUtils {
           for (int i=lastDoc[0]+1; i<=doc; i++) {
             Weight w = s.createNormalizedWeight(q, true);
             Scorer scorer = w.scorer(context.get(leafPtr), liveDocs);
-            Assert.assertTrue("query collected "+doc+" but skipTo("+i+") says no more docs!",scorer.advance(i) != DocIdSetIterator.NO_MORE_DOCS);
-            Assert.assertEquals("query collected "+doc+" but skipTo("+i+") got to "+scorer.docID(),doc,scorer.docID());
-            float skipToScore = scorer.score();
-            Assert.assertEquals("unstable skipTo("+i+") score!",skipToScore,scorer.score(),maxDiff); 
-            Assert.assertEquals("query assigned doc "+doc+" a score of <"+score+"> but skipTo("+i+") has <"+skipToScore+">!",score,skipToScore,maxDiff);
+            Assert.assertTrue("query collected "+doc+" but advance("+i+") says no more docs!",scorer.advance(i) != DocIdSetIterator.NO_MORE_DOCS);
+            Assert.assertEquals("query collected "+doc+" but advance("+i+") got to "+scorer.docID(),doc,scorer.docID());
+            float advanceScore = scorer.score();
+            Assert.assertEquals("unstable advance("+i+") score!",advanceScore,scorer.score(),maxDiff); 
+            Assert.assertEquals("query assigned doc "+doc+" a score of <"+score+"> but advance("+i+") has <"+advanceScore+">!",score,advanceScore,maxDiff);
             
             // Hurry things along if they are going slow (eg
             // if you got SimpleText codec this will kick in):
@@ -404,7 +405,7 @@ public class QueryUtils {
           Scorer scorer = w.scorer((LeafReaderContext)indexSearcher.getTopReaderContext(), previousReader.getLiveDocs());
           if (scorer != null) {
             boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;
-            Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
+            Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but advance("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
           }
           leafPtr++;
         }
@@ -425,7 +426,7 @@ public class QueryUtils {
       Scorer scorer = w.scorer((LeafReaderContext)indexSearcher.getTopReaderContext(), previousReader.getLiveDocs());
       if (scorer != null) {
         boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;
-        Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
+        Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but advance("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
       }
     }
   }

Modified: lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/search/RandomApproximationQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/search/RandomApproximationQuery.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/search/RandomApproximationQuery.java (original)
+++ lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/search/RandomApproximationQuery.java Tue Mar 31 05:22:40 2015
@@ -127,7 +127,7 @@ public class RandomApproximationQuery ex
     }
 
     @Override
-    public TwoPhaseDocIdSetIterator asTwoPhaseIterator() {
+    public TwoPhaseIterator asTwoPhaseIterator() {
       return twoPhaseView;
     }
 
@@ -163,29 +163,23 @@ public class RandomApproximationQuery ex
 
   }
 
-  private static class RandomTwoPhaseView extends TwoPhaseDocIdSetIterator {
+  private static class RandomTwoPhaseView extends TwoPhaseIterator {
 
     private final DocIdSetIterator disi;
-    private final RandomApproximation approximation;
     private int lastDoc = -1;
 
     RandomTwoPhaseView(Random random, DocIdSetIterator disi) {
+      super(new RandomApproximation(random, disi));
       this.disi = disi;
-      this.approximation = new RandomApproximation(random, disi);
-    }
-
-    @Override
-    public DocIdSetIterator approximation() {
-      return approximation;
     }
 
     @Override
     public boolean matches() throws IOException {
       if (approximation.docID() == -1 || approximation.docID() == DocIdSetIterator.NO_MORE_DOCS) {
-        throw new AssertionError("matches() should not be called on doc ID " + approximation.doc);
+        throw new AssertionError("matches() should not be called on doc ID " + approximation.docID());
       }
       if (lastDoc == approximation.docID()) {
-        throw new AssertionError("matches() has been called twice on doc ID " + approximation.doc);
+        throw new AssertionError("matches() has been called twice on doc ID " + approximation.docID());
       }
       lastDoc = approximation.docID();
       return approximation.docID() == disi.docID();

Modified: lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java (original)
+++ lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java Tue Mar 31 05:22:40 2015
@@ -17,6 +17,7 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.util.BitSet;
 import java.util.Random;
 
@@ -28,10 +29,13 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BitDocIdSet;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.automaton.Automata;
@@ -140,7 +144,86 @@ public abstract class SearchEquivalenceT
    * Returns a random filter over the document set
    */
   protected Filter randomFilter() {
-    return new QueryWrapperFilter(TermRangeQuery.newStringRange("field", "a", "" + randomChar(), true, true));
+    final Query query;
+    if (random().nextBoolean()) {
+      query = TermRangeQuery.newStringRange("field", "a", "" + randomChar(), true, true);
+    } else {
+      // use a query with a two-phase approximation
+      PhraseQuery phrase = new PhraseQuery();
+      phrase.add(new Term("field", "" + randomChar()));
+      phrase.add(new Term("field", "" + randomChar()));
+      phrase.setSlop(100);
+      query = phrase;
+    }
+    
+    // now wrap the query as a filter. QWF has its own codepath
+    if (random().nextBoolean()) {
+      return new QueryWrapperFilter(query);
+    } else {
+      return new SlowWrapperFilter(query, random().nextBoolean());
+    }
+  }
+  
+  static class SlowWrapperFilter extends Filter {
+    final Query query;
+    final boolean useBits;
+    
+    SlowWrapperFilter(Query query, boolean useBits) {
+      this.query = query;
+      this.useBits = useBits;
+    }
+    
+    @Override
+    public Query rewrite(IndexReader reader) throws IOException {
+      Query q = query.rewrite(reader);
+      if (q != query) {
+        return new SlowWrapperFilter(q, useBits);
+      } else {
+        return this;
+      }
+    }
+
+    @Override
+    public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
+      // get a private context that is used to rewrite, createWeight and score eventually
+      final LeafReaderContext privateContext = context.reader().getContext();
+      final Weight weight = new IndexSearcher(privateContext).createNormalizedWeight(query, false);
+      return new DocIdSet() {
+        @Override
+        public DocIdSetIterator iterator() throws IOException {
+          return weight.scorer(privateContext, acceptDocs);
+        }
+
+        @Override
+        public long ramBytesUsed() {
+          return 0L;
+        }
+
+        @Override
+        public Bits bits() throws IOException {
+          if (useBits) {
+            BitDocIdSet.Builder builder = new BitDocIdSet.Builder(context.reader().maxDoc());
+            DocIdSetIterator disi = iterator();
+            if (disi != null) {
+              builder.or(disi);
+            }
+            BitDocIdSet bitset = builder.build();
+            if (bitset == null) {
+              return new Bits.MatchNoBits(context.reader().maxDoc());
+            } else {
+              return bitset.bits();
+            }
+          } else {
+            return null;
+          }
+        }
+      };
+    }
+
+    @Override
+    public String toString(String field) {
+      return "SlowQWF(" + query + ")";
+    }
   }
 
   /**

Modified: lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java (original)
+++ lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java Tue Mar 31 05:22:40 2015
@@ -27,6 +27,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Set;
 import java.util.zip.CRC32;
 
 import org.apache.lucene.index.DirectoryReader;
@@ -351,6 +352,93 @@ public abstract class BaseDirectoryTestC
     input.close();
     dir.close();
   }
+  
+  public void testSetOfStrings() throws Exception {
+    Directory dir = getDirectory(createTempDir("testSetOfStrings"));
+    
+    IndexOutput output = dir.createOutput("stringset", newIOContext(random()));
+    output.writeSetOfStrings(asSet("test1", "test2"));
+    output.writeSetOfStrings(Collections.emptySet());
+    output.writeSetOfStrings(asSet("test3"));
+    output.close();
+    
+    IndexInput input = dir.openInput("stringset", newIOContext(random()));
+    Set<String> set = input.readSetOfStrings();
+    assertEquals(asSet("test1", "test2"), set);
+    try {
+      set.add("bogus");
+      fail("set should be immutable");
+    } catch (UnsupportedOperationException expected) {
+      // ok
+    }
+    
+    set = input.readSetOfStrings();
+    assertEquals(Collections.emptySet(), set);
+    try {
+      set.add("bogus");
+      fail("set should be immutable");
+    } catch (UnsupportedOperationException expected) {
+      // ok
+    }
+    
+    set = input.readSetOfStrings();
+    assertEquals(Collections.singleton("test3"), set);
+    try {
+      set.add("bogus");
+      fail("set should be immutable");
+    } catch (UnsupportedOperationException expected) {
+      // ok
+    }
+    
+    assertEquals(input.length(), input.getFilePointer());
+    input.close();
+    dir.close();
+  }
+  
+  public void testMapOfStrings() throws Exception {
+    Map<String,String> m = new HashMap<>();
+    m.put("test1", "value1");
+    m.put("test2", "value2");
+    
+    Directory dir = getDirectory(createTempDir("testMapOfStrings"));
+    IndexOutput output = dir.createOutput("stringmap", newIOContext(random()));
+    output.writeMapOfStrings(m);
+    output.writeMapOfStrings(Collections.emptyMap());
+    output.writeMapOfStrings(Collections.singletonMap("key", "value"));
+    output.close();
+    
+    IndexInput input = dir.openInput("stringmap", newIOContext(random()));
+    Map<String,String> map = input.readMapOfStrings();
+    assertEquals(m, map);
+    try {
+      map.put("bogus1", "bogus2");
+      fail("map should be immutable");
+    } catch (UnsupportedOperationException expected) {
+      // ok
+    }
+    
+    map = input.readMapOfStrings();
+    assertEquals(Collections.emptyMap(), map);
+    try {
+      map.put("bogus1", "bogus2");
+      fail("map should be immutable");
+    } catch (UnsupportedOperationException expected) {
+      // ok
+    }
+    
+    map = input.readMapOfStrings();
+    assertEquals(Collections.singletonMap("key", "value"), map);
+    try {
+      map.put("bogus1", "bogus2");
+      fail("map should be immutable");
+    } catch (UnsupportedOperationException expected) {
+      // ok
+    }
+    
+    assertEquals(input.length(), input.getFilePointer());
+    input.close();
+    dir.close();
+  }
   
   // TODO: fold in some of the testing of o.a.l.index.TestIndexInput in here!
   public void testChecksum() throws Exception {

Modified: lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java (original)
+++ lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java Tue Mar 31 05:22:40 2015
@@ -28,13 +28,13 @@ import org.apache.lucene.util.TestUtil;
 // do NOT make any methods in this class synchronized, volatile
 // do NOT import anything from the concurrency package.
 // no randoms, no nothing.
-public class BaseDirectoryWrapper extends FilterDirectory {
+public abstract class BaseDirectoryWrapper extends FilterDirectory {
   
   private boolean checkIndexOnClose = true;
   private boolean crossCheckTermVectorsOnClose = true;
   protected volatile boolean isOpen = true;
 
-  public BaseDirectoryWrapper(Directory delegate) {
+  protected BaseDirectoryWrapper(Directory delegate) {
     super(delegate);
   }
 
@@ -72,10 +72,4 @@ public class BaseDirectoryWrapper extend
   public boolean getCrossCheckTermVectorsOnClose() {
     return crossCheckTermVectorsOnClose;
   }
-
-  // why does this class override this method?
-  @Override
-  public void copyFrom(Directory from, String src, String dest, IOContext context) throws IOException {
-    in.copyFrom(from, src, dest, context);
-  }
 }

Modified: lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java Tue Mar 31 05:22:40 2015
@@ -209,23 +209,6 @@ public class MockDirectoryWrapper extend
     useSlowOpenClosers = v;
   }
 
-  /**
-   * Returns true if {@link #in} must sync its files.
-   * Currently, only {@link NRTCachingDirectory} requires sync'ing its files
-   * because otherwise they are cached in an internal {@link RAMDirectory}. If
-   * other directories require that too, they should be added to this method.
-   */
-  private boolean mustSync() {
-    Directory delegate = in;
-    while (delegate instanceof FilterDirectory) {
-      if (delegate instanceof NRTCachingDirectory) {
-        return true;
-      }
-      delegate = ((FilterDirectory) delegate).getDelegate();
-    }
-    return delegate instanceof NRTCachingDirectory;
-  }
-  
   @Override
   public synchronized void sync(Collection<String> names) throws IOException {
     maybeYield();
@@ -233,16 +216,13 @@ public class MockDirectoryWrapper extend
     if (crashed) {
       throw new IOException("cannot sync after crash");
     }
-    // don't wear out our hardware so much in tests.
-    if (LuceneTestCase.rarely(randomState) || mustSync()) {
-      for (String name : names) {
-        // randomly fail with IOE on any file
-        maybeThrowIOException(name);
-        in.sync(Collections.singleton(name));
-        unSyncedFiles.remove(name);
-      }
-    } else {
-      unSyncedFiles.removeAll(names);
+    // always pass thru fsync, directories rely on this.
+    // 90% of time, we use DisableFsyncFS which omits the real calls.
+    for (String name : names) {
+      // randomly fail with IOE on any file
+      maybeThrowIOException(name);
+      in.sync(Collections.singleton(name));
+      unSyncedFiles.remove(name);
     }
   }
 
@@ -1051,20 +1031,38 @@ public class MockDirectoryWrapper extend
     public boolean isLocked() throws IOException {
       return delegateLock.isLocked();
     }
+  }  
+  
+  /** Use this when throwing fake {@code IOException},
+   *  e.g. from {@link MockDirectoryWrapper.Failure}. */
+  public static class FakeIOException extends IOException {
   }
 
-  // TODO: why does this class override this method?
-  // we should use the default implementation so all of our checks work?
   @Override
-  public synchronized void copyFrom(Directory from, String src, String dest, IOContext context) throws IOException {
-    maybeYield();
-    // randomize the IOContext here?
-    in.copyFrom(from, src, dest, context);
+  public String toString() {
+    if (maxSize != 0) {
+      return "MockDirectoryWrapper(" + in + ", current=" + maxUsedSize + ",max=" + maxSize + ")";
+    } else {
+      return super.toString();
+    }
   }
+
+
+  // don't override optional methods like copyFrom: we need the default impl for things like disk 
+  // full checks. we randomly exercise "raw" directories anyway. We ensure default impls are used:
   
-  /** Use this when throwing fake {@code IOException},
-   *  e.g. from {@link MockDirectoryWrapper.Failure}. */
-  public static class FakeIOException extends IOException {
+  @Override
+  public final ChecksumIndexInput openChecksumInput(String name, IOContext context) throws IOException {
+    return super.openChecksumInput(name, context);
   }
 
+  @Override
+  public final void copyFrom(Directory from, String src, String dest, IOContext context) throws IOException {
+    super.copyFrom(from, src, dest, context);
+  }
+
+  @Override
+  protected final void ensureOpen() throws AlreadyClosedException {
+    super.ensureOpen();
+  }
 }

Modified: lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java (original)
+++ lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java Tue Mar 31 05:22:40 2015
@@ -73,10 +73,8 @@ public class LineFileDocs implements Clo
 
   @Override
   public synchronized void close() throws IOException {
-    if (reader != null) {
-      reader.close();
-      reader = null;
-    }
+    IOUtils.close(reader, threadDocs);
+    reader = null;
   }
   
   private long randomSeekPos(Random random, long size) {
@@ -205,7 +203,7 @@ public class LineFileDocs implements Clo
     }
   }
 
-  private final ThreadLocal<DocState> threadDocs = new ThreadLocal<>();
+  private final CloseableThreadLocal<DocState> threadDocs = new CloseableThreadLocal<>();
 
   /** Note: Document instance is re-used per-thread */
   public Document nextDoc() throws IOException {

Modified: lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Tue Mar 31 05:22:40 2015
@@ -81,7 +81,7 @@ import com.carrotsearch.randomizedtestin
 import com.carrotsearch.randomizedtesting.rules.NoClassHooksShadowingRule;
 import com.carrotsearch.randomizedtesting.rules.NoInstanceHooksOverridesRule;
 import com.carrotsearch.randomizedtesting.rules.StaticFieldsInvariantRule;
-import com.carrotsearch.randomizedtesting.rules.SystemPropertiesInvariantRule;
+
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Field;
@@ -93,10 +93,10 @@ import org.apache.lucene.index.*;
 import org.apache.lucene.index.IndexReader.ReaderClosedListener;
 import org.apache.lucene.index.TermsEnum.SeekStatus;
 import org.apache.lucene.search.AssertingIndexSearcher;
-import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.Filter;
-import org.apache.lucene.search.FilterCachingPolicy;
+import org.apache.lucene.search.LRUQueryCache;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryCachingPolicy;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.QueryUtils.FCInvisibleMultiReader;
 import org.apache.lucene.store.BaseDirectoryWrapper;
@@ -110,6 +110,7 @@ import org.apache.lucene.store.MergeInfo
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.store.MockDirectoryWrapper.Throttling;
 import org.apache.lucene.store.NRTCachingDirectory;
+import org.apache.lucene.store.RawDirectoryWrapper;
 import org.apache.lucene.util.automaton.AutomatonTestUtil;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
 import org.apache.lucene.util.automaton.RegExp;
@@ -416,16 +417,6 @@ public abstract class LuceneTestCase ext
     LEAVE_TEMPORARY = defaultValue;
   }
 
-  /**
-   * These property keys will be ignored in verification of altered properties.
-   * @see SystemPropertiesInvariantRule
-   * @see #ruleChain
-   * @see #classRules
-   */
-  private static final String [] IGNORED_INVARIANT_PROPERTIES = {
-    "user.timezone", "java.rmi.server.randomIDs"
-  };
-
   /** Filesystem-based {@link Directory} implementations. */
   private static final List<String> FS_DIRECTORIES = Arrays.asList(
     "SimpleFSDirectory",
@@ -440,14 +431,14 @@ public abstract class LuceneTestCase ext
     CORE_DIRECTORIES.add("RAMDirectory");
   }
 
-  /** A {@link org.apache.lucene.search.FilterCachingPolicy} that randomly caches. */
-  public static final FilterCachingPolicy MAYBE_CACHE_POLICY = new FilterCachingPolicy() {
+  /** A {@link org.apache.lucene.search.QueryCachingPolicy} that randomly caches. */
+  public static final QueryCachingPolicy MAYBE_CACHE_POLICY = new QueryCachingPolicy() {
 
     @Override
-    public void onUse(Filter filter) {}
+    public void onUse(Query query) {}
 
     @Override
-    public boolean shouldCache(Filter filter, LeafReaderContext context, DocIdSet set) throws IOException {
+    public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
       return random().nextBoolean();
     }
 
@@ -580,8 +571,20 @@ public abstract class LuceneTestCase ext
         return !(name.equals("setUp") || name.equals("tearDown"));
       }
     })
-    .around(new SystemPropertiesInvariantRule(IGNORED_INVARIANT_PROPERTIES))
     .around(classNameRule = new TestRuleStoreClassName())
+    .around(new TestRuleRestoreSystemProperties(
+        // Enlist all properties to which we have write access (security manager);
+        // these should be restored to previous state, no matter what the outcome of the test.
+
+        // We reset the default locale and timezone; these properties change as a side-effect
+        "user.language",
+        "user.timezone",
+        
+        // TODO: these should, ideally, be moved to Solr's base class.
+        "solr.directoryFactory",
+        "solr.solr.home",
+        "solr.data.dir"
+        ))
     .around(classEnvRule = new TestRuleSetupAndRestoreClassEnv());
 
 
@@ -608,7 +611,6 @@ public abstract class LuceneTestCase ext
     .outerRule(testFailureMarker)
     .around(ignoreAfterMaxFailures)
     .around(threadAndTestNameRule)
-    .around(new SystemPropertiesInvariantRule(IGNORED_INVARIANT_PROPERTIES))
     .around(new TestRuleSetupAndRestoreInstanceEnv())
     .around(parentChainCallRule);
 
@@ -1317,7 +1319,7 @@ public abstract class LuceneTestCase ext
     }
 
     if (bare) {
-      BaseDirectoryWrapper base = new BaseDirectoryWrapper(directory);
+      BaseDirectoryWrapper base = new RawDirectoryWrapper(directory);
       closeAfterSuite(new CloseableDirectory(base, suiteFailureMarker));
       return base;
     } else {
@@ -1436,7 +1438,7 @@ public abstract class LuceneTestCase ext
 
   /** 
    * Return a random Locale from the available locales on the system.
-   * see "https://issues.apache.org/jira/browse/LUCENE-4020"
+   * @see <a href="http://issues.apache.org/jira/browse/LUCENE-4020">LUCENE-4020</a>
    */
   public static Locale randomLocale(Random random) {
     Locale locales[] = Locale.getAvailableLocales();
@@ -1445,7 +1447,7 @@ public abstract class LuceneTestCase ext
 
   /** 
    * Return a random TimeZone from the available timezones on the system
-   * see "https://issues.apache.org/jira/browse/LUCENE-4020"
+   * @see <a href="http://issues.apache.org/jira/browse/LUCENE-4020">LUCENE-4020</a>
    */
   public static TimeZone randomTimeZone(Random random) {
     String tzIds[] = TimeZone.getAvailableIDs();
@@ -1648,6 +1650,20 @@ public abstract class LuceneTestCase ext
     }
   }
 
+  @Before
+  public void resetTestDefaultQueryCache() {
+    // Make sure each test method has its own cache
+    resetDefaultQueryCache();
+  }
+
+  @BeforeClass
+  public static void resetDefaultQueryCache() {
+    // we need to reset the query cache in an @BeforeClass so that tests that
+    // instantiate an IndexSearcher in an @BeforeClass method use a fresh new cache
+    IndexSearcher.setDefaultQueryCache(new LRUQueryCache(10000, 1 << 25));
+    IndexSearcher.setDefaultQueryCachingPolicy(MAYBE_CACHE_POLICY);
+  }
+
   /**
    * Create a new searcher over the reader. This searcher might randomly use
    * threads.
@@ -1735,6 +1751,7 @@ public abstract class LuceneTestCase ext
             : new IndexSearcher(r.getContext(), ex);
       }
       ret.setSimilarity(classEnvRule.similarity);
+      ret.setQueryCachingPolicy(MAYBE_CACHE_POLICY);
       return ret;
     }
   }

Modified: lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java (original)
+++ lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java Tue Mar 31 05:22:40 2015
@@ -20,10 +20,8 @@ package org.apache.lucene.util;
 import java.io.PrintStream;
 import java.util.Arrays;
 import java.util.Date;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Locale;
-import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 import java.util.TimeZone;
@@ -38,7 +36,6 @@ import org.apache.lucene.codecs.cheapbas
 import org.apache.lucene.codecs.compressing.CompressingCodec;
 import org.apache.lucene.codecs.lucene50.Lucene50Codec;
 import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
 import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
 import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
 import org.apache.lucene.index.RandomCodec;
@@ -68,11 +65,6 @@ import static org.apache.lucene.util.Luc
  * doesn't fit anywhere else).
  */
 final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
-  /**
-   * Restore these system property values.
-   */
-  private HashMap<String, String> restoreProperties = new HashMap<>();
-
   private Codec savedCodec;
   private Locale savedLocale;
   private TimeZone savedTimeZone;
@@ -116,15 +108,10 @@ final class TestRuleSetupAndRestoreClass
     // enable this by default, for IDE consistency with ant tests (as it's the default from ant)
     // TODO: really should be in solr base classes, but some extend LTC directly.
     // we do this in beforeClass, because some tests currently disable it
-    restoreProperties.put("solr.directoryFactory", System.getProperty("solr.directoryFactory"));
     if (System.getProperty("solr.directoryFactory") == null) {
       System.setProperty("solr.directoryFactory", "org.apache.solr.core.MockDirectoryFactory");
     }
 
-    // Restore more Solr properties. 
-    restoreProperties.put("solr.solr.home", System.getProperty("solr.solr.home"));
-    restoreProperties.put("solr.data.dir", System.getProperty("solr.data.dir"));
-
     // if verbose: print some debugging stuff about which codecs are loaded.
     if (VERBOSE) {
       Set<String> codecs = Codec.availableCodecs();
@@ -224,9 +211,6 @@ final class TestRuleSetupAndRestoreClass
     locale = testLocale.equals("random") ? randomLocale : localeForName(testLocale);
     Locale.setDefault(locale);
 
-    // TimeZone.getDefault will set user.timezone to the default timezone of the user's locale.
-    // So store the original property value and restore it at end.
-    restoreProperties.put("user.timezone", System.getProperty("user.timezone"));
     savedTimeZone = TimeZone.getDefault();
     TimeZone randomTimeZone = randomTimeZone(random());
     timeZone = testTimeZone.equals("random") ? randomTimeZone : TimeZone.getTimeZone(testTimeZone);
@@ -291,15 +275,6 @@ final class TestRuleSetupAndRestoreClass
    */
   @Override
   protected void after() throws Exception {
-    for (Map.Entry<String,String> e : restoreProperties.entrySet()) {
-      if (e.getValue() == null) {
-        System.clearProperty(e.getKey());
-      } else {
-        System.setProperty(e.getKey(), e.getValue());
-      }
-    }
-    restoreProperties.clear();
-
     Codec.setDefault(savedCodec);
     InfoStream.setDefault(savedInfoStream);
     if (savedLocale != null) Locale.setDefault(savedLocale);

Modified: lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java (original)
+++ lucene/dev/branches/lucene6271/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java Tue Mar 31 05:22:40 2015
@@ -73,6 +73,7 @@ import org.apache.lucene.index.CheckInde
 import org.apache.lucene.index.ConcurrentMergeScheduler;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.FilterLeafReader;
@@ -301,14 +302,23 @@ public final class TestUtil {
     ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
     PrintStream infoStream = new PrintStream(bos, false, IOUtils.UTF_8);
 
-    reader.checkIntegrity();
-    CheckIndex.testLiveDocs(reader, infoStream, true);
-    CheckIndex.testFieldInfos(reader, infoStream, true);
-    CheckIndex.testFieldNorms(reader, infoStream, true);
-    CheckIndex.testPostings(reader, infoStream, false, true);
-    CheckIndex.testStoredFields(reader, infoStream, true);
-    CheckIndex.testTermVectors(reader, infoStream, false, crossCheckTermVectors, true);
-    CheckIndex.testDocValues(reader, infoStream, true);
+    final CodecReader codecReader;
+    if (reader instanceof CodecReader) {
+      codecReader = (CodecReader) reader;
+      reader.checkIntegrity();
+    } else {
+      codecReader = SlowCodecReaderWrapper.wrap(reader);
+    }
+    CheckIndex.testLiveDocs(codecReader, infoStream, true);
+    CheckIndex.testFieldInfos(codecReader, infoStream, true);
+    CheckIndex.testFieldNorms(codecReader, infoStream, true);
+    CheckIndex.testPostings(codecReader, infoStream, false, true);
+    CheckIndex.testStoredFields(codecReader, infoStream, true);
+    CheckIndex.testTermVectors(codecReader, infoStream, false, crossCheckTermVectors, true);
+    CheckIndex.testDocValues(codecReader, infoStream, true);
+    
+    // some checks really against the reader API
+    checkReaderSanity(reader);
     
     if (LuceneTestCase.INFOSTREAM) {
       System.out.println(bos.toString(IOUtils.UTF_8));
@@ -324,6 +334,76 @@ public final class TestUtil {
       assert Accountables.toString(sr) != null;
     }
   }
+  
+  // used by TestUtil.checkReader to check some things really unrelated to the index,
+  // just looking for bugs in indexreader implementations.
+  private static void checkReaderSanity(LeafReader reader) throws IOException {
+    for (FieldInfo info : reader.getFieldInfos()) {
+      
+      // reader shouldn't return normValues if the field does not have them
+      if (!info.hasNorms()) {
+        if (reader.getNormValues(info.name) != null) {
+          throw new RuntimeException("field: " + info.name + " should omit norms but has them!");
+        }
+      }
+      
+      // reader shouldn't return docValues if the field does not have them
+      // reader shouldn't return multiple docvalues types for the same field.
+      switch(info.getDocValuesType()) {
+        case NONE:
+          if (reader.getBinaryDocValues(info.name) != null ||
+              reader.getNumericDocValues(info.name) != null ||
+              reader.getSortedDocValues(info.name) != null || 
+              reader.getSortedSetDocValues(info.name) != null || 
+              reader.getDocsWithField(info.name) != null) {
+            throw new RuntimeException("field: " + info.name + " has docvalues but should omit them!");
+          }
+          break;
+        case SORTED:
+          if (reader.getBinaryDocValues(info.name) != null ||
+              reader.getNumericDocValues(info.name) != null ||
+              reader.getSortedNumericDocValues(info.name) != null ||
+              reader.getSortedSetDocValues(info.name) != null) {
+            throw new RuntimeException(info.name + " returns multiple docvalues types!");
+          }
+          break;
+        case SORTED_NUMERIC:
+          if (reader.getBinaryDocValues(info.name) != null ||
+              reader.getNumericDocValues(info.name) != null ||
+              reader.getSortedSetDocValues(info.name) != null ||
+              reader.getSortedDocValues(info.name) != null) {
+            throw new RuntimeException(info.name + " returns multiple docvalues types!");
+          }
+          break;
+        case SORTED_SET:
+          if (reader.getBinaryDocValues(info.name) != null ||
+              reader.getNumericDocValues(info.name) != null ||
+              reader.getSortedNumericDocValues(info.name) != null ||
+              reader.getSortedDocValues(info.name) != null) {
+            throw new RuntimeException(info.name + " returns multiple docvalues types!");
+          }
+          break;
+        case BINARY:
+          if (reader.getNumericDocValues(info.name) != null ||
+              reader.getSortedDocValues(info.name) != null ||
+              reader.getSortedNumericDocValues(info.name) != null ||
+              reader.getSortedSetDocValues(info.name) != null) {
+            throw new RuntimeException(info.name + " returns multiple docvalues types!");
+          }
+          break;
+        case NUMERIC:
+          if (reader.getBinaryDocValues(info.name) != null ||
+              reader.getSortedDocValues(info.name) != null ||
+              reader.getSortedNumericDocValues(info.name) != null ||
+              reader.getSortedSetDocValues(info.name) != null) {
+            throw new RuntimeException(info.name + " returns multiple docvalues types!");
+          }
+          break;
+        default:
+          throw new AssertionError();
+      }
+    }
+  }
 
   /** start and end are BOTH inclusive */
   public static int nextInt(Random r, int start, int end) {

Modified: lucene/dev/branches/lucene6271/lucene/tools/forbiddenApis/base.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/tools/forbiddenApis/base.txt?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/tools/forbiddenApis/base.txt (original)
+++ lucene/dev/branches/lucene6271/lucene/tools/forbiddenApis/base.txt Tue Mar 31 05:22:40 2015
@@ -34,3 +34,7 @@ java.io.File#delete() @ use Files.delete
 
 @defaultMessage Use shuffle(List, Random) instead so that it can be reproduced
 java.util.Collections#shuffle(java.util.List)
+
+@defaultMessage Construct MessageFormat(String pattern, String locale) and then use the format(String,Object...) method
+java.text.MessageFormat#format(java.lang.String,java.lang.Object[])
+java.text.MessageFormat#<init>(java.lang.String)

Modified: lucene/dev/branches/lucene6271/lucene/tools/junit4/tests.policy
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/tools/junit4/tests.policy?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/tools/junit4/tests.policy (original)
+++ lucene/dev/branches/lucene6271/lucene/tools/junit4/tests.policy Tue Mar 31 05:22:40 2015
@@ -15,76 +15,91 @@
  * limitations under the License.
  */
 
-// Policy file to prevent tests from writing outside the test sandbox directory
-// (must be given as a sysprop: tests.sandbox.dir)
-// This policy also disallows stuff like listening on network ports of interfaces
-// different than 127.0.0.1.
-
-// PLEASE NOTE: You may need to enable other permissions when new tests are added,
-// everything not allowed here is forbidden!
+// Policy file for lucene tests. Please keep minimal and avoid wildcards.
 
 grant {
-  // permissions for file access, write access only to sandbox:
-  permission java.io.FilePermission "<<ALL FILES>>", "read,execute";
-  permission java.io.FilePermission "${junit4.childvm.cwd}", "read,execute";
-  permission java.io.FilePermission "${junit4.childvm.cwd}${/}temp", "read,execute,write,delete";
-  permission java.io.FilePermission "${junit4.childvm.cwd}${/}temp${/}-", "read,execute,write,delete";
-  permission java.io.FilePermission "${junit4.tempDir}${/}*", "read,execute,write,delete";
-  permission java.io.FilePermission "${clover.db.dir}${/}-", "read,execute,write,delete";
-  
-  // all possibilities of accepting/binding connections on localhost with ports >=1024:
-  permission java.net.SocketPermission "localhost:1024-", "accept,listen";
-  permission java.net.SocketPermission "127.0.0.1:1024-", "accept,listen";
-  permission java.net.SocketPermission "[::1]:1024-", "accept,listen";
+  // contain read access to only what we need:
+  // 3rd party jar resources (where symlinks are not supported), test-files/ resources
+  permission java.io.FilePermission "${common.dir}${/}-", "read";
+  // 3rd party jar resources (where symlinks are supported)
+  permission java.io.FilePermission "${user.home}${/}.ivy2${/}cache${/}-", "read";
+  // system jar resources, and let TestIndexWriterOnJRECrash fork its jvm
+  permission java.io.FilePermission "${java.home}${/}-", "read,execute";
+  // should be enclosed within common.dir, but just in case:
+  permission java.io.FilePermission "${junit4.childvm.cwd}", "read";
+
+  // jenkins wants to read outside its sandbox, to use a special linedocs file.
+  // this is best effort and not really supported.
+  permission java.io.FilePermission "/home/jenkins/lucene-data/enwiki.random.lines.txt", "read";
+
+  // write only to sandbox
+  permission java.io.FilePermission "${junit4.childvm.cwd}${/}temp", "read,write,delete";
+  permission java.io.FilePermission "${junit4.childvm.cwd}${/}temp${/}-", "read,write,delete";
+  permission java.io.FilePermission "${junit4.tempDir}${/}*", "read,write,delete";
+  permission java.io.FilePermission "${clover.db.dir}${/}-", "read,write,delete";
+
+  // needed by gson serialization of junit4 runner: TODO clean that up
+  permission java.lang.RuntimePermission "accessDeclaredMembers";
+  permission java.lang.reflect.ReflectPermission "suppressAccessChecks";
+  // needed by junit4 runner to capture sysout/syserr:
+  permission java.lang.RuntimePermission "setIO";
+  // needed by randomized runner to catch failures from other threads:
+  permission java.lang.RuntimePermission "setDefaultUncaughtExceptionHandler";
+  // needed by randomized runner getTopThreadGroup:
+  permission java.lang.RuntimePermission "modifyThreadGroup";
+  // needed by tests e.g. shutting down executors:
+  permission java.lang.RuntimePermission "modifyThread";
+  // needed for tons of test hacks etc
+  permission java.lang.RuntimePermission "getStackTrace";
+  // needed for mock filesystems in tests
+  permission java.lang.RuntimePermission "fileSystemProvider";
+  // needed for mock filesystems in tests (to capture implCloseChannel) 
+  permission java.lang.RuntimePermission "accessClassInPackage.sun.nio.ch";
+  // needed by junit nested compat tests (due to static fields reflection), TODO clean these up:
+  permission java.lang.RuntimePermission "accessClassInPackage.sun.util.calendar";
+  permission java.lang.RuntimePermission "accessClassInPackage.sun.util.locale";
+  permission java.lang.RuntimePermission "accessClassInPackage.sun.nio.fs";
+  // needed by queryparser/ NLS., TODO clean this up:
+  permission java.lang.RuntimePermission "accessClassInPackage.sun.util";
+  // needed for test of IOUtils.spins (maybe it can be avoided)
+  permission java.lang.RuntimePermission "getFileStoreAttributes";
+  // analyzers/morfologik: needed for a horrible context classloader hack for solr in morfologikfilter: nuke this
+  permission java.lang.RuntimePermission "setContextClassLoader";
+  // analyzers/uima: needed by UIMA message localization... (?)
+  permission java.lang.RuntimePermission "createSecurityManager";
+  permission java.lang.RuntimePermission "createClassLoader";
+  // expressions TestCustomFunctions (only on older java8?)
+  permission java.lang.RuntimePermission "getClassLoader";
+  // needed to test unmap hack on platforms that support it
+  permission java.lang.RuntimePermission "accessClassInPackage.sun.misc";
+  
+  // read access to all system properties:
+  permission java.util.PropertyPermission "*", "read";
+  // write access to only these:
+  // locale randomization
+  permission java.util.PropertyPermission "user.language", "write";
+  // timezone randomization
+  permission java.util.PropertyPermission "user.timezone", "write";
+
+  // used by nested tests? (e.g. TestLeaveFilesIfTestFails). TODO: look into this
+  permission java.util.PropertyPermission "tests.runnested", "write";
+
+  // solr properties. TODO: move these out to SolrTestCase
+  permission java.util.PropertyPermission "solr.data.dir", "write";
+  permission java.util.PropertyPermission "solr.solr.home", "write";
+  permission java.util.PropertyPermission "solr.directoryFactory", "write";
+
+  // replicator: jetty tests require some network permissions:
+  // all possibilities of accepting/binding/connecting on localhost with ports >= 1024:
+  permission java.net.SocketPermission "localhost:1024-", "accept,listen,connect,resolve";
+  permission java.net.SocketPermission "127.0.0.1:1024-", "accept,listen,connect,resolve";
+  permission java.net.SocketPermission "[::1]:1024-", "accept,listen,connect,resolve";
   
   // This is a special case, because the network config of the ASF Jenkins server is broken,
   // see: http://freebsd.1045724.n5.nabble.com/jail-external-and-localhost-distinction-td3967320.html
-  permission java.net.SocketPermission "lucene.zones.apache.org:1024-", "accept,listen";
-  
-  // Allow connecting to the internet anywhere
-  permission java.net.SocketPermission "*", "connect,resolve";
-  
-  // Basic permissions needed for Lucene to work:
-  permission java.util.PropertyPermission "*", "read,write";
-  permission java.lang.reflect.ReflectPermission "*";
-  permission java.lang.RuntimePermission "*";
-
-  // These two *have* to be spelled out a separate
-  permission java.lang.management.ManagementPermission "control";
-  permission java.lang.management.ManagementPermission "monitor";
-
-  // Solr needs those:
-  permission java.net.NetPermission "*";
-  permission java.sql.SQLPermission "*";
-  permission java.util.logging.LoggingPermission "control";
-  permission javax.management.MBeanPermission "*", "*";
-  permission javax.management.MBeanServerPermission "*";
-  permission javax.management.MBeanTrustPermission "*";
-  permission javax.security.auth.AuthPermission "*";
-  permission javax.security.auth.PrivateCredentialPermission "org.apache.hadoop.security.Credentials * \"*\"", "read";
-  permission java.security.SecurityPermission "putProviderProperty.SaslPlainServer";
-  permission java.security.SecurityPermission "insertProvider.SaslPlainServer";
-  permission javax.xml.bind.JAXBPermission "setDatatypeConverter";
-  
-  // TIKA uses BouncyCastle and that registers new provider for PDF parsing + MSOffice parsing. Maybe report as bug!
-  permission java.security.SecurityPermission "putProviderProperty.BC";
-  permission java.security.SecurityPermission "insertProvider.BC";
-
-  // Needed for some things in DNS caching in the JVM
-  permission java.security.SecurityPermission "getProperty.networkaddress.cache.ttl";
-  permission java.security.SecurityPermission "getProperty.networkaddress.cache.negative.ttl";
-
-  // SSL related properties for Solr tests
-  permission java.security.SecurityPermission "getProperty.ssl.*";
+  permission java.net.SocketPermission "lucene.zones.apache.org:1024-", "accept,listen,connect,resolve";
 
-  // SASL/Kerberos related properties for Solr tests
-  permission javax.security.auth.PrivateCredentialPermission "javax.security.auth.kerberos.KerberosTicket * \"*\"", "read";
-  
-  // may only be necessary with Java 7?
-  permission javax.security.auth.PrivateCredentialPermission "javax.security.auth.kerberos.KeyTab * \"*\"", "read";
-  permission javax.security.auth.PrivateCredentialPermission "sun.security.jgss.krb5.Krb5Util$KeysFromKeyTab * \"*\"", "read";
-  
-  permission javax.security.auth.kerberos.ServicePermission "krbtgt/EXAMPLE.COM@EXAMPLE.COM", "initiate";
-  permission javax.security.auth.kerberos.ServicePermission "zookeeper/127.0.0.1@EXAMPLE.COM", "initiate";
-  permission javax.security.auth.kerberos.ServicePermission "zookeeper/127.0.0.1@EXAMPLE.COM", "accept";
+  // SSL related properties for jetty
+  permission java.security.SecurityPermission "getProperty.ssl.KeyManagerFactory.algorithm";
+  permission java.security.SecurityPermission "getProperty.ssl.TrustManagerFactory.algorithm";
 };

Modified: lucene/dev/branches/lucene6271/lucene/version.properties
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/version.properties?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/version.properties (original)
+++ lucene/dev/branches/lucene6271/lucene/version.properties Tue Mar 31 05:22:40 2015
@@ -1,6 +1,6 @@
 # This file contains some version properties as used by various build files.
 
-# RELEAE MANAGER must change this file after creating a release and
+# RELEASE MANAGER must change this file after creating a release and
 # enter new base version (format "x.y.z", no prefix/appendix): 
 version.base=6.0.0
 

Modified: lucene/dev/branches/lucene6271/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/CHANGES.txt?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/CHANGES.txt (original)
+++ lucene/dev/branches/lucene6271/solr/CHANGES.txt Tue Mar 31 05:22:40 2015
@@ -4,8 +4,7 @@ Introduction
 ------------
 Apache Solr is an open source enterprise search server based on the Apache Lucene Java
 search library, with XML/HTTP and JSON APIs, hit highlighting, faceted search,
-caching, replication, and a web administration interface. It runs in a Java
-servlet container such as Jetty.
+caching, replication, and a web administration interface.
 
 See http://lucene.apache.org/solr for more information.
 
@@ -28,6 +27,7 @@ Carrot2 3.9.0
 Velocity 1.7 and Velocity Tools 2.0
 Apache UIMA 2.3.1
 Apache ZooKeeper 3.4.6
+Jetty 9.2.9.v20150224
 
 System Requirements
 ----------------------
@@ -69,7 +69,36 @@ Carrot2 3.9.0
 Velocity 1.7 and Velocity Tools 2.0
 Apache UIMA 2.3.1
 Apache ZooKeeper 3.4.6
-Jetty 9.2.6.v20141205
+Jetty 8.1.10.v20130312
+
+Upgrading from Solr 5.0
+-----------------------
+
+* SolrClient query functions now declare themselves as throwing IOException in
+  addition to SolrServerException, to bring them in line with the update
+  functions.
+
+* SolrRequest.process() is now final.  Subclasses should instead be parameterized
+  by their corresponding SolrResponse type, and implement createResponse()
+
+* The signature of SolrDispatchFilter.createCoreContainer() has changed to take
+  (String,Properties) arguments
+
+* Deprecated the 'lib' option added to create-requesthandler as part of SOLR-6801 in 5.0 release.
+  Please use the add-runtimelib command
+  
+* Tika's runtime dependency of 'jhighlight' was removed as the latter was found to 
+  contain some LGPL-only code. Until that's resolved by Tika, you can download the
+  .jar yourself and place it under contrib/extraction/lib.  
+
+* A twitter engineer discovered a JVM bug that causes GC pause problems.  The
+  workaround for those problems makes certain functionality impossible, such as
+  running the jstat program on your Solr instance.  That workaround has been
+  implemented in the bin/solr start scripts.  If you need the missing java
+  functionality, delete the "-XX:+PerfDisableSharedMem" parameter from
+  bin/solr.in.sh or bin/solr.in.cmd.
+
+  http://www.evanjones.ca/jvm-mmap-pause.html
 
 Detailed Change List
 ----------------------
@@ -112,11 +141,107 @@ New Features
 
 * SOLR-1945: Add support for child docs in DocumentObjectBinder (Noble Paul, Mark Miller)
 
-* SOLR-7125: You can upload and download configurations via CloudSolrClient
-  (Alan Woodward)
+* SOLR-7125, SOLR-7158: You can upload and download configurations via CloudSolrClient
+  (Alan Woodward, Ishan Chattopadhyaya)
 
 * SOLR-5507: Admin UI - Refactoring using AngularJS, first part (Upayavira via 
   Erick Erickson)
+  
+* SOLR-7164: BBoxField defaults sub fields to not-stored (ryan)
+
+* SOLR-7155,SOLR-7201: All SolrClient methods now take an optional 'collection' argument
+  (Alan Woodward, Shawn Heisey)
+
+* SOLR-7073: Support adding a jar to a collections classpath (Noble Paul)
+
+* SOLR-6359: Allow number of logs and records kept by UpdateLog to be configured
+  (Ramkumar Aiyengar)
+
+* SOLR-7189: Allow DIH to extract content from embedded documents via Tika.
+  (Tim Allison via shalin)
+
+* SOLR-6841: Visualize lucene segment information in Admin UI.
+  (Alexey Kozhemiakin, Michal Bienkowski, hossman, Shawn Heisey, Varun Thacker via shalin)
+
+* SOLR-5846: EnumField supports DocValues functionality. (Elran Dvir, shalin)
+
+* SOLR-4044: CloudSolrClient.connect() throws a more useful exception if the
+  cluster is not ready, and can now take an optional timeout argument to wait
+  for the cluster. (Alan Woodward, shalin, yonik, Mark Miller, Vitaliy Zhovtyuk)
+
+* SOLR-7126: Secure loading of runtime external jars (Noble Paul)
+
+* SOLR-6349: Added support for stats.field localparams to enable/disable individual stats to 
+  limit the amount of computation done and the amount of data returned. 
+  eg: stats.field={!min=true max=true}field_name
+  (Tomas Fernandez-Lobbe, Xu Zhang, hossman)
+
+* SOLR-7218: lucene/solr query syntax to give any query clause a constant score.
+  General Form: <clause>^=<constant_score>
+  Example: (color:blue color:green)^=2.0 text:shoes
+  (yonik)
+
+* SOLR-7214: New Facet module with a JSON API, facet functions, aggregations, and analytics.
+  Any facet type can have sub facets, and facets can be sorted by arbitrary aggregation functions.
+  Examples:
+    json.facet={x:'avg(price)', y:'unique(color)'}
+    json.facet={count1:{query:"price:[10 TO 20]"}, count2:{query:"color:blue AND popularity:[0 TO 50]"} }
+    json.facet={categories:{terms:{field:cat, sort:"x desc", facet:{x:"avg(price)", y:"sum(price)"}}}}
+  (yonik)
+  
+* SOLR-6141: Schema API: Remove fields, dynamic fields, field types and copy
+  fields; and replace fields, dynamic fields and field types. (Steve Rowe)
+
+* SOLR-7217: HTTP POST body is auto-detected when the client is curl and the content
+  type is form data (curl's default), allowing users to use curl to send
+  JSON or XML without having to specify the content type. (yonik)
+
+* SOLR-6892: Update processors can now be top-level components and they can be
+  specified in request to create a new custom update chain (Noble Paul)
+
+* SOLR-7216: Solr JSON Request API:
+  - HTTP search requests can have a JSON body.
+  - JSON request can also be passed via the "json" parameter.
+  - Smart merging of multiple JSON parameters: ruery parameters starting with "json."
+    will be merged into the JSON request.
+  - Legacy query parameters can also be passed in the "params" block of
+    the JSON request.
+  (yonik)
+
+* SOLR-7245: Temporary ZK election or connection loss should not stall indexing
+  due to leader initiated recovery (Ramkumar Aiyengar)
+
+* SOLR-6350: StatsComponent now supports Percentiles (Xu Zhang, hossman)
+
+* SOLR-7306: Percentiles support for the new facet module.  Percentiles
+  can be calculated for all facet buckets and field faceting can sort
+  by percentile values.
+  Examples:
+    json.facet={ median_age : "percentile(age,50)" }
+    json.facet={ salary_percentiles : "percentile(salary,25,50,75)" }
+  (yonik)
+
+* SOLR-7307: EmbeddedSolrServer can now be started up by passing a path to a
+  solr home directory, or a NodeConfig object (Alan Woodward, Mike Drob)
+
+* SOLR-1387: Add facet.contains and facet.contains.ignoreCase options (Tom Winch
+  via Alan Woodward)
+
+* SOLR-7082: Streaming Aggregation for SolrCloud (Joel bernstein, Yonik Seeley)
+
+* SOLR-7212: Parameter substitution / macro expansion across entire request.
+  Substitution can contain further expansions and default values are supported.
+  Example: q=price:[ ${low:0} TO ${high} ]&low=100&high=200
+  (yonik)
+
+* SOLR-7226: Make /query/* jmx/* , requestDispatcher/*, <listener> <initParams>
+  properties in solrconfig.xml editable (Noble Paul)
+
+* SOLR-7240: '/' redirects to '/solr/' for convenience (Martijn Koster, hossman)
+
+* SOLR-5911: Added payload support for term vectors. New "termPayloads" option for fields
+  / types in the schema, and "tv.payloads" param for the term vector component.
+  (Mike McCandless, David Smiley)
 
 Bug Fixes
 ----------------------
@@ -148,11 +273,101 @@ Bug Fixes
 * SOLR-7104: Propagate property prefix parameters for ADDREPLICA Collections API call.
   (Varun Thacker via Anshum Gupta)
 
+* SOLR-7113: Multiple calls to UpdateLog#init is not thread safe with respect to the
+  HDFS FileSystem client object usage. (Mark Miller, Vamsee Yarlagadda)
+
+* SOLR-7128: Two phase distributed search is fetching extra fields in GET_TOP_IDS phase.
+  (Pablo Queixalos, shalin)
+
+* SOLR-7139: Fix SolrContentHandler for TIKA to ignore multiple startDocument events.
+  (Chris A. Mattmann, Uwe Schindler)
+
+* SOLR-7178: OverseerAutoReplicaFailoverThread compares Integer objects using ==
+  (shalin)
+
+* SOLR-7171: BaseDistributedSearchTestCase now clones getSolrHome() for each subclass, 
+  and consistently uses getSolrXml().  (hossman)
+
+* SOLR-6657:  DocumentDictionaryFactory requires weightField to be mandatory, but it shouldn't
+  (Erick Erickson)
+
+* SOLR-7206: MiniSolrCloudCluster wasn't dealing with SSL mode correctly (Alan
+  Woodward)
+
+* SOLR-4464: DIH Processed documents counter resets to zero after first entity is processed.
+  (Dave Cook, Shawn Heisey, Aaron Greenspan, Thomas Champagne via shalin)
+
+* SOLR-7209: /update/json/docs carry forward fields from previous records (Noble Paul)
+
+* SOLR-7195: Fixed a bug where the bin/solr shell script would incorrectly
+  detect another Solr process listening on the same port number.  If the
+  requested listen port was 8983, it would match on another Solr using port
+  18983 for any purpose.  Also escapes the dot character in all grep commands
+  looking for start.jar.
+  (Xu Zhang via Shawn Heisey)
+
+* SOLR-6682: Fix response when using EnumField with StatsComponent
+  (Xu Zhang via hossman)
+
+* SOLR-7109: Indexing threads stuck during network partition can put leader into down state.
+  (Mark Miller, Anshum Gupta, Ramkumar Aiyengar, yonik, shalin)
+  
+* SOLR-7092: Stop the HDFS lease recovery retries in HdfsTransactionLog on close and try
+  to avoid lease recovery on closed files. (Mark Miller)
+
+* SOLR-7285: ActionThrottle will not pause if getNanoTime first returns 0.
+  (Mark Miller, Gregory Chanan)
+  
+* SOLR-7141: RecoveryStrategy: Raise time that we wait for any updates from the leader before
+  they saw the recovery state to have finished. (Mark Miller)
+
+* SOLR-7284: HdfsUpdateLog is using hdfs FileSystem.get without turning off the cache.
+  (Mark Miller)
+  
+* SOLR-7286: Using HDFS's FileSystem.newInstance does not guarantee a new instance.
+  (Mark Miller)
+
+* SOLR-7134: Replication can still cause index corruption. (Mark Miller, shalin, Mike Drob)
+
+* SOLR-7248: In legacyCloud=false mode we should check if the core was hosted on the same node before registering it
+  (Varun Thacker, Noble Paul, Mark Miller)
+
+* SOLR-7294: Migrate API fails with 'Invalid status request: notfoundretried 6times' message.
+  (Jessica Cheng Mallet, shalin)
+
+* SOLR-7254: Make an invalid negative start/rows throw a HTTP 400 error (Bad Request) instead
+  of causing a 500 error.  (Ramkumar Aiyengar, Hrishikesh Gadre, yonik)
+
+* SOLR-7305: BlendedInfixLookupFactory swallows root IOException when it occurs.
+  (Stephan Lagraulet via shalin)
+
+* SOLR-7293: Fix bug that Solr server does not listen on IPv6 interfaces by default.
+  (Uwe Schindler, Sebastian Pesman)
+
+* SOLR-7298: Fix Collections API calls (SolrJ) to not add name parameter when not needed.
+  (Shai Erera, Anshum Gupta)
+
+* SOLR-7309: Make bin/solr, bin/post work when Solr installation directory contains spaces
+  (Ramkumar Aiyengar, Martijn Koster)
+
+* SOLR-7319: Workaround for the "Four Month Bug" GC pause problem discovered
+  by a twitter software engineer.  This causes GC pauses when JVM statistics
+  are left enabled and there is heavy MMAP write activity.
+  http://www.evanjones.ca/jvm-mmap-pause.html
+  (Shawn Heisey)
+
 Optimizations
 ----------------------
+
  * SOLR-7049: Move work done by the LIST Collections API call to the Collections
    Handler (Varun Thacker via Anshum Gupta).
 
+ * SOLR-7116: Distributed facet refinement requests would needlessly compute other types
+   of faceting that have already been computed. (David Smiley, Hossman)
+
+ * SOLR-7239: improved performance of min & max in StatsComponent, as well as situations 
+   where local params disable all stats (hossman)
+
 Other Changes
 ----------------------
 
@@ -182,14 +397,67 @@ Other Changes
 
 * SOLR-7032: Clean up test remnants of old-style solr.xml (Erick Erickson)
 
+* SOLR-7145: SolrRequest is now parametrized by its response type (Alan
+  Woodward)
+
 * SOLR-7142: Fix TestFaceting.testFacets. (Michal Kroliczek via shalin)
 
+* SOLR-7156: Fix test failures due to resource leaks on windows.
+  (Ishan Chattopadhyaya via shalin)
+
+* SOLR-7147: Introduce new TrackingShardHandlerFactory for monitoring what requests
+  are sent to shards during tests. (hossman, shalin)
+
+* SOLR-7160: Rename ConfigSolr to NodeConfig, and decouple it from xml
+  representation (Alan Woodward)
+
+* SOLR-7166: Encapsulate JettySolrRunner configuration (Alan Woodward)
+
+* SOLR-7130: Make stale state notification work without failing the requests
+  (Noble Paul, shalin)
+
+* SOLR-7151: SolrClient query methods throw IOException (Alan Woodward)
+
+* SOLR-7179: JettySolrRunner no longer passes configuration to
+  SolrDispatchFilter via system properties, but instead uses a Properties
+  object in the servlet context (Alan Woodward)
+
+* SOLR-6275: Improve accuracy of QTime reporting (Ramkumar Aiyengar)
+
+* SOLR-7174: DIH should reset TikaEntityProcessor so that it is capable
+  of re-use (Alexandre Rafalovitch , Gary Taylor via Noble Paul)
+
+* SOLR-6804: Untangle SnapPuller and ReplicationHandler (Ramkumar Aiyengar)
+
+* SOLR-7180: MiniSolrCloudCluster will startup and shutdown its jetties in
+  parallel (Alan Woodward, Tomás Fernández Löbbe, Vamsee Yarlagadda)
+
+* SOLR-7173: Fix ReplicationFactorTest on Windows by adding better retry
+  support after seeing no response exceptions. (Ishan Chattopadhyaya via Timothy Potter)
+
+* SOLR-7246: Speed up BasicZkTest, TestManagedResourceStorage (Ramkumar Aiyengar)
+
+* SOLR-7258: Forbid MessageFormat.format and MessageFormat single-arg constructor.
+  (shalin)
+
+* SOLR-7162: Remove unused SolrSortField interface. (yonik, Connor Warrington via shalin)
+
+* SOLR-6414: Update to Hadoop 2.6.0. (Mark Miller)
+
+* SOLR-6673: MDC based logging of collection, shard, replica, core
+  (Ishan Chattopadhyaya , Noble Paul)
+
+* SOLR-7291: Test indexing on ZK disconnect with ChaosMonkey tests (Ramkumar Aiyengar)
+
+* SOLR-7203: Remove buggy no-op retry code in HttpSolrClient (Alan Woodward,
+  Mark Miller, Greg Solovyev)
+
 ==================  5.0.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
 
 NOTE: Solr 5.0 only supports creating and removing SolrCloud collections through
-      the collections API, unlike previous versions. While not using the 
+      the collections API, unlike previous versions. While not using the
       collections API may still work in 5.0, it is unsupported, not recommended,
       and the behavior will change in a 5.x release.
 
@@ -200,7 +468,7 @@ Carrot2 3.9.0
 Velocity 1.7 and Velocity Tools 2.0
 Apache UIMA 2.3.1
 Apache ZooKeeper 3.4.6
-Jetty 9.2.6.v20141205
+Jetty 8.1.10.v20130312
 
 Upgrading from Solr 4.x
 ----------------------
@@ -539,7 +807,7 @@ Bug Fixes
 * SOLR-6631: DistributedQueue spinning on calling zookeeper getChildren()
   (Jessica Cheng Mallet, Mark Miller, Timothy Potter)
 
-* SOLR-6579:SnapPuller Replication blocks clean shutdown of tomcat
+* SOLR-6579: SnapPuller Replication blocks clean shutdown of tomcat
   (Philip Black-Knight via Noble Paul)
 
 * SOLR-6721: ZkController.ensureReplicaInLeaderInitiatedRecovery puts replica
@@ -974,6 +1242,100 @@ Other Changes
 * SOLR-6227: Avoid spurious failures of ChaosMonkeySafeLeaderTest by ensuring there's
   at least one jetty to kill. (shalin)
 
+* SOLR-7202: Remove deprecated string action types in Overseer and OverseerCollectionProcessor -
+  "deletecollection", "createcollection", "reloadcollection", "removecollection", "removeshard".
+  (Varun Thacker, shalin)
+
+==================  4.10.4 ==================
+
+Bug Fixes
+----------------------
+
+* SOLR-6931: We should do a limited retry when using HttpClient. 
+  (Mark Miller, Hrishikesh Gadre, Gregory Chanan)
+
+* SOLR-6780: Fixed a bug in how default/appends/invariants params were affecting the set 
+  of all "keys" found in the request parameters, resulting in some key=value param pairs 
+  being duplicated.  This was noticeably affecting some areas of the code where iteration 
+  was done over the set of all params: 
+    - literal.* in ExtractingRequestHandler
+    - facet.* in FacetComponent
+    - spellcheck.[dictionary name].* and spellcheck.collateParam.* in SpellCheckComponent
+    - olap.* in AnalyticsComponent
+  (Alexandre Rafalovitch & hossman)
+
+* SOLR-6426: SolrZkClient clean can fail due to a race with children nodes. (Mark Miller)
+
+* SOLR-6457: LBHttpSolrClient: ArrayIndexOutOfBoundsException risk if counter overflows
+  (longkey via Noble Paul)
+
+* SOLR-6481: CLUSTERSTATUS should check if the node hosting a replica is live when
+  reporting replica status (Timothy Potter)
+
+* SOLR-6631: DistributedQueue spinning on calling zookeeper getChildren()
+  (Jessica Cheng Mallet, Mark Miller, Timothy Potter)
+
+* SOLR-6579: SnapPuller Replication blocks clean shutdown of tomcat
+  (Philip Black-Knight via Noble Paul)
+
+* SOLR-6763: Shard leader elections should not persist across session expiry
+  (Alan Woodward, Mark Miller)
+  
+* SOLR-3881: Avoid OOMs in LanguageIdentifierUpdateProcessor:
+  - Added langid.maxFieldValueChars and langid.maxTotalChars params to limit
+    input, by default 10k and 20k chars, respectively.
+  - Moved input concatenation to Tika implementation; the langdetect
+    implementation instead appends each input piece via the langdetect API.
+  (Vitaliy Zhovtyuk, Tomás Fernández Löbbe, Rob Tulloh, Steve Rowe)
+
+* SOLR-6850: AutoAddReplicas makes a call to wait to see live replicas that times
+  out after 30 milliseconds instead of 30 seconds. (Varun Thacker via Mark Miller)
+
+* SOLR-6839: Direct routing with CloudSolrServer will ignore the Overwrite document option.
+  (Mark Miller)
+
+* SOLR-7139: Fix SolrContentHandler for TIKA to ignore multiple startDocument events.
+  (Chris A. Mattmann, Uwe Schindler)  
+
+* SOLR-6941: DistributedQueue#containsTaskWithRequestId can fail with NPE. (Mark Miller)
+
+* SOLR-7011: Delete collection returns before collection is actually removed.
+  (Christine Poerschke via shalin)
+
+* SOLR-6856: Restore ExtractingRequestHandler's ability to capture all HTML tags when
+  parsing (X)HTML. (hossman, Uwe Schindler, ehatcher, Steve Rowe)
+  
+* SOLR-6928: solr.cmd stop works only in english (john.work, Jan Høydahl, Timothy Potter)
+
+* SOLR-7038: Validate the presence of configset before trying to create a collection.
+  (Anshum Gupta, Mark Miller)
+
+* SOLR-7016: Fix bin\solr.cmd to work in a directory with spaces in the name.
+  (Timothy Potter, Uwe Schindler)
+    
+* SOLR-6693: bin\solr.cmd doesn't support 32-bit JRE/JDK running on Windows due to 
+  parenthesis in JAVA_HOME. (Timothy Potter, Christopher Hewitt, Jan Høydahl)
+  
+* SOLR-7067: bin/solr won't run under bash 4.2+. (Steve Rowe)
+
+* SOLR-7033, SOLR-5961: RecoveryStrategy should not publish any state when
+  closed / cancelled and there should always be a pause between recoveries 
+  even when recoveries are rapidly stopped and started as well as when a
+  node attempts to become the leader for a shard. 
+  (Mark Miller, Maxim Novikov)
+  
+* SOLR-6847: LeaderInitiatedRecoveryThread compares wrong replica's state with lirState.
+  (shalin)
+
+* SOLR-7128: Two phase distributed search is fetching extra fields in GET_TOP_IDS phase.
+  (Pablo Queixalos, shalin)
+
+Other Changes
+----------------------
+
+* SOLR-7147: Introduce new TrackingShardHandlerFactory for monitoring what requests
+  are sent to shards during tests. (hossman, shalin)
+
 ==================  4.10.3 ==================
 
 Bug Fixes

Modified: lucene/dev/branches/lucene6271/solr/bin/post
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/bin/post?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/bin/post (original)
+++ lucene/dev/branches/lucene6271/solr/bin/post Tue Mar 31 05:22:40 2015
@@ -34,7 +34,7 @@ SOLR_TIP=`dirname "$THIS_SCRIPT"`/..
 SOLR_TIP=`cd "$SOLR_TIP"; pwd`
 
 if [ -n "$SOLR_JAVA_HOME" ]; then
-  JAVA=$SOLR_JAVA_HOME/bin/java
+  JAVA="$SOLR_JAVA_HOME/bin/java"
 elif [ -n "$JAVA_HOME" ]; then
   for java in "$JAVA_HOME"/bin/amd64/java "$JAVA_HOME"/bin/java; do
     if [ -x "$java" ]; then
@@ -47,12 +47,12 @@ else
 fi
 
 # test that Java exists and is executable on this server
-$JAVA -version >/dev/null 2>&1 || { echo >&2 "Java is required to run this tool! Please install Java 8 or greater before running this script."; exit 1; }
+"$JAVA" -version >/dev/null 2>&1 || { echo >&2 "Java is required to run this tool! Please install Java 8 or greater before running this script."; exit 1; }
 
 
 # ===== post specific code
 
-TOOL_JAR=$SOLR_TIP/dist/solr-core-*.jar
+TOOL_JAR=("$SOLR_TIP/dist"/solr-core-*.jar)
 
 function print_usage() {
   echo ""
@@ -104,8 +104,8 @@ if [[ $# -eq 1 && ("$1" == "-help" || "$
 fi
 
 
-COLLECTION=$DEFAULT_SOLR_COLLECTION
-PROPS="-Dauto=yes"
+COLLECTION="$DEFAULT_SOLR_COLLECTION"
+PROPS=('-Dauto=yes')
 RECURSIVE=""
 FILES=()
 URLS=()
@@ -118,7 +118,7 @@ while [ $# -gt 0 ]; do
   if [[ -d "$1" ]]; then
     # Directory
 #    echo "$1: DIRECTORY"
-    RECURSIVE="-Drecursive=yes"
+    RECURSIVE=yes
     FILES+=("$1")
   elif [[ -f "$1" ]]; then
     # File
@@ -129,12 +129,12 @@ while [ $# -gt 0 ]; do
 #    echo "$1: URL"
     URLS+=("$1")
   else
-    if [[ $1 == -* ]]; then
-      if [[ $1 == "-c" ]]; then
+    if [[ "$1" == -* ]]; then
+      if [[ "$1" == "-c" ]]; then
         # Special case, pull out collection name
         shift
-        COLLECTION=$1
-      elif [[ ($1 == "-d" || $1 == "--data" || $1 == "-") ]]; then
+        COLLECTION="$1"
+      elif [[ ("$1" == "-d" || "$1" == "--data" || "$1" == "-") ]]; then
         if [[ -s /dev/stdin ]]; then
           MODE="stdin"
         else
@@ -148,10 +148,10 @@ while [ $# -gt 0 ]; do
           fi
         fi
       else
-        key=${1:1}
+        key="${1:1}"
         shift
 #       echo "$1: PROP"
-        PROPS="$PROPS -D$key=$1"
+        PROPS+=("-D$key=$1")
       fi
     else
       echo -e "\nUnrecognized argument: $1\n"
@@ -208,10 +208,13 @@ else
   PARAMS=("${ARGS[@]}")
 fi
 
-PROPS="$PROPS -Dc=$COLLECTION -Ddata=$MODE $RECURSIVE"
+PROPS+=("-Dc=$COLLECTION" "-Ddata=$MODE")
+if [[ -n "$RECURSIVE" ]]; then
+  PROPS+=('-Drecursive=yes')
+fi
 
-echo "$JAVA" -classpath $TOOL_JAR $PROPS org.apache.solr.util.SimplePostTool "${PARAMS[@]}"
-"$JAVA" -classpath $TOOL_JAR $PROPS org.apache.solr.util.SimplePostTool "${PARAMS[@]}"
+echo "$JAVA" -classpath "${TOOL_JAR[0]}" "${PROPS[@]}" org.apache.solr.util.SimplePostTool "${PARAMS[@]}"
+"$JAVA" -classpath "${TOOL_JAR[0]}" "${PROPS[@]}" org.apache.solr.util.SimplePostTool "${PARAMS[@]}"
 
 # post smoker:
 # bin/post -c signals -out yes -type application/json -d '[{"id": 2, "val": 0.47}]'