You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2013/01/18 19:31:23 UTC

svn commit: r1435287 [9/41] - in /lucene/dev/branches/LUCENE-2878: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/analysis/icu/ dev-tools/maven/ dev-tools/maven/lucene/benchmark/ dev-tools/maven/solr/ dev-tools/m...

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/NRTManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/NRTManager.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/NRTManager.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/NRTManager.java Fri Jan 18 18:30:54 2013
@@ -28,10 +28,8 @@ import java.util.concurrent.locks.Reentr
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexDocument;
-import org.apache.lucene.index.SegmentInfoPerCommit;
 import org.apache.lucene.index.IndexReader; // javadocs
 import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.IndexSearcher; // javadocs
 import org.apache.lucene.search.SearcherFactory; // javadocs
@@ -70,7 +68,7 @@ import org.apache.lucene.util.ThreadInte
  * @lucene.experimental
  */
 
-public class NRTManager extends ReferenceManager<IndexSearcher> {
+public final class NRTManager extends ReferenceManager<IndexSearcher> {
   private static final long MAX_SEARCHER_GEN = Long.MAX_VALUE;
   private final TrackingIndexWriter writer;
   private final List<WaitingListener> waitingListeners = new CopyOnWriteArrayList<WaitingListener>();
@@ -361,7 +359,7 @@ public class NRTManager extends Referenc
   }
 
   @Override
-  protected void afterRefresh() {
+  protected void afterMaybeRefresh() {
     genLock.lock();
     try {
       if (searchingGen != MAX_SEARCHER_GEN) {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/NRTManagerReopenThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/NRTManagerReopenThread.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/NRTManagerReopenThread.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/NRTManagerReopenThread.java Fri Jan 18 18:30:54 2013
@@ -113,6 +113,7 @@ public class NRTManagerReopenThread exte
     manager.addWaitingListener(this);
   }
 
+  @Override
   public synchronized void close() {
     //System.out.println("NRT: set finish");
     manager.removeWaitingListener(this);
@@ -125,6 +126,7 @@ public class NRTManagerReopenThread exte
     }
   }
 
+  @Override
   public synchronized void waiting(long targetGen) {
     waitingGen = Math.max(waitingGen, targetGen);
     notify();

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java Fri Jan 18 18:30:54 2013
@@ -73,14 +73,9 @@ import org.apache.lucene.index.Term; // 
  * details.
  *
  * <p>This query defaults to {@linkplain
- * MultiTermQuery#CONSTANT_SCORE_AUTO_REWRITE_DEFAULT} for
- * 32 bit (int/float) ranges with precisionStep &le;8 and 64
- * bit (long/double) ranges with precisionStep &le;6.
- * Otherwise it uses {@linkplain
- * MultiTermQuery#CONSTANT_SCORE_FILTER_REWRITE} as the
- * number of terms is likely to be high.  With precision
- * steps of &le;4, this query can be run with one of the
- * BooleanQuery rewrite methods without changing
+ * MultiTermQuery#CONSTANT_SCORE_AUTO_REWRITE_DEFAULT}.
+ * With precision steps of &le;4, this query can be run with
+ * one of the BooleanQuery rewrite methods without changing
  * BooleanQuery's default max clause count.
  *
  * <br><h3>How it works</h3>
@@ -117,17 +112,29 @@ import org.apache.lucene.index.Term; // 
  *
  * <a name="precisionStepDesc"><h3>Precision Step</h3>
  * <p>You can choose any <code>precisionStep</code> when encoding values.
- * Lower step values mean more precisions and so more terms in index (and index gets larger).
- * On the other hand, the maximum number of terms to match reduces, which optimized query speed.
- * The formula to calculate the maximum term count is:
- * <pre>
- *  n = [ (bitsPerValue/precisionStep - 1) * (2^precisionStep - 1 ) * 2 ] + (2^precisionStep - 1 )
- * </pre>
- * <p><em>(this formula is only correct, when <code>bitsPerValue/precisionStep</code> is an integer;
- * in other cases, the value must be rounded up and the last summand must contain the modulo of the division as
- * precision step)</em>.
- * For longs stored using a precision step of 4, <code>n = 15*15*2 + 15 = 465</code>, and for a precision
- * step of 2, <code>n = 31*3*2 + 3 = 189</code>. But the faster search speed is reduced by more seeking
+ * Lower step values mean more precisions and so more terms in index (and index gets larger). The number
+ * of indexed terms per value is (those are generated by {@link NumericTokenStream}):
+ * <p style="font-family:serif">
+ * &nbsp;&nbsp;indexedTermsPerValue = <b>ceil</b><big>(</big>bitsPerValue / precisionStep<big>)</big>
+ * </p>
+ * As the lower precision terms are shared by many values, the additional terms only
+ * slightly grow the term dictionary (approx. 7% for <code>precisionStep=4</code>), but have a larger
+ * impact on the postings (the postings file will have  more entries, as every document is linked to
+ * <code>indexedTermsPerValue</code> terms instead of one). The formula to estimate the growth
+ * of the term dictionary in comparison to one term per value:
+ * <p>
+ * <!-- the formula in the alt attribute was transformed from latex to PNG with http://1.618034.com/latex.php (with 110 dpi): -->
+ * &nbsp;&nbsp;<img src="doc-files/nrq-formula-1.png" alt="\mathrm{termDictOverhead} = \sum\limits_{i=0}^{\mathrm{indexedTermsPerValue}-1} \frac{1}{2^{\mathrm{precisionStep}\cdot i}}" />
+ * </p>
+ * <p>On the other hand, if the <code>precisionStep</code> is smaller, the maximum number of terms to match reduces,
+ * which optimizes query speed. The formula to calculate the maximum number of terms that will be visited while
+ * executing the query is:
+ * <p>
+ * <!-- the formula in the alt attribute was transformed from latex to PNG with http://1.618034.com/latex.php (with 110 dpi): -->
+ * &nbsp;&nbsp;<img src="doc-files/nrq-formula-2.png" alt="\mathrm{maxQueryTerms} = \left[ \left( \mathrm{indexedTermsPerValue} - 1 \right) \cdot \left(2^\mathrm{precisionStep} - 1 \right) \cdot 2 \right] + \left( 2^\mathrm{precisionStep} - 1 \right)" />
+ * </p>
+ * <p>For longs stored using a precision step of 4, <code>maxQueryTerms = 15*15*2 + 15 = 465</code>, and for a precision
+ * step of 2, <code>maxQueryTerms = 31*3*2 + 3 = 189</code>. But the faster search speed is reduced by more seeking
  * in the term enum of the index. Because of this, the ideal <code>precisionStep</code> value can only
  * be found out by testing. <b>Important:</b> You can index with a lower precision step value and test search speed
  * using a multiple of the original step value.</p>
@@ -143,7 +150,7 @@ import org.apache.lucene.index.Term; // 
  *  per value in the index and querying is as slow as a conventional {@link TermRangeQuery}. But it can be used
  *  to produce fields, that are solely used for sorting (in this case simply use {@link Integer#MAX_VALUE} as
  *  <code>precisionStep</code>). Using {@link IntField},
- * {@link LongField}, {@link FloatField} or {@link DoubleField} for sorting
+ *  {@link LongField}, {@link FloatField} or {@link DoubleField} for sorting
  *  is ideal, because building the field cache is much faster than with text-only numbers.
  *  These fields have one term per value and therefore also work with term enumeration for building distinct lists
  *  (e.g. facets / preselected values to search for).

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java Fri Jan 18 18:30:54 2013
@@ -20,6 +20,7 @@ package org.apache.lucene.search;
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.Term;
@@ -161,6 +162,7 @@ public class PhraseQuery extends Query {
       }
     }
 
+    @Override
     public int compareTo(PostingsAndFreq other) {
       if (docFreq != other.docFreq) {
         return docFreq - other.docFreq;
@@ -265,7 +267,8 @@ public class PhraseQuery extends Query {
           return null;
         }
         te.seekExact(t.bytes(), state);
-        final DocsAndPositionsEnum postingsEnum = te.docsAndPositions(liveDocs, null, 0);
+        DocsAndPositionsEnum postingsEnum = te.docsAndPositions(liveDocs, null, DocsEnum.FLAG_NONE);
+
         // PhraseQuery on a field that did not index
         // positions.
         if (postingsEnum == null) {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReferenceManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReferenceManager.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReferenceManager.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReferenceManager.java Fri Jan 18 18:30:54 2013
@@ -19,6 +19,8 @@ package org.apache.lucene.search;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
@@ -44,7 +46,9 @@ public abstract class ReferenceManager<G
   protected volatile G current;
   
   private final Lock refreshLock = new ReentrantLock();
-  
+
+  private final List<RefreshListener> refreshListeners = new CopyOnWriteArrayList<RefreshListener>();
+
   private void ensureOpen() {
     if (current == null) {
       throw new AlreadyClosedException(REFERENCE_MANAGER_IS_CLOSED_MSG);
@@ -117,6 +121,7 @@ public abstract class ReferenceManager<G
     * @throws IOException
     *           if the underlying reader of the current reference could not be closed
    */
+  @Override
   public final synchronized void close() throws IOException {
     if (current != null) {
       // make sure we can call this more than once
@@ -142,18 +147,18 @@ public abstract class ReferenceManager<G
     // Per ReentrantLock's javadoc, calling lock() by the same thread more than
     // once is ok, as long as unlock() is called a matching number of times.
     refreshLock.lock();
+    boolean refreshed = false;
     try {
       final G reference = acquire();
       try {
         G newReference = refreshIfNeeded(reference);
         if (newReference != null) {
           assert newReference != reference : "refreshIfNeeded should return null if refresh wasn't needed";
-          boolean success = false;
           try {
             swapReference(newReference);
-            success = true;
+            refreshed = true;
           } finally {
-            if (!success) {
+            if (!refreshed) {
               release(newReference);
             }
           }
@@ -161,12 +166,15 @@ public abstract class ReferenceManager<G
       } finally {
         release(reference);
       }
-      afterRefresh();
+      afterMaybeRefresh();
+      if (refreshed) {
+        notifyRefreshListeners();
+      }
     } finally {
       refreshLock.unlock();
     }
   }
-  
+
   /**
    * You must call this (or {@link #maybeRefreshBlocking()}), periodically, if
    * you want that {@link #acquire()} will return refreshed instances.
@@ -228,11 +236,11 @@ public abstract class ReferenceManager<G
     }
   }
 
-  /** Called after swapReference has installed a new
-   *  instance.
+  /** Called after a refresh was attempted, regardless of
+   *  whether a new reference was in fact created.
    *  @throws IOException if a low level I/O exception occurs  
    **/
-  protected void afterRefresh() throws IOException {
+  protected void afterMaybeRefresh() throws IOException {
   }
   
   /**
@@ -245,4 +253,40 @@ public abstract class ReferenceManager<G
     assert reference != null;
     decRef(reference);
   }
+
+  private void notifyRefreshListeners() {
+    for (RefreshListener refreshListener : refreshListeners) {
+      refreshListener.afterRefresh();
+    }
+  }
+
+  /**
+   * Adds a listener, to be notified when a reference is refreshed/swapped.
+   */
+  public void addListener(RefreshListener listener) {
+    if (listener == null) {
+      throw new NullPointerException("Listener cannot be null");
+    }
+    refreshListeners.add(listener);
+  }
+
+  /**
+   * Remove a listener added with {@link #addListener(RefreshListener)}.
+   */
+  public void removeListener(RefreshListener listener) {
+    if (listener == null) {
+      throw new NullPointerException("Listener cannot be null");
+    }
+    refreshListeners.remove(listener);
+  }
+
+  /** Use to receive notification when a refresh has
+   *  finished.  See {@link #addListener}. */
+  public interface RefreshListener {
+
+    /**
+     * Called after a successful refresh and a new reference has been installed. When this is called {@link #acquire()} is guaranteed to return a new instance.
+     */
+    void afterRefresh();
+  }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/RegexpQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/RegexpQuery.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/RegexpQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/RegexpQuery.java Fri Jan 18 18:30:54 2013
@@ -51,6 +51,7 @@ public class RegexpQuery extends Automat
    * A provider that provides no named automata
    */
   private static AutomatonProvider defaultProvider = new AutomatonProvider() {
+    @Override
     public Automaton getAutomaton(String name) {
       return null;
     }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java Fri Jan 18 18:30:54 2013
@@ -364,6 +364,7 @@ final class SloppyPhraseScorer extends S
   private void sortRptGroups(ArrayList<ArrayList<PhrasePositions>> rgs) {
     rptGroups = new PhrasePositions[rgs.size()][];
     Comparator<PhrasePositions> cmprtr = new Comparator<PhrasePositions>() {
+      @Override
       public int compare(PhrasePositions pp1, PhrasePositions pp2) {
         return pp1.offset - pp2.offset;
       }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TopDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TopDocs.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TopDocs.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TopDocs.java Fri Jan 18 18:30:54 2013
@@ -91,6 +91,7 @@ public class TopDocs {
     }
 
     // Returns true if first is < second
+    @Override
     public boolean lessThan(ShardRef first, ShardRef second) {
       assert first != second;
       final float firstScore = shardHits[first.shardIndex][first.hitIndex].score;
@@ -156,6 +157,7 @@ public class TopDocs {
     }
 
     // Returns true if first is < second
+    @Override
     @SuppressWarnings({"unchecked","rawtypes"})
     public boolean lessThan(ShardRef first, ShardRef second) {
       assert first != second;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java Fri Jan 18 18:30:54 2013
@@ -183,6 +183,7 @@ public abstract class TopTermsRewrite<Q 
   
   private static final Comparator<ScoreTerm> scoreTermSortByTermComp = 
     new Comparator<ScoreTerm>() {
+      @Override
       public int compare(ScoreTerm st1, ScoreTerm st2) {
         assert st1.termComp == st2.termComp :
           "term comparator should not change between segments";
@@ -200,6 +201,7 @@ public abstract class TopTermsRewrite<Q 
       this.termState = termState;
     }
     
+    @Override
     public int compareTo(ScoreTerm other) {
       if (this.boost == other.boost)
         return termComp.compare(other.bytes, this.bytes);

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/similarities/DefaultSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/similarities/DefaultSimilarity.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/similarities/DefaultSimilarity.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/similarities/DefaultSimilarity.java Fri Jan 18 18:30:54 2013
@@ -28,11 +28,13 @@ public class DefaultSimilarity extends T
   public DefaultSimilarity() {}
   
   /** Implemented as <code>overlap / maxOverlap</code>. */
+  @Override
   public float coord(int overlap, int maxOverlap) {
     return overlap / (float)maxOverlap;
   }
 
   /** Implemented as <code>1/sqrt(sumOfSquaredWeights)</code>. */
+  @Override
   public float queryNorm(float sumOfSquaredWeights) {
     return (float)(1.0 / Math.sqrt(sumOfSquaredWeights));
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java Fri Jan 18 18:30:54 2013
@@ -73,6 +73,7 @@ public class NearSpansOrdered extends Sp
 
   private final Spans[] subSpansByDoc;
   private final Comparator<Spans> spanDocComparator = new Comparator<Spans>() {
+    @Override
     public int compare(Spans o1, Spans o2) {
       return o1.doc() - o2.doc();
     }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java Fri Jan 18 18:30:54 2013
@@ -123,6 +123,7 @@ final class CompoundFileWriter implement
    *           if close() had been called before or if no file has been added to
    *           this object
    */
+  @Override
   public void close() throws IOException {
     if (closed) {
       return;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/Directory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/Directory.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/Directory.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/Directory.java Fri Jan 18 18:30:54 2013
@@ -128,6 +128,7 @@ public abstract class Directory implemen
   }
 
   /** Closes the store. */
+  @Override
   public abstract void close()
        throws IOException;
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java Fri Jan 18 18:30:54 2013
@@ -220,6 +220,7 @@ public abstract class FSDirectory extend
 
     // Exclude subdirs
     String[] result = dir.list(new FilenameFilter() {
+        @Override
         public boolean accept(File dir, String file) {
           return !new File(dir, file).isDirectory();
         }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/IndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/IndexInput.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/IndexInput.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/IndexInput.java Fri Jan 18 18:30:54 2013
@@ -50,6 +50,7 @@ public abstract class IndexInput extends
   }
 
   /** Closes the stream to further operations. */
+  @Override
   public abstract void close() throws IOException;
 
   /** Returns the current position in this file, where the next read will

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java Fri Jan 18 18:30:54 2013
@@ -35,6 +35,7 @@ public abstract class IndexOutput extend
   public abstract void flush() throws IOException;
 
   /** Closes this stream to further operations. */
+  @Override
   public abstract void close() throws IOException;
 
   /** Returns the current position in this file, where the next write will

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java Fri Jan 18 18:30:54 2013
@@ -186,6 +186,7 @@ public class MMapDirectory extends FSDir
     if (useUnmapHack) {
       try {
         AccessController.doPrivileged(new PrivilegedExceptionAction<Object>() {
+          @Override
           public Object run() throws Exception {
             final Method getCleanerMethod = buffer.getClass()
               .getMethod("cleaner");

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java Fri Jan 18 18:30:54 2013
@@ -80,6 +80,7 @@ public class NIOFSDirectory extends FSDi
     return new NIOFSIndexInput(new File(getDirectory(), name), context, getReadChunkSize());
   }
   
+  @Override
   public IndexInputSlicer createSlicer(final String name,
       final IOContext context) throws IOException {
     ensureOpen();

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java Fri Jan 18 18:30:54 2013
@@ -233,6 +233,7 @@ public class NRTCachingDirectory extends
     }
   }
 
+  @Override
   public synchronized IndexInputSlicer createSlicer(final String name, final IOContext context) throws IOException {
     ensureOpen();
     if (VERBOSE) {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/RateLimitedDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/RateLimitedDirectoryWrapper.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/RateLimitedDirectoryWrapper.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/RateLimitedDirectoryWrapper.java Fri Jan 18 18:30:54 2013
@@ -40,26 +40,31 @@ public final class RateLimitedDirectoryW
     this.delegate = wrapped;
   }
   
+  @Override
   public String[] listAll() throws IOException {
     ensureOpen();
     return delegate.listAll();
   }
   
+  @Override
   public boolean fileExists(String name) throws IOException {
     ensureOpen();
     return delegate.fileExists(name);
   }
   
+  @Override
   public void deleteFile(String name) throws IOException {
     ensureOpen();
     delegate.deleteFile(name);
   }
   
+  @Override
   public long fileLength(String name) throws IOException {
     ensureOpen();
     return delegate.fileLength(name);
   }
   
+  @Override
   public IndexOutput createOutput(String name, IOContext context)
       throws IOException {
     ensureOpen();
@@ -71,22 +76,26 @@ public final class RateLimitedDirectoryW
     return output;
   }
   
+  @Override
   public void sync(Collection<String> names) throws IOException {
     ensureOpen();
     delegate.sync(names);
   }
   
+  @Override
   public IndexInput openInput(String name, IOContext context)
       throws IOException {
     ensureOpen();
     return delegate.openInput(name, context);
   }
   
+  @Override
   public void close() throws IOException {
     isOpen = false;
     delegate.close();
   }
   
+  @Override
   public IndexInputSlicer createSlicer(String name, IOContext context)
       throws IOException {
     ensureOpen();

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/RateLimiter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/RateLimiter.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/RateLimiter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/RateLimiter.java Fri Jan 18 18:30:54 2013
@@ -64,6 +64,7 @@ public abstract class RateLimiter {
     /**
      * Sets an updated mb per second rate limit.
      */
+    @Override
     public void setMbPerSec(double mbPerSec) {
       this.mbPerSec = mbPerSec;
       nsPerByte = 1000000000. / (1024*1024*mbPerSec);
@@ -73,6 +74,7 @@ public abstract class RateLimiter {
     /**
      * The current mb per second rate limit.
      */
+    @Override
     public double getMbPerSec() {
       return this.mbPerSec;
     }
@@ -86,6 +88,7 @@ public abstract class RateLimiter {
      *  with a biggish count, not one byte at a time.
      *  @return the pause time in nano seconds 
      * */
+    @Override
     public long pause(long bytes) {
       if (bytes == 1) {
         return 0;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java Fri Jan 18 18:30:54 2013
@@ -58,6 +58,7 @@ public class SimpleFSDirectory extends F
     return new SimpleFSIndexInput("SimpleFSIndexInput(path=\"" + path.getPath() + "\")", path, context, getReadChunkSize());
   }
 
+  @Override
   public IndexInputSlicer createSlicer(final String name,
       final IOContext context) throws IOException {
     ensureOpen();

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/AttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/AttributeImpl.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/AttributeImpl.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/AttributeImpl.java Fri Jan 18 18:30:54 2013
@@ -51,6 +51,7 @@ public abstract class AttributeImpl impl
   public final String reflectAsString(final boolean prependAttClass) {
     final StringBuilder buffer = new StringBuilder();
     reflectWith(new AttributeReflector() {
+      @Override
       public void reflect(Class<? extends Attribute> attClass, String key, Object value) {
         if (buffer.length() > 0) {
           buffer.append(',');

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/AttributeSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/AttributeSource.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/AttributeSource.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/AttributeSource.java Fri Jan 18 18:30:54 2013
@@ -175,10 +175,12 @@ public class AttributeSource {
       return new Iterator<AttributeImpl>() {
         private State state = initState;
       
+        @Override
         public void remove() {
           throw new UnsupportedOperationException();
         }
         
+        @Override
         public AttributeImpl next() {
           if (state == null)
             throw new NoSuchElementException();
@@ -187,6 +189,7 @@ public class AttributeSource {
           return att;
         }
         
+        @Override
         public boolean hasNext() {
           return state != null;
         }
@@ -425,6 +428,7 @@ public class AttributeSource {
   public final String reflectAsString(final boolean prependAttClass) {
     final StringBuilder buffer = new StringBuilder();
     reflectWith(new AttributeReflector() {
+      @Override
       public void reflect(Class<? extends Attribute> attClass, String key, Object value) {
         if (buffer.length() > 0) {
           buffer.append(',');

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/Bits.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/Bits.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/Bits.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/Bits.java Fri Jan 18 18:30:54 2013
@@ -47,10 +47,12 @@ public interface Bits {
       this.len = len;
     }
 
+    @Override
     public boolean get(int index) {
       return true;
     }
 
+    @Override
     public int length() {
       return len;
     }
@@ -66,10 +68,12 @@ public interface Bits {
       this.len = len;
     }
 
+    @Override
     public boolean get(int index) {
       return false;
     }
 
+    @Override
     public int length() {
       return len;
     }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java Fri Jan 18 18:30:54 2013
@@ -99,6 +99,7 @@ public final class ByteBlockPool {
       this.bytesUsed = bytesUsed;
     }
 
+    @Override
     public byte[] getByteBlock() {
       bytesUsed.addAndGet(blockSize);
       return new byte[blockSize];

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/BytesRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/BytesRef.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/BytesRef.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/BytesRef.java Fri Jan 18 18:30:54 2013
@@ -226,6 +226,7 @@ public final class BytesRef implements C
   }
 
   /** Unsigned byte order comparison */
+  @Override
   public int compareTo(BytesRef other) {
     return utf8SortedAsUnicodeSortOrder.compare(this, other);
   }
@@ -240,6 +241,7 @@ public final class BytesRef implements C
     // Only singleton
     private UTF8SortedAsUnicodeComparator() {};
 
+    @Override
     public int compare(BytesRef a, BytesRef b) {
       final byte[] aBytes = a.bytes;
       int aUpto = a.offset;
@@ -278,6 +280,7 @@ public final class BytesRef implements C
     // Only singleton
     private UTF8SortedAsUTF16Comparator() {};
 
+    @Override
     public int compare(BytesRef a, BytesRef b) {
 
       final byte[] aBytes = a.bytes;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/BytesRefIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/BytesRefIterator.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/BytesRefIterator.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/BytesRefIterator.java Fri Jan 18 18:30:54 2013
@@ -54,6 +54,7 @@ public interface BytesRefIterator {
       return null;
     }
     
+    @Override
     public Comparator<BytesRef> getComparator() {
       return null;
     }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/CharsRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/CharsRef.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/CharsRef.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/CharsRef.java Fri Jan 18 18:30:54 2013
@@ -118,6 +118,7 @@ public final class CharsRef implements C
   }
 
   /** Signed int order comparison */
+  @Override
   public int compareTo(CharsRef other) {
     if (this == other)
       return 0;
@@ -197,10 +198,12 @@ public final class CharsRef implements C
     return new String(chars, offset, length);
   }
 
+  @Override
   public int length() {
     return length;
   }
 
+  @Override
   public char charAt(int index) {
     // NOTE: must do a real check here to meet the specs of CharSequence
     if (index < 0 || index >= length) {
@@ -209,12 +212,13 @@ public final class CharsRef implements C
     return chars[offset + index];
   }
 
+  @Override
   public CharSequence subSequence(int start, int end) {
     // NOTE: must do a real check here to meet the specs of CharSequence
     if (start < 0 || end > length || start > end) {
       throw new IndexOutOfBoundsException();
     }
-    return new CharsRef(chars, offset + start, offset + end);
+    return new CharsRef(chars, offset + start, end - start);
   }
   
   /** @deprecated This comparator is only a transition mechanism */
@@ -233,6 +237,7 @@ public final class CharsRef implements C
     // Only singleton
     private UTF16SortedAsUTF8Comparator() {};
 
+    @Override
     public int compare(CharsRef a, CharsRef b) {
       if (a == b)
         return 0;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/CloseableThreadLocal.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/CloseableThreadLocal.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/CloseableThreadLocal.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/CloseableThreadLocal.java Fri Jan 18 18:30:54 2013
@@ -128,6 +128,7 @@ public class CloseableThreadLocal<T> imp
     }
   }
 
+  @Override
   public void close() {
     // Clear the hard refs; then, the only remaining refs to
     // all values we were storing are weak (unless somewhere

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/FilterIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/FilterIterator.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/FilterIterator.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/FilterIterator.java Fri Jan 18 18:30:54 2013
@@ -41,10 +41,12 @@ public abstract class FilterIterator<T, 
     this.iterator = baseIterator;
   }
   
+  @Override
   public final boolean hasNext() {
     return nextIsSet || setNext();
   }
   
+  @Override
   public final T next() {
     if (!hasNext()) {
       throw new NoSuchElementException();
@@ -58,6 +60,7 @@ public abstract class FilterIterator<T, 
     }
   }
   
+  @Override
   public final void remove() {
     throw new UnsupportedOperationException();
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java Fri Jan 18 18:30:54 2013
@@ -38,7 +38,8 @@ import org.apache.lucene.search.DocIdSet
 
 public final class FixedBitSet extends DocIdSet implements Bits {
   private final long[] bits;
-  private int numBits;
+  private final int numBits;
+  private final int wordLength;
 
   /** returns the number of 64 bit words it would take to hold numBits */
   public static int bits2words(int numBits) {
@@ -52,23 +53,29 @@ public final class FixedBitSet extends D
   public FixedBitSet(int numBits) {
     this.numBits = numBits;
     bits = new long[bits2words(numBits)];
+    wordLength = bits.length;
   }
 
-  public FixedBitSet(long[]storedBits,int numBits) {
+  public FixedBitSet(long[] storedBits, int numBits) {
+    this.wordLength = bits2words(numBits);
+    if (wordLength > storedBits.length) {
+      throw new IllegalArgumentException("The given long array is too small  to hold " + numBits + " bits");
+    }
     this.numBits = numBits;
     this.bits = storedBits;
   }      
   
   /** Makes full copy. */
   public FixedBitSet(FixedBitSet other) {
-    bits = new long[other.bits.length];
-    System.arraycopy(other.bits, 0, bits, 0, bits.length);
+    bits = new long[other.wordLength];
+    System.arraycopy(other.bits, 0, bits, 0, other.wordLength);
     numBits = other.numBits;
+    wordLength = other.wordLength;
   }
 
   @Override
   public DocIdSetIterator iterator() {
-    return new OpenBitSetIterator(bits, bits.length);
+    return new OpenBitSetIterator(bits, wordLength);
   }
 
   @Override
@@ -99,6 +106,7 @@ public final class FixedBitSet extends D
     return (int) BitUtil.pop_array(bits, 0, bits.length);
   }
 
+  @Override
   public boolean get(int index) {
     assert index >= 0 && index < numBits: "index=" + index;
     int i = index >> 6;               // div 64
@@ -158,7 +166,7 @@ public final class FixedBitSet extends D
       return (i<<6) + subIndex + Long.numberOfTrailingZeros(word);
     }
 
-    while(++i < bits.length) {
+    while(++i < wordLength) {
       word = bits[i];
       if (word != 0) {
         return (i<<6) + Long.numberOfTrailingZeros(word);
@@ -210,12 +218,12 @@ public final class FixedBitSet extends D
 
   /** this = this OR other */
   public void or(FixedBitSet other) {
-    or(other.bits, other.bits.length);
+    or(other.bits, other.wordLength);
   }
   
   private void or(final long[] otherArr, final int otherLen) {
     final long[] thisArr = this.bits;
-    int pos = Math.min(thisArr.length, otherLen);
+    int pos = Math.min(wordLength, otherLen);
     while (--pos >= 0) {
       thisArr[pos] |= otherArr[pos];
     }
@@ -246,17 +254,17 @@ public final class FixedBitSet extends D
 
   /** this = this AND other */
   public void and(FixedBitSet other) {
-    and(other.bits, other.bits.length);
+    and(other.bits, other.wordLength);
   }
   
   private void and(final long[] otherArr, final int otherLen) {
     final long[] thisArr = this.bits;
-    int pos = Math.min(thisArr.length, otherLen);
+    int pos = Math.min(this.wordLength, otherLen);
     while(--pos >= 0) {
       thisArr[pos] &= otherArr[pos];
     }
-    if (thisArr.length > otherLen) {
-      Arrays.fill(thisArr, otherLen, thisArr.length, 0L);
+    if (this.wordLength > otherLen) {
+      Arrays.fill(thisArr, otherLen, this.wordLength, 0L);
     }
   }
 
@@ -284,7 +292,7 @@ public final class FixedBitSet extends D
   
   private void andNot(final long[] otherArr, final int otherLen) {
     final long[] thisArr = this.bits;
-    int pos = Math.min(thisArr.length, otherLen);
+    int pos = Math.min(this.wordLength, otherLen);
     while(--pos >= 0) {
       thisArr[pos] &= ~otherArr[pos];
     }
@@ -417,7 +425,7 @@ public final class FixedBitSet extends D
   @Override
   public int hashCode() {
     long h = 0;
-    for (int i = bits.length; --i>=0;) {
+    for (int i = wordLength; --i>=0;) {
       h ^= bits[i];
       h = (h << 1) | (h >>> 63); // rotate left
     }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/IntsRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/IntsRef.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/IntsRef.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/IntsRef.java Fri Jan 18 18:30:54 2013
@@ -103,6 +103,7 @@ public final class IntsRef implements Co
   }
 
   /** Signed int order comparison */
+  @Override
   public int compareTo(IntsRef other) {
     if (this == other) return 0;
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/LongsRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/LongsRef.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/LongsRef.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/LongsRef.java Fri Jan 18 18:30:54 2013
@@ -102,6 +102,7 @@ public final class LongsRef implements C
   }
 
   /** Signed int order comparison */
+  @Override
   public int compareTo(LongsRef other) {
     if (this == other) return 0;
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/NamedSPILoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/NamedSPILoader.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/NamedSPILoader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/NamedSPILoader.java Fri Jan 18 18:30:54 2013
@@ -110,6 +110,7 @@ public final class NamedSPILoader<S exte
     return services.keySet();
   }
   
+  @Override
   public Iterator<S> iterator() {
     return services.values().iterator();
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/NamedThreadFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/NamedThreadFactory.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/NamedThreadFactory.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/NamedThreadFactory.java Fri Jan 18 18:30:54 2013
@@ -57,6 +57,7 @@ public class NamedThreadFactory implemen
    * 
    * @see java.util.concurrent.ThreadFactory#newThread(java.lang.Runnable)
    */
+  @Override
   public Thread newThread(Runnable r) {
     final Thread t = new Thread(group, r, String.format(Locale.ROOT, "%s-%d",
         this.threadNamePrefix, threadNumber.getAndIncrement()), 0);

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/OpenBitSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/OpenBitSet.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/OpenBitSet.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/OpenBitSet.java Fri Jan 18 18:30:54 2013
@@ -162,6 +162,7 @@ public class OpenBitSet extends DocIdSet
 
 
   /** Returns true or false for the specified bit index. */
+  @Override
   public boolean get(int index) {
     int i = index >> 6;               // div 64
     // signed shift will keep a negative index and force an

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/StringHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/StringHelper.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/StringHelper.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/StringHelper.java Fri Jan 18 18:30:54 2013
@@ -60,6 +60,7 @@ public abstract class StringHelper {
   }
   
   private static Comparator<String> versionComparator = new Comparator<String>() {
+    @Override
     public int compare(String a, String b) {
       StringTokenizer aTokens = new StringTokenizer(a, ".");
       StringTokenizer bTokens = new StringTokenizer(b, ".");

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/Version.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/Version.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/Version.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/Version.java Fri Jan 18 18:30:54 2013
@@ -47,7 +47,14 @@ public enum Version { 
   @Deprecated
   LUCENE_41,
 
-  /** Match settings and bugs in Lucene's 5.0 release. 
+  /**
+   * Match settings and bugs in Lucene's 4.2 release.
+   * @deprecated (5.0) Use latest
+   */
+  @Deprecated
+  LUCENE_42,
+
+  /** Match settings and bugs in Lucene's 5.0 release.
    *  <p>
    *  Use this to get the latest &amp; greatest settings, bug
    *  fixes, etc, for Lucene.

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/WeakIdentityMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/WeakIdentityMap.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/WeakIdentityMap.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/WeakIdentityMap.java Fri Jan 18 18:30:54 2013
@@ -202,10 +202,12 @@ public final class WeakIdentityMap<K,V> 
       hash = System.identityHashCode(obj);
     }
 
+    @Override
     public int hashCode() {
       return hash;
     }
 
+    @Override
     public boolean equals(Object o) {
       if (this == o) {
         return true;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/automaton/BasicOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/automaton/BasicOperations.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/automaton/BasicOperations.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/automaton/BasicOperations.java Fri Jan 18 18:30:54 2013
@@ -473,6 +473,7 @@ final public class BasicOperations {
     int point;
     final TransitionList ends = new TransitionList();
     final TransitionList starts = new TransitionList();
+    @Override
     public int compareTo(PointTransitions other) {
       return point - other.point;
     }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/automaton/State.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/automaton/State.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/automaton/State.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/automaton/State.java Fri Jan 18 18:30:54 2013
@@ -68,15 +68,19 @@ public class State implements Comparable
   }
 
   private class TransitionsIterable implements Iterable<Transition> {
+    @Override
     public Iterator<Transition> iterator() {
       return new Iterator<Transition>() {
         int upto;
+        @Override
         public boolean hasNext() {
           return upto < numTransitions;
         }
+        @Override
         public Transition next() {
           return transitionsArray[upto++];
         }
+        @Override
         public void remove() {
           throw new UnsupportedOperationException();
         }
@@ -269,6 +273,7 @@ public class State implements Comparable
    * Compares this object with the specified object for order. States are
    * ordered by the time of construction.
    */
+  @Override
   public int compareTo(State s) {
     return s.id - id;
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/automaton/Transition.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/automaton/Transition.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/automaton/Transition.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/automaton/Transition.java Fri Jan 18 18:30:54 2013
@@ -179,6 +179,7 @@ public class Transition implements Clone
   }
 
   private static final class CompareByDestThenMinMaxSingle implements Comparator<Transition> {
+    @Override
     public int compare(Transition t1, Transition t2) {
       if (t1.to != t2.to) {
         if (t1.to.number < t2.to.number) return -1;
@@ -195,6 +196,7 @@ public class Transition implements Clone
   public static final Comparator<Transition> CompareByDestThenMinMax = new CompareByDestThenMinMaxSingle();
 
   private static final class CompareByMinMaxThenDestSingle implements Comparator<Transition> {
+    @Override
     public int compare(Transition t1, Transition t2) {
       if (t1.min < t2.min) return -1;
       if (t1.min > t2.min) return 1;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java Fri Jan 18 18:30:54 2013
@@ -36,9 +36,13 @@ import org.apache.lucene.util.packed.Pac
  * <p>NOTE: The algorithm is described at
  * http://citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.24.3698</p>
  *
- * The parameterized type T is the output type.  See the
+ * <p>The parameterized type T is the output type.  See the
  * subclasses of {@link Outputs}.
  *
+ * <p>FSTs larger than 2.1GB are now possible (as of Lucene
+ * 4.2).  FSTs containing more than 2.1B nodes are also now
+ * possible, however they cannot be packed.
+ *
  * @lucene.experimental
  */
 
@@ -62,6 +66,10 @@ public class Builder<T> {
   private final int shareMaxTailLength;
 
   private final IntsRef lastInput = new IntsRef();
+  
+  // for packing
+  private final boolean doPackFST;
+  private final float acceptableOverheadRatio;
 
   // NOTE: cutting this over to ArrayList instead loses ~6%
   // in build performance on 9.8M Wikipedia terms; so we
@@ -80,22 +88,11 @@ public class Builder<T> {
   /**
    * Instantiates an FST/FSA builder without any pruning. A shortcut
    * to {@link #Builder(FST.INPUT_TYPE, int, int, boolean,
-   * boolean, int, Outputs, FreezeTail, boolean)} with
-   * pruning options turned off.
+   * boolean, int, Outputs, FreezeTail, boolean, float,
+   * boolean, int)} with pruning options turned off.
    */
   public Builder(FST.INPUT_TYPE inputType, Outputs<T> outputs) {
-    this(inputType, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, false, PackedInts.COMPACT);
-  }
-
-  /**
-   * Instantiates an FST/FSA builder with {@link PackedInts#DEFAULT}
-   * <code>acceptableOverheadRatio</code>.
-   */
-  public Builder(FST.INPUT_TYPE inputType, int minSuffixCount1, int minSuffixCount2, boolean doShareSuffix,
-      boolean doShareNonSingletonNodes, int shareMaxTailLength, Outputs<T> outputs,
-      FreezeTail<T> freezeTail, boolean willPackFST) {
-    this(inputType, minSuffixCount1, minSuffixCount2, doShareSuffix, doShareNonSingletonNodes,
-        shareMaxTailLength, outputs, freezeTail, willPackFST, PackedInts.DEFAULT);
+    this(inputType, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, false, PackedInts.COMPACT, true, 15);
   }
 
   /**
@@ -135,25 +132,34 @@ public class Builder<T> {
    *    FSA, use {@link NoOutputs#getSingleton()} and {@link NoOutputs#getNoOutput()} as the
    *    singleton output object.
    *
-   * @param willPackFST Pass true if you will pack the FST before saving.  This
-   *    causes the FST to create additional data structures internally to enable packing, but
-   *    it means the resulting FST cannot be saved until it
-   *    is packed using {@link FST#pack(int, int, float)}
-   *
+   * @param doPackFST Pass true to create a packed FST.
+   * 
    * @param acceptableOverheadRatio How to trade speed for space when building the FST. This option
-   *    is only relevant when willPackFST is true. @see PackedInts#getMutable(int, int, float)
+   *    is only relevant when doPackFST is true. @see PackedInts#getMutable(int, int, float)
+   *
+   * @param allowArrayArcs Pass false to disable the array arc optimization
+   *    while building the FST; this will make the resulting
+   *    FST smaller but slower to traverse.
+   *
+   * @param bytesPageBits How many bits wide to make each
+   *    byte[] block in the BytesStore; if you know the FST
+   *    will be large then make this larger.  For example 15
+   *    bits = 32768 byte pages.
    */
   public Builder(FST.INPUT_TYPE inputType, int minSuffixCount1, int minSuffixCount2, boolean doShareSuffix,
                  boolean doShareNonSingletonNodes, int shareMaxTailLength, Outputs<T> outputs,
-                 FreezeTail<T> freezeTail, boolean willPackFST, float acceptableOverheadRatio) {
+                 FreezeTail<T> freezeTail, boolean doPackFST, float acceptableOverheadRatio, boolean allowArrayArcs,
+                 int bytesPageBits) {
     this.minSuffixCount1 = minSuffixCount1;
     this.minSuffixCount2 = minSuffixCount2;
     this.freezeTail = freezeTail;
     this.doShareNonSingletonNodes = doShareNonSingletonNodes;
     this.shareMaxTailLength = shareMaxTailLength;
-    fst = new FST<T>(inputType, outputs, willPackFST, acceptableOverheadRatio);
+    this.doPackFST = doPackFST;
+    this.acceptableOverheadRatio = acceptableOverheadRatio;
+    fst = new FST<T>(inputType, outputs, doPackFST, acceptableOverheadRatio, allowArrayArcs, bytesPageBits);
     if (doShareSuffix) {
-      dedupHash = new NodeHash<T>(fst);
+      dedupHash = new NodeHash<T>(fst, fst.bytes.getReverseReader(false));
     } else {
       dedupHash = null;
     }
@@ -167,7 +173,7 @@ public class Builder<T> {
     }
   }
 
-  public int getTotStateCount() {
+  public long getTotStateCount() {
     return fst.nodeCount;
   }
 
@@ -175,19 +181,12 @@ public class Builder<T> {
     return frontier[0].inputCount;
   }
 
-  public int getMappedStateCount() {
+  public long getMappedStateCount() {
     return dedupHash == null ? 0 : fst.nodeCount;
   }
 
-  /** Pass false to disable the array arc optimization
-   *  while building the FST; this will make the resulting
-   *  FST smaller but slower to traverse. */
-  public void setAllowArrayArcs(boolean b) {
-    fst.setAllowArrayArcs(b);
-  }
-
   private CompiledNode compileNode(UnCompiledNode<T> nodeIn, int tailLength) throws IOException {
-    final int node;
+    final long node;
     if (dedupHash != null && (doShareNonSingletonNodes || nodeIn.numArcs <= 1) && tailLength <= shareMaxTailLength) {
       if (nodeIn.numArcs == 0) {
         node = fst.addNode(nodeIn);
@@ -474,7 +473,11 @@ public class Builder<T> {
     //if (DEBUG) System.out.println("  builder.finish root.isFinal=" + root.isFinal + " root.output=" + root.output);
     fst.finish(compileNode(root, lastInput.length).node);
 
-    return fst;
+    if (doPackFST) {
+      return fst.pack(3, Math.max(10, (int) (fst.getNodeCount()/4)), acceptableOverheadRatio);
+    } else {
+      return fst;
+    }
   }
 
   private void compileAllTargets(UnCompiledNode<T> node, int tailLength) throws IOException {
@@ -509,8 +512,13 @@ public class Builder<T> {
     boolean isCompiled();
   }
 
+  public long fstSizeInBytes() {
+    return fst.sizeInBytes();
+  }
+
   static final class CompiledNode implements Node {
-    int node;
+    long node;
+    @Override
     public boolean isCompiled() {
       return true;
     }
@@ -547,6 +555,7 @@ public class Builder<T> {
       this.depth = depth;
     }
 
+    @Override
     public boolean isCompiled() {
       return false;
     }