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 2015/09/02 15:06:22 UTC

svn commit: r1700800 [4/24] - in /lucene/dev/branches/lucene6699: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/scripts/ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/common/src/java/org/apache/lucene/analysis/ar/ ...

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java Wed Sep  2 13:06:13 2015
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.CollectionUtil;
+import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.InfoStream;
 
@@ -747,8 +748,9 @@ final class IndexFileDeleter implements
     } catch (IOException e) {  // if delete fails
 
       // IndexWriter should only ask us to delete files it knows it wrote, so if we hit this, something is wrong!
-      assert e instanceof NoSuchFileException == false: "hit unexpected NoSuchFileException: file=" + fileName;
-      assert e instanceof FileNotFoundException == false: "hit unexpected FileNotFoundException: file=" + fileName;
+      // LUCENE-6684: we suppress this assert for Windows, since a file could be in a confusing "pending delete" state:
+      assert Constants.WINDOWS || e instanceof NoSuchFileException == false: "hit unexpected NoSuchFileException: file=" + fileName;
+      assert Constants.WINDOWS || e instanceof FileNotFoundException == false: "hit unexpected FileNotFoundException: file=" + fileName;
 
       // Some operating systems (e.g. Windows) don't
       // permit a file to be deleted while it is opened

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java Wed Sep  2 13:06:13 2015
@@ -260,7 +260,6 @@ public final class IndexWriterConfig ext
 
   /** Expert: Sets the {@link DocumentsWriterPerThreadPool} instance used by the
    * IndexWriter to assign thread-states to incoming indexing threads.
-   * </p>
    * <p>
    * NOTE: The given {@link DocumentsWriterPerThreadPool} instance must not be used with
    * other {@link IndexWriter} instances once it has been initialized / associated with an
@@ -301,22 +300,6 @@ public final class IndexWriterConfig ext
     return readerPooling;
   }
 
-  /** Expert: sets the {@link DocConsumer} chain to be used to process documents.
-   *
-   * <p>Only takes effect when IndexWriter is first created. */
-  IndexWriterConfig setIndexingChain(IndexingChain indexingChain) {
-    if (indexingChain == null) {
-      throw new IllegalArgumentException("indexingChain must not be null");
-    }
-    this.indexingChain = indexingChain;
-    return this;
-  }
-
-  @Override
-  IndexingChain getIndexingChain() {
-    return indexingChain;
-  }
-
   /**
    * Expert: Controls when segments are flushed to disk during indexing.
    * The {@link FlushPolicy} initialized during {@link IndexWriter} instantiation and once initialized

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java Wed Sep  2 13:06:13 2015
@@ -380,8 +380,7 @@ public class LiveIndexWriterConfig {
   }
 
   /**
-   * Returns the indexing chain set on
-   * {@link IndexWriterConfig#setIndexingChain(IndexingChain)}.
+   * Returns the indexing chain.
    */
   IndexingChain getIndexingChain() {
     return indexingChain;

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/index/QueryTimeout.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/index/QueryTimeout.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/index/QueryTimeout.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/index/QueryTimeout.java Wed Sep  2 13:06:13 2015
@@ -29,5 +29,10 @@ public interface QueryTimeout {
    */
   public abstract boolean shouldExit();
 
+  /** Returns true if timeouts are enabled for this query (i.e. if shouldExit would ever return true) */
+  public default boolean isTimeoutEnabled() {
+    return true;
+  }
+
 }
 

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java Wed Sep  2 13:06:13 2015
@@ -1,6 +1,6 @@
 package org.apache.lucene.search;
 
-/**
+/*
  * Copyright 2004 The Apache Software Foundation
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
@@ -123,7 +123,7 @@ public class DisjunctionMaxQuery extends
     public DisjunctionMaxWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
       super(DisjunctionMaxQuery.this);
       for (Query disjunctQuery : disjuncts) {
-        weights.add(disjunctQuery.createWeight(searcher, needsScores));
+        weights.add(searcher.createWeight(disjunctQuery, needsScores));
       }
       this.needsScores = needsScores;
     }

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java Wed Sep  2 13:06:13 2015
@@ -31,15 +31,40 @@ public final class MatchAllDocsQuery ext
 
   @Override
   public Weight createWeight(IndexSearcher searcher, boolean needsScores) {
-    return new RandomAccessWeight(this) {
-      @Override
-      protected Bits getMatchingDocs(LeafReaderContext context) throws IOException {
-        return new Bits.MatchAllBits(context.reader().maxDoc());
-      }
+    return new ConstantScoreWeight(this) {
       @Override
       public String toString() {
         return "weight(" + MatchAllDocsQuery.this + ")";
       }
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        return new ConstantScoreScorer(this, score(), DocIdSetIterator.all(context.reader().maxDoc()));
+      }
+      @Override
+      public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
+        final float score = score();
+        final int maxDoc = context.reader().maxDoc();
+        return new BulkScorer() {
+          @Override
+          public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
+            max = Math.min(max, maxDoc);
+            FakeScorer scorer = new FakeScorer();
+            scorer.score = score;
+            collector.setScorer(scorer);
+            for (int doc = min; doc < max; ++doc) {
+              scorer.doc = doc;
+              if (acceptDocs == null || acceptDocs.get(doc)) {
+                collector.collect(doc);
+              }
+            }
+            return max == maxDoc ? DocIdSetIterator.NO_MORE_DOCS : max;
+          }
+          @Override
+          public long cost() {
+            return maxDoc;
+          }
+        };
+      }
     };
   }
 

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/TermScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/TermScorer.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/TermScorer.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/TermScorer.java Wed Sep  2 13:06:13 2015
@@ -36,7 +36,7 @@ final class TermScorer extends Scorer {
    * @param td
    *          An iterator over the documents matching the <code>Term</code>.
    * @param docScorer
-   *          The </code>Similarity.SimScorer</code> implementation
+   *          The <code>Similarity.SimScorer</code> implementation
    *          to be used for score computations.
    */
   TermScorer(Weight weight, PostingsEnum td, Similarity.SimScorer docScorer) {

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/UsageTrackingQueryCachingPolicy.java Wed Sep  2 13:06:13 2015
@@ -123,6 +123,24 @@ public final class UsageTrackingQueryCac
 
   @Override
   public boolean shouldCache(Query query, LeafReaderContext context) throws IOException {
+    if (query instanceof MatchAllDocsQuery
+        // MatchNoDocsQuery currently rewrites to a BooleanQuery,
+        // but who knows, it might get its own Weight one day
+        || query instanceof MatchNoDocsQuery) {
+      return false;
+    }
+    if (query instanceof BooleanQuery) {
+      BooleanQuery bq = (BooleanQuery) query;
+      if (bq.clauses().isEmpty()) {
+        return false;
+      }
+    }
+    if (query instanceof DisjunctionMaxQuery) {
+      DisjunctionMaxQuery dmq = (DisjunctionMaxQuery) query;
+      if (dmq.getDisjuncts().isEmpty()) {
+        return false;
+      }
+    }
     if (segmentPolicy.shouldCache(query, context) == false) {
       return false;
     }

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java Wed Sep  2 13:06:13 2015
@@ -63,9 +63,9 @@ public class BM25Similarity extends Simi
     this.b  = 0.75f;
   }
   
-  /** Implemented as <code>log(1 + (numDocs - docFreq + 0.5)/(docFreq + 0.5))</code>. */
-  protected float idf(long docFreq, long numDocs) {
-    return (float) Math.log(1 + (numDocs - docFreq + 0.5D)/(docFreq + 0.5D));
+  /** Implemented as <code>log(1 + (docCount - docFreq + 0.5)/(docFreq + 0.5))</code>. */
+  protected float idf(long docFreq, long docCount) {
+    return (float) Math.log(1 + (docCount - docFreq + 0.5D)/(docFreq + 0.5D));
   }
   
   /** Implemented as <code>1 / (distance + 1)</code>. */
@@ -78,7 +78,7 @@ public class BM25Similarity extends Simi
     return 1;
   }
   
-  /** The default implementation computes the average as <code>sumTotalTermFreq / maxDoc</code>,
+  /** The default implementation computes the average as <code>sumTotalTermFreq / docCount</code>,
    * or returns <code>1</code> if the index does not store sumTotalTermFreq:
    * any field that omits frequency information). */
   protected float avgFieldLength(CollectionStatistics collectionStats) {
@@ -86,7 +86,8 @@ public class BM25Similarity extends Simi
     if (sumTotalTermFreq <= 0) {
       return 1f;       // field does not exist, or stat is unsupported
     } else {
-      return (float) (sumTotalTermFreq / (double) collectionStats.maxDoc());
+      final long docCount = collectionStats.docCount() == -1 ? collectionStats.maxDoc() : collectionStats.docCount();
+      return (float) (sumTotalTermFreq / (double) docCount);
     }
   }
   
@@ -150,14 +151,14 @@ public class BM25Similarity extends Simi
    * The default implementation uses:
    * 
    * <pre class="prettyprint">
-   * idf(docFreq, searcher.maxDoc());
+   * idf(docFreq, docCount);
    * </pre>
    * 
-   * Note that {@link CollectionStatistics#maxDoc()} is used instead of
+   * Note that {@link CollectionStatistics#docCount()} is used instead of
    * {@link org.apache.lucene.index.IndexReader#numDocs() IndexReader#numDocs()} because also 
    * {@link TermStatistics#docFreq()} is used, and when the latter 
-   * is inaccurate, so is {@link CollectionStatistics#maxDoc()}, and in the same direction.
-   * In addition, {@link CollectionStatistics#maxDoc()} is more efficient to compute
+   * is inaccurate, so is {@link CollectionStatistics#docCount()}, and in the same direction.
+   * In addition, {@link CollectionStatistics#docCount()} does not skew when fields are sparse.
    *   
    * @param collectionStats collection-level statistics
    * @param termStats term-level statistics for the term
@@ -166,9 +167,9 @@ public class BM25Similarity extends Simi
    */
   public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics termStats) {
     final long df = termStats.docFreq();
-    final long max = collectionStats.maxDoc();
-    final float idf = idf(df, max);
-    return Explanation.match(idf, "idf(docFreq=" + df + ", maxDocs=" + max + ")");
+    final long docCount = collectionStats.docCount() == -1 ? collectionStats.maxDoc() : collectionStats.docCount();
+    final float idf = idf(df, docCount);
+    return Explanation.match(idf, "idf(docFreq=" + df + ", docCount=" + docCount + ")");
   }
 
   /**
@@ -185,13 +186,13 @@ public class BM25Similarity extends Simi
    *         for each term.
    */
   public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics termStats[]) {
-    final long max = collectionStats.maxDoc();
+    final long docCount = collectionStats.docCount() == -1 ? collectionStats.maxDoc() : collectionStats.docCount();
     float idf = 0.0f;
     List<Explanation> details = new ArrayList<>();
     for (final TermStatistics stat : termStats ) {
       final long df = stat.docFreq();
-      final float termIdf = idf(df, max);
-      details.add(Explanation.match(termIdf, "idf(docFreq=" + df + ", maxDocs=" + max + ")"));
+      final float termIdf = idf(df, docCount);
+      details.add(Explanation.match(termIdf, "idf(docFreq=" + df + ", docCount=" + docCount + ")"));
       idf += termIdf;
     }
     return Explanation.match(idf, "idf(), sum of:", details);

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/similarities/DefaultSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/similarities/DefaultSimilarity.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/similarities/DefaultSimilarity.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/similarities/DefaultSimilarity.java Wed Sep  2 13:06:13 2015
@@ -133,10 +133,10 @@ public class DefaultSimilarity extends T
     return 1;
   }
 
-  /** Implemented as <code>log(numDocs/(docFreq+1)) + 1</code>. */
+  /** Implemented as <code>log(docCount/(docFreq+1)) + 1</code>. */
   @Override
-  public float idf(long docFreq, long numDocs) {
-    return (float)(Math.log(numDocs/(double)(docFreq+1)) + 1.0);
+  public float idf(long docFreq, long docCount) {
+    return (float)(Math.log(docCount/(double)(docFreq+1)) + 1.0);
   }
     
   /** 

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationH1.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationH1.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationH1.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/similarities/NormalizationH1.java Wed Sep  2 13:06:13 2015
@@ -48,7 +48,7 @@ public class NormalizationH1 extends Nor
   
   @Override
   public final float tfn(BasicStats stats, float tf, float len) {
-    return tf * stats.getAvgFieldLength() / len;
+    return tf * c * stats.getAvgFieldLength() / len;
   }
 
   @Override

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java Wed Sep  2 13:06:13 2015
@@ -102,7 +102,7 @@ public abstract class SimilarityBase ext
   protected void fillBasicStats(BasicStats stats, CollectionStatistics collectionStats, TermStatistics termStats) {
     // #positions(field) must be >= #positions(term)
     assert collectionStats.sumTotalTermFreq() == -1 || collectionStats.sumTotalTermFreq() >= termStats.totalTermFreq();
-    long numberOfDocuments = collectionStats.maxDoc();
+    long numberOfDocuments = collectionStats.docCount() == -1 ? collectionStats.maxDoc() : collectionStats.docCount();
     
     long docFreq = termStats.docFreq();
     long totalTermFreq = termStats.totalTermFreq();

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java Wed Sep  2 13:06:13 2015
@@ -346,7 +346,7 @@ import org.apache.lucene.util.BytesRef;
  *          </td>
  *          <td valign="middle" align="center">
  *            <table summary="inverse document frequency computation">
- *               <tr><td align="center" style="text-align: center"><small>numDocs</small></td></tr>
+ *               <tr><td align="center" style="text-align: center"><small>docCount</small></td></tr>
  *               <tr><td align="center" style="text-align: center">&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;&ndash;</td></tr>
  *               <tr><td align="center" style="text-align: center"><small>docFreq+1</small></td></tr>
  *            </table>
@@ -566,14 +566,14 @@ public abstract class TFIDFSimilarity ex
    * The default implementation uses:
    * 
    * <pre class="prettyprint">
-   * idf(docFreq, searcher.maxDoc());
+   * idf(docFreq, docCount);
    * </pre>
    * 
-   * Note that {@link CollectionStatistics#maxDoc()} is used instead of
+   * Note that {@link CollectionStatistics#docCount()} is used instead of
    * {@link org.apache.lucene.index.IndexReader#numDocs() IndexReader#numDocs()} because also 
    * {@link TermStatistics#docFreq()} is used, and when the latter 
-   * is inaccurate, so is {@link CollectionStatistics#maxDoc()}, and in the same direction.
-   * In addition, {@link CollectionStatistics#maxDoc()} is more efficient to compute
+   * is inaccurate, so is {@link CollectionStatistics#docCount()}, and in the same direction.
+   * In addition, {@link CollectionStatistics#docCount()} does not skew when fields are sparse.
    *   
    * @param collectionStats collection-level statistics
    * @param termStats term-level statistics for the term
@@ -582,9 +582,9 @@ public abstract class TFIDFSimilarity ex
    */
   public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics termStats) {
     final long df = termStats.docFreq();
-    final long max = collectionStats.maxDoc();
-    final float idf = idf(df, max);
-    return Explanation.match(idf, "idf(docFreq=" + df + ", maxDocs=" + max + ")");
+    final long docCount = collectionStats.docCount() == -1 ? collectionStats.maxDoc() : collectionStats.docCount();
+    final float idf = idf(df, docCount);
+    return Explanation.match(idf, "idf(docFreq=" + df + ", docCount=" + docCount + ")");
   }
 
   /**
@@ -601,13 +601,13 @@ public abstract class TFIDFSimilarity ex
    *         for each term.
    */
   public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics termStats[]) {
-    final long max = collectionStats.maxDoc();
+    final long docCount = collectionStats.docCount() == -1 ? collectionStats.maxDoc() : collectionStats.docCount();
     float idf = 0.0f;
     List<Explanation> subs = new ArrayList<>();
     for (final TermStatistics stat : termStats ) {
       final long df = stat.docFreq();
-      final float termIdf = idf(df, max);
-      subs.add(Explanation.match(termIdf, "idf(docFreq=" + df + ", maxDocs=" + max + ")"));
+      final float termIdf = idf(df, docCount);
+      subs.add(Explanation.match(termIdf, "idf(docFreq=" + df + ", docCount=" + docCount + ")"));
       idf += termIdf;
     }
     return Explanation.match(idf, "idf(), sum of:", subs);
@@ -623,10 +623,10 @@ public abstract class TFIDFSimilarity ex
    * and smaller values for common terms.
    *
    * @param docFreq the number of documents which contain the term
-   * @param numDocs the total number of documents in the collection
+   * @param docCount the total number of documents in the collection
    * @return a score factor based on the term's document frequency
    */
-  public abstract float idf(long docFreq, long numDocs);
+  public abstract float idf(long docFreq, long docCount);
 
   /**
    * Compute an index-time normalization value for this field instance.

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java Wed Sep  2 13:06:13 2015
@@ -1,5 +1,5 @@
 package org.apache.lucene.search.spans;
-/**
+/*
  * Copyright 2005 The Apache Software Foundation
  *
  * Licensed under the Apache License, Version 2.0 (the "License");

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/store/ChecksumIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/store/ChecksumIndexInput.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/store/ChecksumIndexInput.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/store/ChecksumIndexInput.java Wed Sep  2 13:06:13 2015
@@ -44,9 +44,10 @@ public abstract class ChecksumIndexInput
    */
   @Override
   public void seek(long pos) throws IOException {
-    final long skip = pos - getFilePointer();
+    final long curFP = getFilePointer();
+    final long skip = pos - curFP;
     if (skip < 0) {
-      throw new IllegalStateException(getClass() + " cannot seek backwards");
+      throw new IllegalStateException(getClass() + " cannot seek backwards (pos=" + pos + " getFilePointer()=" + curFP + ")");
     }
     skipBytes(skip);
   }

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/store/IndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/store/IndexInput.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/store/IndexInput.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/store/IndexInput.java Wed Sep  2 13:06:13 2015
@@ -83,6 +83,10 @@ public abstract class IndexInput extends
    * 
    * <p>If you access the cloned IndexInput after closing the original object,
    * any <code>readXXX</code> methods will throw {@link AlreadyClosedException}.
+   *
+   * <p>This method is NOT thread safe, so if the current {@code IndexInput}
+   * is being used by one thread while {@code clone} is called by another,
+   * disaster could strike.
    */
   @Override
   public IndexInput clone() {

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java Wed Sep  2 13:06:13 2015
@@ -167,7 +167,9 @@ public class MMapDirectory extends FSDir
   
   private static boolean checkUnmapSupported() {
     try {
-      Class.forName("java.nio.DirectByteBuffer").getMethod("cleaner");
+      Class<?> clazz = Class.forName("java.nio.DirectByteBuffer");
+      Method method = clazz.getMethod("cleaner");
+      method.setAccessible(true);
       return true;
     } catch (Exception e) {
       return false;

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/store/RAMInputStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/store/RAMInputStream.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/store/RAMInputStream.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/store/RAMInputStream.java Wed Sep  2 13:06:13 2015
@@ -116,7 +116,7 @@ public class RAMInputStream extends Inde
 
   @Override
   public void seek(long pos) throws IOException {
-    if (currentBuffer==null || pos < bufferStart || pos >= bufferStart + BUFFER_SIZE) {
+    if (currentBuffer == null || pos < bufferStart || pos >= bufferStart + BUFFER_SIZE) {
       currentBufferIndex = (int) (pos / BUFFER_SIZE);
       switchCurrentBuffer(false);
     }

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/util/FilterIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/util/FilterIterator.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/util/FilterIterator.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/util/FilterIterator.java Wed Sep  2 13:06:13 2015
@@ -1,6 +1,6 @@
 package org.apache.lucene.util;
 
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements. See the NOTICE file distributed with this
  * work for additional information regarding copyright ownership. The ASF

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/util/SmallFloat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/util/SmallFloat.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/util/SmallFloat.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/util/SmallFloat.java Wed Sep  2 13:06:13 2015
@@ -1,5 +1,5 @@
 package org.apache.lucene.util;
-/**
+/*
  * Copyright 2005 The Apache Software Foundation
  *
  * Licensed under the Apache License, Version 2.0 (the "License");

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/util/fst/FST.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/util/fst/FST.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/util/fst/FST.java Wed Sep  2 13:06:13 2015
@@ -43,7 +43,6 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.PriorityQueue;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.fst.Builder.UnCompiledNode;
 import org.apache.lucene.util.packed.GrowableWriter;
 import org.apache.lucene.util.packed.PackedInts;
 
@@ -104,17 +103,17 @@ public final class FST<T> implements Acc
   private static final byte ARCS_AS_FIXED_ARRAY = BIT_ARC_HAS_FINAL_OUTPUT;
 
   /**
-   * @see #shouldExpand(Builder, UnCompiledNode)
+   * @see #shouldExpand(Builder, Builder.UnCompiledNode)
    */
   static final int FIXED_ARRAY_SHALLOW_DISTANCE = 3; // 0 => only root node.
 
   /**
-   * @see #shouldExpand(Builder, UnCompiledNode)
+   * @see #shouldExpand(Builder, Builder.UnCompiledNode)
    */
   static final int FIXED_ARRAY_NUM_ARCS_SHALLOW = 5;
 
   /**
-   * @see #shouldExpand(Builder, UnCompiledNode)
+   * @see #shouldExpand(Builder, Builder.UnCompiledNode)
    */
   static final int FIXED_ARRAY_NUM_ARCS_DEEP = 10;
 
@@ -1332,7 +1331,7 @@ public final class FST<T> implements Acc
    * @see #FIXED_ARRAY_NUM_ARCS_DEEP
    * @see Builder.UnCompiledNode#depth
    */
-  private boolean shouldExpand(Builder<T> builder, UnCompiledNode<T> node) {
+  private boolean shouldExpand(Builder<T> builder, Builder.UnCompiledNode<T> node) {
     return builder.allowArrayArcs &&
       ((node.depth <= FIXED_ARRAY_SHALLOW_DISTANCE && node.numArcs >= FIXED_ARRAY_NUM_ARCS_SHALLOW) || 
        node.numArcs >= FIXED_ARRAY_NUM_ARCS_DEEP);

Modified: lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java Wed Sep  2 13:06:13 2015
@@ -26,11 +26,11 @@ import org.apache.lucene.util.RamUsageEs
 /**
  * Space optimized random access capable array of values with a fixed number of
  * bits/value. Values are packed contiguously.
- * </p><p>
+ * <p>
  * The implementation strives to perform as fast as possible under the
  * constraint of contiguous bits, by avoiding expensive operations. This comes
  * at the cost of code clarity.
- * </p><p>
+ * <p>
  * Technical details: This implementation is a refinement of a non-branching
  * version. The non-branching get and set methods meant that 2 or 4 atomics in
  * the underlying array were always accessed, even for the cases where only

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/TestMergeSchedulerExternal.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/TestMergeSchedulerExternal.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/TestMergeSchedulerExternal.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/TestMergeSchedulerExternal.java Wed Sep  2 13:06:13 2015
@@ -67,7 +67,7 @@ public class TestMergeSchedulerExternal
       excCalled = true;
     }
 
-    ;@Override
+    @Override
     protected void doMerge(IndexWriter writer, MergePolicy.OneMerge merge) throws IOException {
       mergeCalled = true;
       super.doMerge(writer, merge);

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java Wed Sep  2 13:06:13 2015
@@ -1,6 +1,6 @@
 package org.apache.lucene.index;
 
-/**
+/*
  * Copyright 2004 The Apache Software Foundation
  *
  * Licensed under the Apache License, Version 2.0 (the "License");

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestByteSlices.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestByteSlices.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestByteSlices.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestByteSlices.java Wed Sep  2 13:06:13 2015
@@ -1,6 +1,6 @@
 package org.apache.lucene.index;
 
-/**
+/*
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
  * You may obtain a copy of the License at

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java Wed Sep  2 13:06:13 2015
@@ -245,14 +245,6 @@ public class TestIndexWriterConfig exten
 
     // Test IndexingChain
     assertTrue(DocumentsWriterPerThread.defaultIndexingChain == conf.getIndexingChain());
-    conf.setIndexingChain(new MyIndexingChain());
-    assertEquals(MyIndexingChain.class, conf.getIndexingChain().getClass());
-    try {
-      conf.setIndexingChain(null);
-      fail();
-    } catch (IllegalArgumentException e) {
-      // ok
-    }
 
     try {
       conf.setMaxBufferedDeleteTerms(0);

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMerging.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMerging.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMerging.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMerging.java Wed Sep  2 13:06:13 2015
@@ -1,5 +1,5 @@
 package org.apache.lucene.index;
-/**
+/*
  * Copyright 2006 The Apache Software Foundation
  *
  * Licensed under the Apache License, Version 2.0 (the "License");

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java Wed Sep  2 13:06:13 2015
@@ -1,6 +1,6 @@
 package org.apache.lucene.index;
 
-/**
+/*
  *  Licensed to the Apache Software Foundation (ASF) under one or more
  *  contributor license agreements.  See the NOTICE file distributed with
  *  this work for additional information regarding copyright ownership.

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java Wed Sep  2 13:06:13 2015
@@ -1,6 +1,6 @@
 package org.apache.lucene.index;
 
-/**
+/*
  * Copyright 2004 The Apache Software Foundation
  *
  * Licensed under the Apache License, Version 2.0 (the "License");

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java Wed Sep  2 13:06:13 2015
@@ -118,7 +118,7 @@ public class TestMaxTermFrequency extend
     @Override public float coord(int overlap, int maxOverlap) { return 0; }
     @Override public float queryNorm(float sumOfSquaredWeights) { return 0; }
     @Override public float tf(float freq) { return 0; }
-    @Override public float idf(long docFreq, long numDocs) { return 0; }
+    @Override public float idf(long docFreq, long docCount) { return 0; }
     @Override public float sloppyFreq(int distance) { return 0; }
     @Override public float scorePayload(int doc, int start, int end, BytesRef payload) { return 0; }
   }

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestNorms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestNorms.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestNorms.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestNorms.java Wed Sep  2 13:06:13 2015
@@ -67,7 +67,7 @@ public class TestNorms extends LuceneTes
     @Override public float coord(int overlap, int maxOverlap) { return 0; }
     @Override public float queryNorm(float sumOfSquaredWeights) { return 0; }
     @Override public float tf(float freq) { return 0; }
-    @Override public float idf(long docFreq, long numDocs) { return 0; }
+    @Override public float idf(long docFreq, long docCount) { return 0; }
     @Override public float sloppyFreq(int distance) { return 0; }
     @Override public float scorePayload(int doc, int start, int end, BytesRef payload) { return 0; }
   }

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java Wed Sep  2 13:06:13 2015
@@ -54,7 +54,7 @@ public class TestOmitTf extends LuceneTe
     @Override public float lengthNorm(FieldInvertState state) { return state.getBoost(); }
     @Override public float tf(float freq) { return freq; }
     @Override public float sloppyFreq(int distance) { return 2.0f; }
-    @Override public float idf(long docFreq, long numDocs) { return 1.0f; }
+    @Override public float idf(long docFreq, long docCount) { return 1.0f; }
     @Override public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics[] termStats) {
       return Explanation.match(1.0f, "Inexplicable");
     }

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestStressDeletes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestStressDeletes.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestStressDeletes.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestStressDeletes.java Wed Sep  2 13:06:13 2015
@@ -1,6 +1,6 @@
 package org.apache.lucene.index;
 
-/**
+/*
  * Copyright 2004 The Apache Software Foundation
  *
  * Licensed under the Apache License, Version 2.0 (the "License");

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing.java Wed Sep  2 13:06:13 2015
@@ -1,6 +1,6 @@
 package org.apache.lucene.index;
 
-/**
+/*
  * Copyright 2004 The Apache Software Foundation
  *
  * Licensed under the Apache License, Version 2.0 (the "License");

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java Wed Sep  2 13:06:13 2015
@@ -1,6 +1,6 @@
 package org.apache.lucene.index;
 
-/**
+/*
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
  * You may obtain a copy of the License at

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java Wed Sep  2 13:06:13 2015
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Random;
 
 import org.apache.lucene.analysis.*;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
@@ -73,12 +74,13 @@ public class TestTermVectorsReader exten
 
     Arrays.sort(testTerms);
     int tokenUpto = 0;
+    Random rnd = random();
     for (int i = 0; i < testTerms.length; i++) {
       positions[i] = new int[TERM_FREQ];
       // first position must be 0
       for (int j = 0; j < TERM_FREQ; j++) {
         // positions are always sorted in increasing order
-        positions[i][j] = (int) (j * 10 + Math.random() * 10);
+        positions[i][j] = (int) (j * 10 + rnd.nextDouble() * 10);
         TestToken token = tokens[tokenUpto++] = new TestToken();
         token.text = testTerms[i];
         token.pos = positions[i][j];

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestTermdocPerf.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestTermdocPerf.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestTermdocPerf.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/index/TestTermdocPerf.java Wed Sep  2 13:06:13 2015
@@ -1,6 +1,6 @@
 package org.apache.lucene.index;
 
-/**
+/*
  * Copyright 2006 The Apache Software Foundation
  *
  * Licensed under the Apache License, Version 2.0 (the "License");

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/TestCustomSearcherSort.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/TestCustomSearcherSort.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/TestCustomSearcherSort.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/TestCustomSearcherSort.java Wed Sep  2 13:06:13 2015
@@ -1,6 +1,6 @@
 package org.apache.lucene.search;
 
-/**
+/*
  * Copyright 2005 The Apache Software Foundation
  *
  * Licensed under the Apache License, Version 2.0 (the "License");

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java Wed Sep  2 13:06:13 2015
@@ -80,7 +80,7 @@ public class TestDisjunctionMaxQuery ext
     }
     
     @Override
-    public float idf(long docFreq, long numDocs) {
+    public float idf(long docFreq, long docCount) {
       return 1.0f;
     }
   }

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/TestSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/TestSimilarity.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/TestSimilarity.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/TestSimilarity.java Wed Sep  2 13:06:13 2015
@@ -46,7 +46,7 @@ public class TestSimilarity extends Luce
     @Override public float lengthNorm(FieldInvertState state) { return state.getBoost(); }
     @Override public float tf(float freq) { return freq; }
     @Override public float sloppyFreq(int distance) { return 2.0f; }
-    @Override public float idf(long docFreq, long numDocs) { return 1.0f; }
+    @Override public float idf(long docFreq, long docCount) { return 1.0f; }
     @Override public Explanation idfExplain(CollectionStatistics collectionStats, TermStatistics[] stats) {
       return Explanation.match(1.0f, "Inexplicable"); 
     }

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/TestSimilarityProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/TestSimilarityProvider.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/TestSimilarityProvider.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/TestSimilarityProvider.java Wed Sep  2 13:06:13 2015
@@ -142,7 +142,7 @@ public class TestSimilarityProvider exte
     }
 
     @Override
-    public float idf(long docFreq, long numDocs) {
+    public float idf(long docFreq, long docCount) {
       return 1f;
     }
 
@@ -190,7 +190,7 @@ public class TestSimilarityProvider exte
     }
 
     @Override
-    public float idf(long docFreq, long numDocs) {
+    public float idf(long docFreq, long docCount) {
       return 10f;
     }
 

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java Wed Sep  2 13:06:13 2015
@@ -17,6 +17,8 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import org.apache.lucene.index.MultiReader;
+import org.apache.lucene.index.SlowCompositeReaderWrapper;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.util.LuceneTestCase;
 
@@ -41,4 +43,13 @@ public class TestUsageTrackingFilterCach
     assertEquals(2, policy.frequency(q3));
   }
 
+  public void testNeverCacheMatchAll() throws Exception {
+    Query q = new MatchAllDocsQuery();
+    UsageTrackingQueryCachingPolicy policy = new UsageTrackingQueryCachingPolicy();
+    for (int i = 0; i < 1000; ++i) {
+      policy.onUse(q);
+    }
+    assertFalse(policy.shouldCache(q, SlowCompositeReaderWrapper.wrap(new MultiReader()).getContext()));
+  }
+
 }

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadSpans.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadSpans.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadSpans.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadSpans.java Wed Sep  2 13:06:13 2015
@@ -1,6 +1,6 @@
 package org.apache.lucene.search.payloads;
 
-/**
+/*
  * Copyright 2004 The Apache Software Foundation
  *
  * Licensed under the Apache License, Version 2.0 (the "License");

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadTermQuery.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadTermQuery.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadTermQuery.java Wed Sep  2 13:06:13 2015
@@ -288,7 +288,7 @@ public class TestPayloadTermQuery extend
     }
 
     @Override
-    public float idf(long docFreq, long numDocs) {
+    public float idf(long docFreq, long docCount) {
       return 1;
     }
 

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/similarities/TestSimilarity2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/similarities/TestSimilarity2.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/similarities/TestSimilarity2.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/search/similarities/TestSimilarity2.java Wed Sep  2 13:06:13 2015
@@ -26,10 +26,12 @@ import org.apache.lucene.document.FieldT
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
@@ -38,6 +40,7 @@ import org.apache.lucene.search.spans.Sp
 import org.apache.lucene.search.spans.SpanTermQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
 
 /**
  * Tests against all the similarities we have
@@ -159,6 +162,52 @@ public class TestSimilarity2 extends Luc
     ir.close();
     dir.close();
   }
+  
+  /** make sure scores are not skewed by docs not containing the field */
+  public void testNoFieldSkew() throws Exception {
+    Directory dir = newDirectory();
+    // an evil merge policy could reorder our docs for no reason
+    IndexWriterConfig iwConfig = newIndexWriterConfig().setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwConfig);
+    Document doc = new Document();
+    doc.add(newTextField("foo", "bar baz somethingelse", Field.Store.NO));
+    iw.addDocument(doc);
+    IndexReader ir = iw.getReader();
+    IndexSearcher is = newSearcher(ir);
+    
+    BooleanQuery.Builder queryBuilder = new BooleanQuery.Builder();
+    queryBuilder.add(new TermQuery(new Term("foo", "bar")), BooleanClause.Occur.SHOULD);
+    queryBuilder.add(new TermQuery(new Term("foo", "baz")), BooleanClause.Occur.SHOULD);
+    Query query = queryBuilder.build();
+    
+    // collect scores
+    List<Explanation> scores = new ArrayList<>();
+    for (Similarity sim : sims) {
+      is.setSimilarity(sim);
+      scores.add(is.explain(query, 0));
+    }
+    ir.close();
+    
+    // add some additional docs without the field
+    int numExtraDocs = TestUtil.nextInt(random(), 1, 1000);
+    for (int i = 0; i < numExtraDocs; i++) {
+      iw.addDocument(new Document());
+    }
+    
+    // check scores are the same
+    ir = iw.getReader();
+    is = newSearcher(ir);
+    for (int i = 0; i < sims.size(); i++) {
+      is.setSimilarity(sims.get(i));
+      Explanation expected = scores.get(i);
+      Explanation actual = is.explain(query, 0);
+      assertEquals(sims.get(i).toString() + ": actual=" + actual + ",expected=" + expected, expected.getValue(), actual.getValue(), 0F);
+    }
+    
+    iw.close();
+    ir.close();
+    dir.close();
+  }
   
   /** make sure all sims work if TF is omitted */
   public void testOmitTF() throws Exception {

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/util/Test2BPagedBytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/util/Test2BPagedBytes.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/util/Test2BPagedBytes.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/util/Test2BPagedBytes.java Wed Sep  2 13:06:13 2015
@@ -1,6 +1,6 @@
 package org.apache.lucene.util;
 
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
  * this work for additional information regarding copyright ownership.

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/util/TestNumericUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/util/TestNumericUtils.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/util/TestNumericUtils.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/util/TestNumericUtils.java Wed Sep  2 13:06:13 2015
@@ -1,21 +1,21 @@
 package org.apache.lucene.util;
 
-/**
-* Licensed to the Apache Software Foundation (ASF) under one or more
-* contributor license agreements.  See the NOTICE file distributed with
-* this work for additional information regarding copyright ownership.
-* The ASF licenses this file to You under the Apache License, Version 2.0
-* (the "License"); you may not use this file except in compliance with
-* the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 
 import java.util.Arrays;
 import java.util.Collections;

Modified: lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/util/TestSmallFloat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/util/TestSmallFloat.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/util/TestSmallFloat.java (original)
+++ lucene/dev/branches/lucene6699/lucene/core/src/test/org/apache/lucene/util/TestSmallFloat.java Wed Sep  2 13:06:13 2015
@@ -1,6 +1,6 @@
 package org.apache.lucene.util;
 
-/**
+/*
  * Copyright 2005 The Apache Software Foundation
  *
  * Licensed under the Apache License, Version 2.0 (the "License");

Modified: lucene/dev/branches/lucene6699/lucene/demo/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/demo/build.xml?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/demo/build.xml (original)
+++ lucene/dev/branches/lucene6699/lucene/demo/build.xml Wed Sep  2 13:06:13 2015
@@ -47,9 +47,9 @@
       <links>
         <link href="../analyzers-common"/>
         <link href="../queryparser"/>
-      	<link href="../queries"/>
-      	<link href="../facet"/>
-      	<link href="../expressions"/>
+        <link href="../queries"/>
+        <link href="../facet"/>
+        <link href="../expressions"/>
       </links>
     </invoke-module-javadoc>
   </target>

Modified: lucene/dev/branches/lucene6699/lucene/demo/src/resources/org/apache/lucene/demo/xmlparser/WEB-INF/query.xsl
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/demo/src/resources/org/apache/lucene/demo/xmlparser/WEB-INF/query.xsl?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/demo/src/resources/org/apache/lucene/demo/xmlparser/WEB-INF/query.xsl (original)
+++ lucene/dev/branches/lucene6699/lucene/demo/src/resources/org/apache/lucene/demo/xmlparser/WEB-INF/query.xsl Wed Sep  2 13:06:13 2015
@@ -18,57 +18,57 @@
 <xsl:stylesheet version="1.0" xmlns:xsl="http://www.w3.org/1999/XSL/Transform">
 <xsl:template match="/Document">
 <BooleanQuery>
-	<!-- Clause if user selects a preference for type of job - apply choice of 
-	     permanent/contract filter and cache -->
-	<xsl:if test="type">
-	    <Clause occurs="must">
-	    	<ConstantScoreQuery>
-	    	   <CachedFilter>
-	    	     	<TermsFilter fieldName="type"><xsl:value-of select="type"/></TermsFilter>
-	    	   </CachedFilter>
-	    	 </ConstantScoreQuery>
-	   </Clause>
-	</xsl:if>
-		
-	<!-- Use standard Lucene query parser for any job description input -->
-	<xsl:if test="description">
-		<Clause occurs="must">
-			<UserQuery fieldName="description"><xsl:value-of select="description"/></UserQuery>
-		</Clause>
-	</xsl:if>      
-	
-	<!-- If any of the location fields are set OR them ALL in a Boolean filter and cache individual filters -->
-	<xsl:if test="South|North|East|West">
-		<Clause occurs="must">
-			<ConstantScoreQuery>
-				<BooleanFilter>
-					<xsl:for-each select="South|North|East|West">
-					<Clause occurs="should">
-						<CachedFilter>
-							<TermsFilter fieldName="location"><xsl:value-of select="name()"/></TermsFilter>
-						</CachedFilter>
-				  	</Clause>
-					</xsl:for-each>					
-				</BooleanFilter>
-	     		</ConstantScoreQuery>
-	 	</Clause>
-	</xsl:if>   	
-	
-	<!-- Use XSL functions to split and zero pad salary range value -->
-	<xsl:if test="salaryRange">
-		<Clause occurs="must">
-			<ConstantScoreQuery>
-				<RangeFilter fieldName="salary" >
-					<xsl:attribute name="lowerTerm">
-						<xsl:value-of select='format-number( substring-before(salaryRange,"-"), "000" )' />
-					</xsl:attribute> 
-					<xsl:attribute name="upperTerm">
-						<xsl:value-of select='format-number( substring-after(salaryRange,"-"), "000" )' />
-					</xsl:attribute> 
-				</RangeFilter>
-			</ConstantScoreQuery>
-		</Clause>
-	</xsl:if>	
+  <!-- Clause if user selects a preference for type of job - apply choice of 
+       permanent/contract filter and cache -->
+  <xsl:if test="type">
+      <Clause occurs="must">
+        <ConstantScoreQuery>
+           <CachedFilter>
+               <TermsFilter fieldName="type"><xsl:value-of select="type"/></TermsFilter>
+           </CachedFilter>
+         </ConstantScoreQuery>
+     </Clause>
+  </xsl:if>
+    
+  <!-- Use standard Lucene query parser for any job description input -->
+  <xsl:if test="description">
+    <Clause occurs="must">
+      <UserQuery fieldName="description"><xsl:value-of select="description"/></UserQuery>
+    </Clause>
+  </xsl:if>      
+  
+  <!-- If any of the location fields are set OR them ALL in a Boolean filter and cache individual filters -->
+  <xsl:if test="South|North|East|West">
+    <Clause occurs="must">
+      <ConstantScoreQuery>
+        <BooleanFilter>
+          <xsl:for-each select="South|North|East|West">
+          <Clause occurs="should">
+            <CachedFilter>
+              <TermsFilter fieldName="location"><xsl:value-of select="name()"/></TermsFilter>
+            </CachedFilter>
+            </Clause>
+          </xsl:for-each>          
+        </BooleanFilter>
+           </ConstantScoreQuery>
+     </Clause>
+  </xsl:if>     
+  
+  <!-- Use XSL functions to split and zero pad salary range value -->
+  <xsl:if test="salaryRange">
+    <Clause occurs="must">
+      <ConstantScoreQuery>
+        <RangeFilter fieldName="salary" >
+          <xsl:attribute name="lowerTerm">
+            <xsl:value-of select='format-number( substring-before(salaryRange,"-"), "000" )' />
+          </xsl:attribute> 
+          <xsl:attribute name="upperTerm">
+            <xsl:value-of select='format-number( substring-after(salaryRange,"-"), "000" )' />
+          </xsl:attribute> 
+        </RangeFilter>
+      </ConstantScoreQuery>
+    </Clause>
+  </xsl:if>  
 </BooleanQuery>
 </xsl:template>
 </xsl:stylesheet>
\ No newline at end of file

Modified: lucene/dev/branches/lucene6699/lucene/demo/src/resources/org/apache/lucene/demo/xmlparser/WEB-INF/web.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/demo/src/resources/org/apache/lucene/demo/xmlparser/WEB-INF/web.xml?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/demo/src/resources/org/apache/lucene/demo/xmlparser/WEB-INF/web.xml (original)
+++ lucene/dev/branches/lucene6699/lucene/demo/src/resources/org/apache/lucene/demo/xmlparser/WEB-INF/web.xml Wed Sep  2 13:06:13 2015
@@ -16,34 +16,34 @@
  limitations under the License.
 -->
 <web-app id="WebApp_ID" version="2.4" xmlns="http://java.sun.com/xml/ns/j2ee" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://java.sun.com/xml/ns/j2ee http://java.sun.com/xml/ns/j2ee/web-app_2_4.xsd">
-	<display-name>
-	LuceneXmlQueryWebDemo</display-name>
-	<servlet>
-		<description>
-		Servlet demonstrating XMLQueryParser</description>
-		<display-name>
-		FormBasedXmlQueryDemo</display-name>
-		<servlet-name>FormBasedXmlQueryDemo</servlet-name>
-		<servlet-class>
-		org.apache.lucene.xmlparser.webdemo.FormBasedXmlQueryDemo</servlet-class>
-		<init-param>
-			<description>
-			Name of query file held in /WEB-INF</description>
-			<param-name>xslFile</param-name>
-			<param-value>query.xsl</param-value>
-		</init-param>
-		<init-param>
-			<description>
-			Default field used in standard Lucene QueryParser used in UserQuery tag</description>
-			<param-name>defaultStandardQueryParserField</param-name>
-			<param-value>jobDescription</param-value>
-		</init-param>
-	</servlet>
-	<servlet-mapping>
-		<servlet-name>FormBasedXmlQueryDemo</servlet-name>
-		<url-pattern>/FormBasedXmlQueryDemo</url-pattern>
-	</servlet-mapping>
-	<welcome-file-list>
-		<welcome-file>index.jsp</welcome-file>
-	</welcome-file-list>
+  <display-name>
+  LuceneXmlQueryWebDemo</display-name>
+  <servlet>
+    <description>
+    Servlet demonstrating XMLQueryParser</description>
+    <display-name>
+    FormBasedXmlQueryDemo</display-name>
+    <servlet-name>FormBasedXmlQueryDemo</servlet-name>
+    <servlet-class>
+    org.apache.lucene.xmlparser.webdemo.FormBasedXmlQueryDemo</servlet-class>
+    <init-param>
+      <description>
+      Name of query file held in /WEB-INF</description>
+      <param-name>xslFile</param-name>
+      <param-value>query.xsl</param-value>
+    </init-param>
+    <init-param>
+      <description>
+      Default field used in standard Lucene QueryParser used in UserQuery tag</description>
+      <param-name>defaultStandardQueryParserField</param-name>
+      <param-value>jobDescription</param-value>
+    </init-param>
+  </servlet>
+  <servlet-mapping>
+    <servlet-name>FormBasedXmlQueryDemo</servlet-name>
+    <url-pattern>/FormBasedXmlQueryDemo</url-pattern>
+  </servlet-mapping>
+  <welcome-file-list>
+    <welcome-file>index.jsp</welcome-file>
+  </welcome-file-list>
 </web-app>

Modified: lucene/dev/branches/lucene6699/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptCompiler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptCompiler.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptCompiler.java (original)
+++ lucene/dev/branches/lucene6699/lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptCompiler.java Wed Sep  2 13:06:13 2015
@@ -222,6 +222,9 @@ public final class JavascriptCompiler {
     }
   }
 
+  /**
+   * Sends the bytecode of class file to {@link ClassWriter}.
+   */
   private void generateClass(final ParseTree parseTree, final ClassWriter classWriter, final Map<String, Integer> externalsMap) {
     classWriter.visit(CLASSFILE_VERSION,
         Opcodes.ACC_PUBLIC | Opcodes.ACC_SUPER | Opcodes.ACC_FINAL | Opcodes.ACC_SYNTHETIC,
@@ -246,9 +249,6 @@ public final class JavascriptCompiler {
     new JavascriptBaseVisitor<Void>() {
       private final Deque<Type> typeStack = new ArrayDeque<>();
 
-      /**
-       * For internal compiler use only, do NOT use
-       */
       @Override
       public Void visitCompile(JavascriptParser.CompileContext ctx) {
         typeStack.push(Type.DOUBLE_TYPE);
@@ -258,9 +258,6 @@ public final class JavascriptCompiler {
         return null;
       }
 
-      /**
-       * For internal compiler use only, do NOT use
-       */
       @Override
       public Void visitPrecedence(JavascriptParser.PrecedenceContext ctx) {
         visit(ctx.expression());
@@ -268,9 +265,6 @@ public final class JavascriptCompiler {
         return null;
       }
 
-      /**
-       * For internal compiler use only, do NOT use
-       */
       @Override
       public Void visitNumeric(JavascriptParser.NumericContext ctx) {
         if (ctx.HEX() != null) {
@@ -287,9 +281,6 @@ public final class JavascriptCompiler {
         return null;
       }
 
-      /**
-       * For internal compiler use only, do NOT use
-       */
       @Override
       public Void visitExternal(JavascriptParser.ExternalContext ctx) {
         String text = ctx.VARIABLE().getText();
@@ -342,9 +333,6 @@ public final class JavascriptCompiler {
         return null;
       }
 
-      /**
-       * For internal compiler use only, do NOT use
-       */
       @Override
       public Void visitUnary(JavascriptParser.UnaryContext ctx) {
         if (ctx.BOOLNOT() != null) {
@@ -386,9 +374,6 @@ public final class JavascriptCompiler {
         return null;
       }
 
-      /**
-       * For internal compiler use only, do NOT use
-       */
       @Override
       public Void visitMuldiv(JavascriptParser.MuldivContext ctx) {
         int opcode;
@@ -408,9 +393,6 @@ public final class JavascriptCompiler {
         return null;
       }
 
-      /**
-       * For internal compiler use only, do NOT use
-       */
       @Override
       public Void visitAddsub(JavascriptParser.AddsubContext ctx) {
         int opcode;
@@ -428,9 +410,6 @@ public final class JavascriptCompiler {
         return null;
       }
 
-      /**
-       * For internal compiler use only, do NOT use
-       */
       @Override
       public Void visitBwshift(JavascriptParser.BwshiftContext ctx) {
         int opcode;
@@ -450,9 +429,6 @@ public final class JavascriptCompiler {
         return null;
       }
 
-      /**
-       * For internal compiler use only, do NOT use
-       */
       @Override
       public Void visitBoolcomp(JavascriptParser.BoolcompContext ctx) {
         int opcode;
@@ -474,9 +450,6 @@ public final class JavascriptCompiler {
         return null;
       }
 
-      /**
-       * For internal compiler use only, do NOT use
-       */
       @Override
       public Void visitBooleqne(JavascriptParser.BooleqneContext ctx) {
         int opcode;
@@ -494,9 +467,6 @@ public final class JavascriptCompiler {
         return null;
       }
 
-      /**
-       * For internal compiler use only, do NOT use
-       */
       @Override
       public Void visitBwand(JavascriptParser.BwandContext ctx) {
         pushBitwise(Opcodes.LAND, ctx.expression(0), ctx.expression(1));
@@ -504,9 +474,6 @@ public final class JavascriptCompiler {
         return null;
       }
 
-      /**
-       * For internal compiler use only, do NOT use
-       */
       @Override
       public Void visitBwxor(JavascriptParser.BwxorContext ctx) {
         pushBitwise(Opcodes.LXOR, ctx.expression(0), ctx.expression(1));
@@ -514,9 +481,6 @@ public final class JavascriptCompiler {
         return null;
       }
 
-      /**
-       * For internal compiler use only, do NOT use
-       */
       @Override
       public Void visitBwor(JavascriptParser.BworContext ctx) {
         pushBitwise(Opcodes.LOR, ctx.expression(0), ctx.expression(1));
@@ -524,9 +488,6 @@ public final class JavascriptCompiler {
         return null;
       }
 
-      /**
-       * For internal compiler use only, do NOT use
-       */
       @Override
       public Void visitBooland(JavascriptParser.BoolandContext ctx) {
         Label andFalse = new Label();
@@ -547,9 +508,6 @@ public final class JavascriptCompiler {
         return null;
       }
 
-      /**
-       * For internal compiler use only, do NOT use
-       */
       @Override
       public Void visitBoolor(JavascriptParser.BoolorContext ctx) {
         Label orTrue = new Label();
@@ -570,9 +528,6 @@ public final class JavascriptCompiler {
         return null;
       }
 
-      /**
-       * For internal compiler use only, do NOT use
-       */
       @Override
       public Void visitConditional(JavascriptParser.ConditionalContext ctx) {
         Label condFalse = new Label();

Modified: lucene/dev/branches/lucene6699/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java (original)
+++ lucene/dev/branches/lucene6699/lucene/grouping/src/java/org/apache/lucene/search/grouping/AbstractFirstPassGroupingCollector.java Wed Sep  2 13:06:13 2015
@@ -35,7 +35,6 @@ import java.util.*;
  */
 abstract public class AbstractFirstPassGroupingCollector<GROUP_VALUE_TYPE> extends SimpleCollector {
 
-  private final Sort groupSort;
   private final FieldComparator<?>[] comparators;
   private final LeafFieldComparator[] leafComparators;
   private final int[] reversed;
@@ -61,6 +60,7 @@ abstract public class AbstractFirstPassG
    *  @param topNGroups How many top groups to keep.
    *  @throws IOException If I/O related errors occur
    */
+  @SuppressWarnings({"unchecked", "rawtypes"})
   public AbstractFirstPassGroupingCollector(Sort groupSort, int topNGroups) throws IOException {
     if (topNGroups < 1) {
       throw new IllegalArgumentException("topNGroups must be >= 1 (got " + topNGroups + ")");
@@ -68,7 +68,6 @@ abstract public class AbstractFirstPassG
 
     // TODO: allow null groupSort to mean "by relevance",
     // and specialize it?
-    this.groupSort = groupSort;
 
     this.topNGroups = topNGroups;
 
@@ -116,7 +115,7 @@ abstract public class AbstractFirstPassG
 
     final Collection<SearchGroup<GROUP_VALUE_TYPE>> result = new ArrayList<>();
     int upto = 0;
-    final int sortFieldCount = groupSort.getSort().length;
+    final int sortFieldCount = comparators.length;
     for(CollectedSearchGroup<GROUP_VALUE_TYPE> group : orderedGroups) {
       if (upto++ < groupOffset) {
         continue;

Modified: lucene/dev/branches/lucene6699/lucene/grouping/src/java/org/apache/lucene/search/grouping/SearchGroup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/grouping/src/java/org/apache/lucene/search/grouping/SearchGroup.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/grouping/src/java/org/apache/lucene/search/grouping/SearchGroup.java (original)
+++ lucene/dev/branches/lucene6699/lucene/grouping/src/java/org/apache/lucene/search/grouping/SearchGroup.java Wed Sep  2 13:06:13 2015
@@ -18,7 +18,6 @@ package org.apache.lucene.search.groupin
  */
 
 import org.apache.lucene.search.FieldComparator;
-import org.apache.lucene.search.LeafFieldComparator;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 
@@ -154,9 +153,12 @@ public class SearchGroup<GROUP_VALUE_TYP
 
   private static class GroupComparator<T> implements Comparator<MergedGroup<T>> {
 
+    @SuppressWarnings("rawtypes")
     public final FieldComparator[] comparators;
+    
     public final int[] reversed;
 
+    @SuppressWarnings({"unchecked", "rawtypes"})
     public GroupComparator(Sort groupSort) throws IOException {
       final SortField[] sortFields = groupSort.getSort();
       comparators = new FieldComparator[sortFields.length];

Modified: lucene/dev/branches/lucene6699/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java (original)
+++ lucene/dev/branches/lucene6699/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java Wed Sep  2 13:06:13 2015
@@ -119,7 +119,7 @@ public abstract class TermAllGroupHeadsC
     private final Sort sortWithinGroup;
     private final Map<BytesRef, GroupHead> groups;
 
-    private Scorer scorer;
+    Scorer scorer;
 
     GeneralAllGroupHeadsCollector(String groupField, Sort sortWithinGroup) {
       super(groupField, sortWithinGroup.getSort().length);
@@ -182,10 +182,13 @@ public abstract class TermAllGroupHeadsC
 
     class GroupHead extends AbstractAllGroupHeadsCollector.GroupHead<BytesRef> {
 
+      @SuppressWarnings({"unchecked", "rawtypes"})
       final FieldComparator[] comparators;
+      
       final LeafFieldComparator[] leafComparators;
 
-      private GroupHead(BytesRef groupValue, Sort sort, int doc) throws IOException {
+      @SuppressWarnings({"unchecked", "rawtypes"})
+      GroupHead(BytesRef groupValue, Sort sort, int doc) throws IOException {
         super(groupValue, doc + readerContext.docBase);
         final SortField[] sortFields = sort.getSort();
         comparators = new FieldComparator[sortFields.length];
@@ -221,10 +224,10 @@ public abstract class TermAllGroupHeadsC
 
     private final SentinelIntSet ordSet;
     private final List<GroupHead> collectedGroups;
-    private final SortField[] fields;
+    final SortField[] fields;
 
-    private SortedDocValues[] sortsIndex;
-    private Scorer scorer;
+    SortedDocValues[] sortsIndex;
+    Scorer scorer;
     private GroupHead[] segmentGroupHeads;
 
     OrdScoreAllGroupHeadsCollector(String groupField, Sort sortWithinGroup, int initialSize) {
@@ -322,7 +325,7 @@ public abstract class TermAllGroupHeadsC
       int[] sortOrds;
       float[] scores;
 
-      private GroupHead(int doc, BytesRef groupValue) throws IOException {
+      GroupHead(int doc, BytesRef groupValue) throws IOException {
         super(groupValue, doc + readerContext.docBase);
         sortValues = new BytesRefBuilder[sortsIndex.length];
         sortOrds = new int[sortsIndex.length];
@@ -384,8 +387,8 @@ public abstract class TermAllGroupHeadsC
     private final List<GroupHead> collectedGroups;
     private final SortField[] fields;
 
-    private SortedDocValues[] sortsIndex;
-    private GroupHead[] segmentGroupHeads;
+    SortedDocValues[] sortsIndex;
+    GroupHead[] segmentGroupHeads;
 
     OrdAllGroupHeadsCollector(String groupField, Sort sortWithinGroup, int initialSize) {
       super(groupField, sortWithinGroup.getSort().length);
@@ -473,7 +476,7 @@ public abstract class TermAllGroupHeadsC
       BytesRefBuilder[] sortValues;
       int[] sortOrds;
 
-      private GroupHead(int doc, BytesRef groupValue) {
+      GroupHead(int doc, BytesRef groupValue) {
         super(groupValue, doc + readerContext.docBase);
         sortValues = new BytesRefBuilder[sortsIndex.length];
         sortOrds = new int[sortsIndex.length];
@@ -512,12 +515,12 @@ public abstract class TermAllGroupHeadsC
   // AbstractAllGroupHeadsCollector optimized for scores.
   static class ScoreAllGroupHeadsCollector extends TermAllGroupHeadsCollector<ScoreAllGroupHeadsCollector.GroupHead> {
 
-    private final SentinelIntSet ordSet;
-    private final List<GroupHead> collectedGroups;
-    private final SortField[] fields;
+    final SentinelIntSet ordSet;
+    final List<GroupHead> collectedGroups;
+    final SortField[] fields;
 
-    private Scorer scorer;
-    private GroupHead[] segmentGroupHeads;
+    Scorer scorer;
+    GroupHead[] segmentGroupHeads;
 
     ScoreAllGroupHeadsCollector(String groupField, Sort sortWithinGroup, int initialSize) {
       super(groupField, sortWithinGroup.getSort().length);
@@ -591,7 +594,7 @@ public abstract class TermAllGroupHeadsC
 
       float[] scores;
 
-      private GroupHead(int doc, BytesRef groupValue) throws IOException {
+      GroupHead(int doc, BytesRef groupValue) throws IOException {
         super(groupValue, doc + readerContext.docBase);
         scores = new float[fields.length];
         float score = scorer.score();

Modified: lucene/dev/branches/lucene6699/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java (original)
+++ lucene/dev/branches/lucene6699/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java Wed Sep  2 13:06:13 2015
@@ -42,7 +42,7 @@ public class TermSecondPassGroupingColle
   private SortedDocValues index;
   private final String groupField;
 
-  @SuppressWarnings({"unchecked"})
+  @SuppressWarnings({"unchecked", "rawtypes"})
   public TermSecondPassGroupingCollector(String groupField, Collection<SearchGroup<BytesRef>> groups, Sort groupSort, Sort withinGroupSort,
                                          int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields)
       throws IOException {

Modified: lucene/dev/branches/lucene6699/lucene/highlighter/src/java/org/apache/lucene/search/highlight/DefaultEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/highlighter/src/java/org/apache/lucene/search/highlight/DefaultEncoder.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/highlighter/src/java/org/apache/lucene/search/highlight/DefaultEncoder.java (original)
+++ lucene/dev/branches/lucene6699/lucene/highlighter/src/java/org/apache/lucene/search/highlight/DefaultEncoder.java Wed Sep  2 13:06:13 2015
@@ -1,5 +1,5 @@
 package org.apache.lucene.search.highlight;
-/**
+/*
  * Copyright 2005 The Apache Software Foundation
  *
  * Licensed under the Apache License, Version 2.0 (the "License");

Modified: lucene/dev/branches/lucene6699/lucene/highlighter/src/java/org/apache/lucene/search/highlight/Encoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/highlighter/src/java/org/apache/lucene/search/highlight/Encoder.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/highlighter/src/java/org/apache/lucene/search/highlight/Encoder.java (original)
+++ lucene/dev/branches/lucene6699/lucene/highlighter/src/java/org/apache/lucene/search/highlight/Encoder.java Wed Sep  2 13:06:13 2015
@@ -1,5 +1,5 @@
 package org.apache.lucene.search.highlight;
-/**
+/*
  * Copyright 2005 The Apache Software Foundation
  *
  * Licensed under the Apache License, Version 2.0 (the "License");

Modified: lucene/dev/branches/lucene6699/lucene/highlighter/src/java/org/apache/lucene/search/highlight/SimpleHTMLEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/highlighter/src/java/org/apache/lucene/search/highlight/SimpleHTMLEncoder.java?rev=1700800&r1=1700799&r2=1700800&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/highlighter/src/java/org/apache/lucene/search/highlight/SimpleHTMLEncoder.java (original)
+++ lucene/dev/branches/lucene6699/lucene/highlighter/src/java/org/apache/lucene/search/highlight/SimpleHTMLEncoder.java Wed Sep  2 13:06:13 2015
@@ -1,5 +1,5 @@
 package org.apache.lucene.search.highlight;
-/**
+/*
  * Copyright 2005 The Apache Software Foundation
  *
  * Licensed under the Apache License, Version 2.0 (the "License");