You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ha...@apache.org on 2013/08/13 06:06:27 UTC

svn commit: r1513336 [4/11] - in /lucene/dev/branches/lucene3069/lucene: ./ analysis/ analysis/common/ analysis/common/src/java/org/apache/lucene/analysis/charfilter/ analysis/common/src/java/org/apache/lucene/analysis/hunspell/ analysis/common/src/jav...

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java Tue Aug 13 04:06:18 2013
@@ -19,7 +19,6 @@ package org.apache.lucene.index;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
 import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
 import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
@@ -41,9 +40,7 @@ public class LiveIndexWriterConfig {
   private volatile int maxBufferedDocs;
   private volatile double ramBufferSizeMB;
   private volatile int maxBufferedDeleteTerms;
-  private volatile int readerTermsIndexDivisor;
   private volatile IndexReaderWarmer mergedSegmentWarmer;
-  private volatile int termIndexInterval; // TODO: this should be private to the codec, not settable here
 
   // modified by IndexWriterConfig
   /** {@link IndexDeletionPolicy} controlling when commit
@@ -108,9 +105,7 @@ public class LiveIndexWriterConfig {
     ramBufferSizeMB = IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB;
     maxBufferedDocs = IndexWriterConfig.DEFAULT_MAX_BUFFERED_DOCS;
     maxBufferedDeleteTerms = IndexWriterConfig.DEFAULT_MAX_BUFFERED_DELETE_TERMS;
-    readerTermsIndexDivisor = IndexWriterConfig.DEFAULT_READER_TERMS_INDEX_DIVISOR;
     mergedSegmentWarmer = null;
-    termIndexInterval = IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL; // TODO: this should be private to the codec, not settable here
     delPolicy = new KeepOnlyLastCommitDeletionPolicy();
     commit = null;
     useCompoundFile = IndexWriterConfig.DEFAULT_USE_COMPOUND_FILE_SYSTEM;
@@ -140,8 +135,6 @@ public class LiveIndexWriterConfig {
     maxBufferedDocs = config.getMaxBufferedDocs();
     mergedSegmentWarmer = config.getMergedSegmentWarmer();
     ramBufferSizeMB = config.getRAMBufferSizeMB();
-    readerTermsIndexDivisor = config.getReaderTermsIndexDivisor();
-    termIndexInterval = config.getTermIndexInterval();
     matchVersion = config.matchVersion;
     analyzer = config.getAnalyzer();
     delPolicy = config.getIndexDeletionPolicy();
@@ -165,73 +158,11 @@ public class LiveIndexWriterConfig {
   public Analyzer getAnalyzer() {
     return analyzer;
   }
-  
-  /**
-   * Expert: set the interval between indexed terms. Large values cause less
-   * memory to be used by IndexReader, but slow random-access to terms. Small
-   * values cause more memory to be used by an IndexReader, and speed
-   * random-access to terms.
-   * <p>
-   * This parameter determines the amount of computation required per query
-   * term, regardless of the number of documents that contain that term. In
-   * particular, it is the maximum number of other terms that must be scanned
-   * before a term is located and its frequency and position information may be
-   * processed. In a large index with user-entered query terms, query processing
-   * time is likely to be dominated not by term lookup but rather by the
-   * processing of frequency and positional data. In a small index or when many
-   * uncommon query terms are generated (e.g., by wildcard queries) term lookup
-   * may become a dominant cost.
-   * <p>
-   * In particular, <code>numUniqueTerms/interval</code> terms are read into
-   * memory by an IndexReader, and, on average, <code>interval/2</code> terms
-   * must be scanned for each random term access.
-   * 
-   * <p>
-   * Takes effect immediately, but only applies to newly flushed/merged
-   * segments.
-   * 
-   * <p>
-   * <b>NOTE:</b> This parameter does not apply to all PostingsFormat implementations,
-   * including the default one in this release. It only makes sense for term indexes
-   * that are implemented as a fixed gap between terms. For example, 
-   * {@link Lucene41PostingsFormat} implements the term index instead based upon how
-   * terms share prefixes. To configure its parameters (the minimum and maximum size
-   * for a block), you would instead use  {@link Lucene41PostingsFormat#Lucene41PostingsFormat(int, int)}.
-   * which can also be configured on a per-field basis:
-   * <pre class="prettyprint">
-   * //customize Lucene41PostingsFormat, passing minBlockSize=50, maxBlockSize=100
-   * final PostingsFormat tweakedPostings = new Lucene41PostingsFormat(50, 100);
-   * iwc.setCodec(new Lucene42Codec() {
-   *   &#64;Override
-   *   public PostingsFormat getPostingsFormatForField(String field) {
-   *     if (field.equals("fieldWithTonsOfTerms"))
-   *       return tweakedPostings;
-   *     else
-   *       return super.getPostingsFormatForField(field);
-   *   }
-   * });
-   * </pre>
-   * Note that other implementations may have their own parameters, or no parameters at all.
-   * 
-   * @see IndexWriterConfig#DEFAULT_TERM_INDEX_INTERVAL
-   */
-  public LiveIndexWriterConfig setTermIndexInterval(int interval) { // TODO: this should be private to the codec, not settable here
-    this.termIndexInterval = interval;
-    return this;
-  }
-
-  /**
-   * Returns the interval between indexed terms.
-   *
-   * @see #setTermIndexInterval(int)
-   */
-  public int getTermIndexInterval() { // TODO: this should be private to the codec, not settable here
-    return termIndexInterval;
-  }
 
   /**
-   * Determines the minimal number of delete terms required before the buffered
-   * in-memory delete terms and queries are applied and flushed.
+   * Determines the maximum number of delete-by-term operations that will be
+   * buffered before both the buffered in-memory delete terms and queries are
+   * applied and flushed.
    * <p>
    * Disabled by default (writer flushes by RAM usage).
    * <p>
@@ -239,7 +170,8 @@ public class LiveIndexWriterConfig {
    * 
    * <p>
    * Takes effect immediately, but only the next time a document is added,
-   * updated or deleted.
+   * updated or deleted. Also, if you only delete-by-query, this setting has no
+   * effect, i.e. delete queries are buffered until the next segment is flushed.
    * 
    * @throws IllegalArgumentException
    *           if maxBufferedDeleteTerms is enabled but smaller than 1
@@ -390,37 +322,6 @@ public class LiveIndexWriterConfig {
   public IndexReaderWarmer getMergedSegmentWarmer() {
     return mergedSegmentWarmer;
   }
-
-  /**
-   * Sets the termsIndexDivisor passed to any readers that IndexWriter opens,
-   * for example when applying deletes or creating a near-real-time reader in
-   * {@link DirectoryReader#open(IndexWriter, boolean)}. If you pass -1, the
-   * terms index won't be loaded by the readers. This is only useful in advanced
-   * situations when you will only .next() through all terms; attempts to seek
-   * will hit an exception.
-   * 
-   * <p>
-   * Takes effect immediately, but only applies to readers opened after this
-   * call
-   * <p>
-   * <b>NOTE:</b> divisor settings &gt; 1 do not apply to all PostingsFormat
-   * implementations, including the default one in this release. It only makes
-   * sense for terms indexes that can efficiently re-sample terms at load time.
-   */
-  public LiveIndexWriterConfig setReaderTermsIndexDivisor(int divisor) {
-    if (divisor <= 0 && divisor != -1) {
-      throw new IllegalArgumentException("divisor must be >= 1, or -1 (got " + divisor + ")");
-    }
-    readerTermsIndexDivisor = divisor;
-    return this;
-  }
-
-  /** Returns the {@code termInfosIndexDivisor}.
-   * 
-   * @see #setReaderTermsIndexDivisor(int) */
-  public int getReaderTermsIndexDivisor() {
-    return readerTermsIndexDivisor;
-  }
   
   /** Returns the {@link OpenMode} set by {@link IndexWriterConfig#setOpenMode(OpenMode)}. */
   public OpenMode getOpenMode() {
@@ -583,8 +484,6 @@ public class LiveIndexWriterConfig {
     sb.append("maxBufferedDocs=").append(getMaxBufferedDocs()).append("\n");
     sb.append("maxBufferedDeleteTerms=").append(getMaxBufferedDeleteTerms()).append("\n");
     sb.append("mergedSegmentWarmer=").append(getMergedSegmentWarmer()).append("\n");
-    sb.append("readerTermsIndexDivisor=").append(getReaderTermsIndexDivisor()).append("\n");
-    sb.append("termIndexInterval=").append(getTermIndexInterval()).append("\n"); // TODO: this should be private to the codec, not settable here
     sb.append("delPolicy=").append(getIndexDeletionPolicy().getClass().getName()).append("\n");
     IndexCommit commit = getIndexCommit();
     sb.append("commit=").append(commit == null ? "null" : commit).append("\n");

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java Tue Aug 13 04:06:18 2013
@@ -23,8 +23,9 @@ import java.util.List;
 import org.apache.lucene.index.MultiTermsEnum.TermsEnumIndex;
 import org.apache.lucene.index.MultiTermsEnum.TermsEnumWithSlice;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.packed.AppendingLongBuffer;
+import org.apache.lucene.util.packed.AppendingPackedLongBuffer;
 import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
+import org.apache.lucene.util.packed.PackedInts;
 
 /**
  * A wrapper for CompositeIndexReader providing access to DocValues.
@@ -277,7 +278,7 @@ public class MultiDocValues {
     // globalOrd -> (globalOrd - segmentOrd)
     final MonotonicAppendingLongBuffer globalOrdDeltas;
     // globalOrd -> sub index
-    final AppendingLongBuffer subIndexes;
+    final AppendingPackedLongBuffer subIndexes;
     // segmentOrd -> (globalOrd - segmentOrd)
     final MonotonicAppendingLongBuffer ordDeltas[];
     
@@ -293,8 +294,8 @@ public class MultiDocValues {
       // create the ordinal mappings by pulling a termsenum over each sub's 
       // unique terms, and walking a multitermsenum over those
       this.owner = owner;
-      globalOrdDeltas = new MonotonicAppendingLongBuffer();
-      subIndexes = new AppendingLongBuffer();
+      globalOrdDeltas = new MonotonicAppendingLongBuffer(PackedInts.COMPACT);
+      subIndexes = new AppendingPackedLongBuffer(PackedInts.COMPACT);
       ordDeltas = new MonotonicAppendingLongBuffer[subs.length];
       for (int i = 0; i < ordDeltas.length; i++) {
         ordDeltas[i] = new MonotonicAppendingLongBuffer();

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/MultiFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/MultiFields.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/MultiFields.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/MultiFields.java Tue Aug 13 04:06:18 2013
@@ -150,7 +150,7 @@ public final class MultiFields extends F
     final Terms terms = getTerms(r, field);
     if (terms != null) {
       final TermsEnum termsEnum = terms.iterator(null);
-      if (termsEnum.seekExact(term, true)) {
+      if (termsEnum.seekExact(term)) {
         return termsEnum.docs(liveDocs, null, flags);
       }
     }
@@ -178,7 +178,7 @@ public final class MultiFields extends F
     final Terms terms = getTerms(r, field);
     if (terms != null) {
       final TermsEnum termsEnum = terms.iterator(null);
-      if (termsEnum.seekExact(term, true)) {
+      if (termsEnum.seekExact(term)) {
         return termsEnum.docsAndPositions(liveDocs, null, flags);
       }
     }

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java Tue Aug 13 04:06:18 2013
@@ -144,7 +144,7 @@ public final class MultiTermsEnum extend
   }
 
   @Override
-  public boolean seekExact(BytesRef term, boolean useCache) throws IOException {
+  public boolean seekExact(BytesRef term) throws IOException {
     queue.clear();
     numTop = 0;
 
@@ -173,13 +173,13 @@ public final class MultiTermsEnum extend
           } else if (cmp < 0) {
             status = false;
           } else {
-            status = currentSubs[i].terms.seekExact(term, useCache);
+            status = currentSubs[i].terms.seekExact(term);
           }
         } else {
           status = false;
         }
       } else {
-        status = currentSubs[i].terms.seekExact(term, useCache);
+        status = currentSubs[i].terms.seekExact(term);
       }
 
       if (status) {
@@ -195,7 +195,7 @@ public final class MultiTermsEnum extend
   }
 
   @Override
-  public SeekStatus seekCeil(BytesRef term, boolean useCache) throws IOException {
+  public SeekStatus seekCeil(BytesRef term) throws IOException {
     queue.clear();
     numTop = 0;
     lastSeekExact = false;
@@ -225,13 +225,13 @@ public final class MultiTermsEnum extend
           } else if (cmp < 0) {
             status = SeekStatus.NOT_FOUND;
           } else {
-            status = currentSubs[i].terms.seekCeil(term, useCache);
+            status = currentSubs[i].terms.seekCeil(term);
           }
         } else {
           status = SeekStatus.END;
         }
       } else {
-        status = currentSubs[i].terms.seekCeil(term, useCache);
+        status = currentSubs[i].terms.seekCeil(term);
       }
 
       if (status == SeekStatus.FOUND) {

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java Tue Aug 13 04:06:18 2013
@@ -23,7 +23,8 @@ import java.util.NoSuchElementException;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.packed.AppendingLongBuffer;
+import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
+import org.apache.lucene.util.packed.PackedInts;
 
 /** Buffers up pending long per doc, then flushes when
  *  segment flushes. */
@@ -31,13 +32,13 @@ class NumericDocValuesWriter extends Doc
 
   private final static long MISSING = 0L;
 
-  private AppendingLongBuffer pending;
+  private AppendingDeltaPackedLongBuffer pending;
   private final Counter iwBytesUsed;
   private long bytesUsed;
   private final FieldInfo fieldInfo;
 
   public NumericDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
-    pending = new AppendingLongBuffer();
+    pending = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
     bytesUsed = pending.ramBytesUsed();
     this.fieldInfo = fieldInfo;
     this.iwBytesUsed = iwBytesUsed;
@@ -89,7 +90,7 @@ class NumericDocValuesWriter extends Doc
   
   // iterates over the values we have in ram
   private class NumericIterator implements Iterator<Number> {
-    final AppendingLongBuffer.Iterator iter = pending.iterator();
+    final AppendingDeltaPackedLongBuffer.Iterator iter = pending.iterator();
     final int size = (int)pending.size();
     final int maxDoc;
     int upto;

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java Tue Aug 13 04:06:18 2013
@@ -43,16 +43,6 @@ class ReadersAndLiveDocs {
   // Set once (null, and then maybe set, and never set again):
   private SegmentReader reader;
 
-  // TODO: it's sometimes wasteful that we hold open two
-  // separate SRs (one for merging one for
-  // reading)... maybe just use a single SR?  The gains of
-  // not loading the terms index (for merging in the
-  // non-NRT case) are far less now... and if the app has
-  // any deletes it'll open real readers anyway.
-
-  // Set once (null, and then maybe set, and never set again):
-  private SegmentReader mergeReader;
-
   // Holds the current shared (readable and writable
   // liveDocs).  This is null when there are no deleted
   // docs, and it's copy-on-write (cloned whenever we need
@@ -118,7 +108,7 @@ class ReadersAndLiveDocs {
 
     if (reader == null) {
       // We steal returned ref:
-      reader = new SegmentReader(info, writer.getConfig().getReaderTermsIndexDivisor(), context);
+      reader = new SegmentReader(info, context);
       if (liveDocs == null) {
         liveDocs = reader.getLiveDocs();
       }
@@ -131,37 +121,6 @@ class ReadersAndLiveDocs {
     return reader;
   }
 
-  // Get reader for merging (does not load the terms
-  // index):
-  public synchronized SegmentReader getMergeReader(IOContext context) throws IOException {
-    //System.out.println("  livedocs=" + rld.liveDocs);
-
-    if (mergeReader == null) {
-
-      if (reader != null) {
-        // Just use the already opened non-merge reader
-        // for merging.  In the NRT case this saves us
-        // pointless double-open:
-        //System.out.println("PROMOTE non-merge reader seg=" + rld.info);
-        // Ref for us:
-        reader.incRef();
-        mergeReader = reader;
-        //System.out.println(Thread.currentThread().getName() + ": getMergeReader share seg=" + info.name);
-      } else {
-        //System.out.println(Thread.currentThread().getName() + ": getMergeReader seg=" + info.name);
-        // We steal returned ref:
-        mergeReader = new SegmentReader(info, -1, context);
-        if (liveDocs == null) {
-          liveDocs = mergeReader.getLiveDocs();
-        }
-      }
-    }
-
-    // Ref for caller
-    mergeReader.incRef();
-    return mergeReader;
-  }
-
   public synchronized void release(SegmentReader sr) throws IOException {
     assert info == sr.getSegmentInfo();
     sr.decRef();
@@ -185,23 +144,12 @@ class ReadersAndLiveDocs {
   public synchronized void dropReaders() throws IOException {
     // TODO: can we somehow use IOUtils here...?  problem is
     // we are calling .decRef not .close)...
-    try {
-      if (reader != null) {
-        //System.out.println("  pool.drop info=" + info + " rc=" + reader.getRefCount());
-        try {
-          reader.decRef();
-        } finally {
-          reader = null;
-        }
-      }
-    } finally {
-      if (mergeReader != null) {
-        //System.out.println("  pool.drop info=" + info + " merge rc=" + mergeReader.getRefCount());
-        try {
-          mergeReader.decRef();
-        } finally {
-          mergeReader = null;
-        }
+    if (reader != null) {
+      //System.out.println("  pool.drop info=" + info + " rc=" + reader.getRefCount());
+      try {
+        reader.decRef();
+      } finally {
+        reader = null;
       }
     }
 

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java Tue Aug 13 04:06:18 2013
@@ -57,8 +57,6 @@ final class SegmentCoreReaders {
   final DocValuesProducer dvProducer;
   final DocValuesProducer normsProducer;
 
-  final int termsIndexDivisor;
-  
   private final SegmentReader owner;
   
   final StoredFieldsReader fieldsReaderOrig;
@@ -100,11 +98,7 @@ final class SegmentCoreReaders {
   private final Set<CoreClosedListener> coreClosedListeners = 
       Collections.synchronizedSet(new LinkedHashSet<CoreClosedListener>());
   
-  SegmentCoreReaders(SegmentReader owner, Directory dir, SegmentInfoPerCommit si, IOContext context, int termsIndexDivisor) throws IOException {
-    
-    if (termsIndexDivisor == 0) {
-      throw new IllegalArgumentException("indexDivisor must be < 0 (don't load terms index) or greater than 0 (got 0)");
-    }
+  SegmentCoreReaders(SegmentReader owner, Directory dir, SegmentInfoPerCommit si, IOContext context) throws IOException {
     
     final Codec codec = si.info.getCodec();
     final Directory cfsDir; // confusing name: if (cfs) its the cfsdir, otherwise its the segment's directory.
@@ -120,9 +114,8 @@ final class SegmentCoreReaders {
       }
       fieldInfos = codec.fieldInfosFormat().getFieldInfosReader().read(cfsDir, si.info.name, IOContext.READONCE);
 
-      this.termsIndexDivisor = termsIndexDivisor;
       final PostingsFormat format = codec.postingsFormat();
-      final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, si.info, fieldInfos, context, termsIndexDivisor);
+      final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, si.info, fieldInfos, context);
       // Ask codec for its Fields
       fields = format.fieldsProducer(segmentReadState);
       assert fields != null;

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java Tue Aug 13 04:06:18 2013
@@ -42,7 +42,6 @@ import org.apache.lucene.util.InfoStream
  */
 final class SegmentMerger {
   private final Directory directory;
-  private final int termIndexInterval;
 
   private final Codec codec;
   
@@ -52,11 +51,10 @@ final class SegmentMerger {
   private final FieldInfos.Builder fieldInfosBuilder;
 
   // note, just like in codec apis Directory 'dir' is NOT the same as segmentInfo.dir!!
-  SegmentMerger(List<AtomicReader> readers, SegmentInfo segmentInfo, InfoStream infoStream, Directory dir, int termIndexInterval,
+  SegmentMerger(List<AtomicReader> readers, SegmentInfo segmentInfo, InfoStream infoStream, Directory dir,
                 MergeState.CheckAbort checkAbort, FieldInfos.FieldNumbers fieldNumbers, IOContext context) {
     mergeState = new MergeState(readers, segmentInfo, infoStream, checkAbort);
     directory = dir;
-    this.termIndexInterval = termIndexInterval;
     this.codec = segmentInfo.getCodec();
     this.context = context;
     this.fieldInfosBuilder = new FieldInfos.Builder(fieldNumbers);
@@ -91,7 +89,7 @@ final class SegmentMerger {
     assert numMerged == mergeState.segmentInfo.getDocCount();
 
     final SegmentWriteState segmentWriteState = new SegmentWriteState(mergeState.infoStream, directory, mergeState.segmentInfo,
-                                                                      mergeState.fieldInfos, termIndexInterval, null, context);
+                                                                      mergeState.fieldInfos, null, context);
     if (mergeState.infoStream.isEnabled("SM")) {
       t0 = System.nanoTime();
     }

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SegmentReadState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SegmentReadState.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SegmentReadState.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SegmentReadState.java Tue Aug 13 04:06:18 2013
@@ -41,17 +41,6 @@ public class SegmentReadState {
    *  Directory#openInput(String,IOContext)}. */
   public final IOContext context;
 
-  /** The {@code termInfosIndexDivisor} to use, if
-   *  appropriate (not all {@link PostingsFormat}s support
-   *  it; in particular the current default does not).
-   *
-   * <p>  NOTE: if this is &lt; 0, that means "defer terms index
-   *  load until needed".  But if the codec must load the
-   *  terms index on init (preflex is the only once currently
-   *  that must do so), then it should negate this value to
-   *  get the app's terms divisor */
-  public int termsIndexDivisor;
-
   /** Unique suffix for any postings files read for this
    *  segment.  {@link PerFieldPostingsFormat} sets this for
    *  each of the postings formats it wraps.  If you create
@@ -62,8 +51,8 @@ public class SegmentReadState {
 
   /** Create a {@code SegmentReadState}. */
   public SegmentReadState(Directory dir, SegmentInfo info,
-      FieldInfos fieldInfos, IOContext context, int termsIndexDivisor) {
-    this(dir, info, fieldInfos,  context, termsIndexDivisor, "");
+      FieldInfos fieldInfos, IOContext context) {
+    this(dir, info, fieldInfos,  context, "");
   }
   
   /** Create a {@code SegmentReadState}. */
@@ -71,13 +60,11 @@ public class SegmentReadState {
                           SegmentInfo info,
                           FieldInfos fieldInfos,
                           IOContext context,
-                          int termsIndexDivisor,
                           String segmentSuffix) {
     this.directory = dir;
     this.segmentInfo = info;
     this.fieldInfos = fieldInfos;
     this.context = context;
-    this.termsIndexDivisor = termsIndexDivisor;
     this.segmentSuffix = segmentSuffix;
   }
 
@@ -88,7 +75,6 @@ public class SegmentReadState {
     this.segmentInfo = other.segmentInfo;
     this.fieldInfos = other.fieldInfos;
     this.context = other.context;
-    this.termsIndexDivisor = other.termsIndexDivisor;
     this.segmentSuffix = newSegmentSuffix;
   }
 }

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java Tue Aug 13 04:06:18 2013
@@ -51,9 +51,9 @@ public final class SegmentReader extends
    * @throws IOException if there is a low-level IO error
    */
   // TODO: why is this public?
-  public SegmentReader(SegmentInfoPerCommit si, int termInfosIndexDivisor, IOContext context) throws IOException {
+  public SegmentReader(SegmentInfoPerCommit si, IOContext context) throws IOException {
     this.si = si;
-    core = new SegmentCoreReaders(this, si.info.dir, si, context, termInfosIndexDivisor);
+    core = new SegmentCoreReaders(this, si.info.dir, si, context);
     boolean success = false;
     try {
       if (si.hasDeletions()) {
@@ -217,12 +217,6 @@ public final class SegmentReader extends
     return this;
   }
 
-  /** Returns term infos index divisor originally passed to
-   *  {@link #SegmentReader(SegmentInfoPerCommit, int, IOContext)}. */
-  public int getTermInfosIndexDivisor() {
-    return core.termsIndexDivisor;
-  }
-
   @Override
   public NumericDocValues getNumericDocValues(String field) throws IOException {
     ensureOpen();

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java Tue Aug 13 04:06:18 2013
@@ -66,13 +66,6 @@ public class SegmentWriteState {
    *  write/read must be derived using this suffix (use
    *  {@link IndexFileNames#segmentFileName(String,String,String)}). */
   public final String segmentSuffix;
-
-  /** Expert: The fraction of terms in the "dictionary" which should be stored
-   * in RAM.  Smaller values use more memory, but make searching slightly
-   * faster, while larger values use less memory and make searching slightly
-   * slower.  Searching is typically not dominated by dictionary lookup, so
-   * tweaking this is rarely useful.*/
-  public int termIndexInterval;                   // TODO: this should be private to the codec, not settable here or in IWC
   
   /** {@link IOContext} for all writes; you should pass this
    *  to {@link Directory#createOutput(String,IOContext)}. */
@@ -80,26 +73,22 @@ public class SegmentWriteState {
 
   /** Sole constructor. */
   public SegmentWriteState(InfoStream infoStream, Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos,
-      int termIndexInterval, BufferedDeletes segDeletes, IOContext context) {
+      BufferedDeletes segDeletes, IOContext context) {
     this.infoStream = infoStream;
     this.segDeletes = segDeletes;
     this.directory = directory;
     this.segmentInfo = segmentInfo;
     this.fieldInfos = fieldInfos;
-    this.termIndexInterval = termIndexInterval;
     segmentSuffix = "";
     this.context = context;
   }
   
-  /**
-   * Create a shallow {@link SegmentWriteState} copy final a format ID
-   */
+  /** Create a shallow copy of {@link SegmentWriteState} with a new segment suffix. */
   public SegmentWriteState(SegmentWriteState state, String segmentSuffix) {
     infoStream = state.infoStream;
     directory = state.directory;
     segmentInfo = state.segmentInfo;
     fieldInfos = state.fieldInfos;
-    termIndexInterval = state.termIndexInterval;
     context = state.context;
     this.segmentSuffix = segmentSuffix;
     segDeletes = state.segDeletes;

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java Tue Aug 13 04:06:18 2013
@@ -37,7 +37,7 @@ class SortedDocValuesTermsEnum extends T
   }
 
   @Override
-  public SeekStatus seekCeil(BytesRef text, boolean useCache /* ignored */) throws IOException {
+  public SeekStatus seekCeil(BytesRef text) throws IOException {
     int ord = values.lookupTerm(text);
     if (ord >= 0) {
       currentOrd = ord;
@@ -61,7 +61,7 @@ class SortedDocValuesTermsEnum extends T
   }
 
   @Override
-  public boolean seekExact(BytesRef text, boolean useCache) throws IOException {
+  public boolean seekExact(BytesRef text) throws IOException {
     int ord = values.lookupTerm(text);
     if (ord >= 0) {
       term.offset = 0;

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java Tue Aug 13 04:06:18 2013
@@ -30,13 +30,14 @@ import org.apache.lucene.util.BytesRefHa
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.packed.AppendingLongBuffer;
+import org.apache.lucene.util.packed.AppendingPackedLongBuffer;
+import org.apache.lucene.util.packed.PackedInts;
 
 /** Buffers up pending byte[] per doc, deref and sorting via
  *  int ord, then flushes when segment flushes. */
 class SortedDocValuesWriter extends DocValuesWriter {
   final BytesRefHash hash;
-  private AppendingLongBuffer pending;
+  private AppendingPackedLongBuffer pending;
   private final Counter iwBytesUsed;
   private long bytesUsed; // this currently only tracks differences in 'pending'
   private final FieldInfo fieldInfo;
@@ -51,7 +52,7 @@ class SortedDocValuesWriter extends DocV
             new ByteBlockPool.DirectTrackingAllocator(iwBytesUsed)),
             BytesRefHash.DEFAULT_CAPACITY,
             new DirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY, iwBytesUsed));
-    pending = new AppendingLongBuffer();
+    pending = new AppendingPackedLongBuffer(PackedInts.COMPACT);
     bytesUsed = pending.ramBytesUsed();
     iwBytesUsed.addAndGet(bytesUsed);
   }
@@ -176,7 +177,7 @@ class SortedDocValuesWriter extends DocV
   
   // iterates over the ords for each doc we have in ram
   private class OrdsIterator implements Iterator<Number> {
-    final AppendingLongBuffer.Iterator iter = pending.iterator();
+    final AppendingPackedLongBuffer.Iterator iter = pending.iterator();
     final int ordMap[];
     final int maxDoc;
     int docUpto;

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java Tue Aug 13 04:06:18 2013
@@ -37,7 +37,7 @@ class SortedSetDocValuesTermsEnum extend
   }
 
   @Override
-  public SeekStatus seekCeil(BytesRef text, boolean useCache /* ignored */) throws IOException {
+  public SeekStatus seekCeil(BytesRef text) throws IOException {
     long ord = values.lookupTerm(text);
     if (ord >= 0) {
       currentOrd = ord;
@@ -61,7 +61,7 @@ class SortedSetDocValuesTermsEnum extend
   }
 
   @Override
-  public boolean seekExact(BytesRef text, boolean useCache) throws IOException {
+  public boolean seekExact(BytesRef text) throws IOException {
     long ord = values.lookupTerm(text);
     if (ord >= 0) {
       term.offset = 0;

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java Tue Aug 13 04:06:18 2013
@@ -32,14 +32,16 @@ import org.apache.lucene.util.BytesRefHa
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.packed.AppendingLongBuffer;
+import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
+import org.apache.lucene.util.packed.AppendingPackedLongBuffer;
+import org.apache.lucene.util.packed.PackedInts;
 
 /** Buffers up pending byte[]s per doc, deref and sorting via
  *  int ord, then flushes when segment flushes. */
 class SortedSetDocValuesWriter extends DocValuesWriter {
   final BytesRefHash hash;
-  private AppendingLongBuffer pending; // stream of all termIDs
-  private AppendingLongBuffer pendingCounts; // termIDs per doc
+  private AppendingPackedLongBuffer pending; // stream of all termIDs
+  private AppendingDeltaPackedLongBuffer pendingCounts; // termIDs per doc
   private final Counter iwBytesUsed;
   private long bytesUsed; // this only tracks differences in 'pending' and 'pendingCounts'
   private final FieldInfo fieldInfo;
@@ -56,8 +58,8 @@ class SortedSetDocValuesWriter extends D
             new ByteBlockPool.DirectTrackingAllocator(iwBytesUsed)),
             BytesRefHash.DEFAULT_CAPACITY,
             new DirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY, iwBytesUsed));
-    pending = new AppendingLongBuffer();
-    pendingCounts = new AppendingLongBuffer();
+    pending = new AppendingPackedLongBuffer(PackedInts.COMPACT);
+    pendingCounts = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
     bytesUsed = pending.ramBytesUsed() + pendingCounts.ramBytesUsed();
     iwBytesUsed.addAndGet(bytesUsed);
   }
@@ -224,8 +226,8 @@ class SortedSetDocValuesWriter extends D
   
   // iterates over the ords for each doc we have in ram
   private class OrdsIterator implements Iterator<Number> {
-    final AppendingLongBuffer.Iterator iter = pending.iterator();
-    final AppendingLongBuffer.Iterator counts = pendingCounts.iterator();
+    final AppendingPackedLongBuffer.Iterator iter = pending.iterator();
+    final AppendingDeltaPackedLongBuffer.Iterator counts = pendingCounts.iterator();
     final int ordMap[];
     final long numOrds;
     long ordUpto;
@@ -273,7 +275,7 @@ class SortedSetDocValuesWriter extends D
   }
   
   private class OrdCountIterator implements Iterator<Number> {
-    final AppendingLongBuffer.Iterator iter = pendingCounts.iterator();
+    final AppendingDeltaPackedLongBuffer.Iterator iter = pendingCounts.iterator();
     final int maxDoc;
     int docUpto;
     

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java Tue Aug 13 04:06:18 2013
@@ -33,22 +33,19 @@ final class StandardDirectoryReader exte
 
   private final IndexWriter writer;
   private final SegmentInfos segmentInfos;
-  private final int termInfosIndexDivisor;
   private final boolean applyAllDeletes;
   
   /** called only from static open() methods */
   StandardDirectoryReader(Directory directory, AtomicReader[] readers, IndexWriter writer,
-    SegmentInfos sis, int termInfosIndexDivisor, boolean applyAllDeletes) {
+    SegmentInfos sis, boolean applyAllDeletes) {
     super(directory, readers);
     this.writer = writer;
     this.segmentInfos = sis;
-    this.termInfosIndexDivisor = termInfosIndexDivisor;
     this.applyAllDeletes = applyAllDeletes;
   }
 
   /** called from DirectoryReader.open(...) methods */
-  static DirectoryReader open(final Directory directory, final IndexCommit commit,
-                          final int termInfosIndexDivisor) throws IOException {
+  static DirectoryReader open(final Directory directory, final IndexCommit commit) throws IOException {
     return (DirectoryReader) new SegmentInfos.FindSegmentsFile(directory) {
       @Override
       protected Object doBody(String segmentFileName) throws IOException {
@@ -59,7 +56,7 @@ final class StandardDirectoryReader exte
           IOException prior = null;
           boolean success = false;
           try {
-            readers[i] = new SegmentReader(sis.info(i), termInfosIndexDivisor, IOContext.READ);
+            readers[i] = new SegmentReader(sis.info(i), IOContext.READ);
             success = true;
           } catch(IOException ex) {
             prior = ex;
@@ -68,7 +65,7 @@ final class StandardDirectoryReader exte
               IOUtils.closeWhileHandlingException(prior, readers);
           }
         }
-        return new StandardDirectoryReader(directory, readers, null, sis, termInfosIndexDivisor, false);
+        return new StandardDirectoryReader(directory, readers, null, sis, false);
       }
     }.run(commit);
   }
@@ -119,12 +116,11 @@ final class StandardDirectoryReader exte
       }
     }
     return new StandardDirectoryReader(dir, readers.toArray(new SegmentReader[readers.size()]),
-      writer, segmentInfos, writer.getConfig().getReaderTermsIndexDivisor(), applyAllDeletes);
+      writer, segmentInfos, applyAllDeletes);
   }
 
   /** This constructor is only used for {@link #doOpenIfChanged(SegmentInfos)} */
-  private static DirectoryReader open(Directory directory, SegmentInfos infos, List<? extends AtomicReader> oldReaders,
-    int termInfosIndexDivisor) throws IOException {
+  private static DirectoryReader open(Directory directory, SegmentInfos infos, List<? extends AtomicReader> oldReaders) throws IOException {
 
     // we put the old SegmentReaders in a map, that allows us
     // to lookup a reader using its segment name
@@ -162,7 +158,7 @@ final class StandardDirectoryReader exte
         if (newReaders[i] == null || infos.info(i).info.getUseCompoundFile() != newReaders[i].getSegmentInfo().info.getUseCompoundFile()) {
 
           // this is a new reader; in case we hit an exception we can close it safely
-          newReader = new SegmentReader(infos.info(i), termInfosIndexDivisor, IOContext.READ);
+          newReader = new SegmentReader(infos.info(i), IOContext.READ);
           readerShared[i] = false;
           newReaders[i] = newReader;
         } else {
@@ -212,7 +208,7 @@ final class StandardDirectoryReader exte
         }
       }
     }    
-    return new StandardDirectoryReader(directory, newReaders, null, infos, termInfosIndexDivisor, false);
+    return new StandardDirectoryReader(directory, newReaders, null, infos, false);
   }
 
   @Override
@@ -313,7 +309,7 @@ final class StandardDirectoryReader exte
   }
 
   DirectoryReader doOpenIfChanged(SegmentInfos infos) throws IOException {
-    return StandardDirectoryReader.open(directory, infos, getSequentialSubReaders(), termInfosIndexDivisor);
+    return StandardDirectoryReader.open(directory, infos, getSequentialSubReaders());
   }
 
   @Override

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/TermContext.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/TermContext.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/TermContext.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/TermContext.java Tue Aug 13 04:06:18 2013
@@ -78,7 +78,7 @@ public final class TermContext {
    * <p>
    * Note: the given context must be a top-level context.
    */
-  public static TermContext build(IndexReaderContext context, Term term, boolean cache)
+  public static TermContext build(IndexReaderContext context, Term term)
       throws IOException {
     assert context != null && context.isTopLevel;
     final String field = term.field();
@@ -92,7 +92,7 @@ public final class TermContext {
         final Terms terms = fields.terms(field);
         if (terms != null) {
           final TermsEnum termsEnum = terms.iterator(null);
-          if (termsEnum.seekExact(bytes, cache)) { 
+          if (termsEnum.seekExact(bytes)) { 
             final TermState termState = termsEnum.termState();
             //if (DEBUG) System.out.println("    found");
             perReaderTermState.register(termState, ctx.ord, termsEnum.docFreq(), termsEnum.totalTermFreq());

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java Tue Aug 13 04:06:18 2013
@@ -76,31 +76,31 @@ final class TermVectorsConsumerPerField 
             doVectorPayloads |= field.fieldType().storeTermVectorPayloads();
           } else if (field.fieldType().storeTermVectorPayloads()) {
             // TODO: move this check somewhere else, and impl the other missing ones
-            throw new IllegalArgumentException("cannot index term vector payloads for field: " + field + " without term vector positions");
+            throw new IllegalArgumentException("cannot index term vector payloads without term vector positions (field=\"" + field.name() + "\")");
           }
         } else {
           if (field.fieldType().storeTermVectorOffsets()) {
-            throw new IllegalArgumentException("cannot index term vector offsets when term vectors are not indexed (field=\"" + field.name());
+            throw new IllegalArgumentException("cannot index term vector offsets when term vectors are not indexed (field=\"" + field.name() + "\")");
           }
           if (field.fieldType().storeTermVectorPositions()) {
-            throw new IllegalArgumentException("cannot index term vector positions when term vectors are not indexed (field=\"" + field.name());
+            throw new IllegalArgumentException("cannot index term vector positions when term vectors are not indexed (field=\"" + field.name() + "\")");
           }
           if (field.fieldType().storeTermVectorPayloads()) {
-            throw new IllegalArgumentException("cannot index term vector payloads when term vectors are not indexed (field=\"" + field.name());
+            throw new IllegalArgumentException("cannot index term vector payloads when term vectors are not indexed (field=\"" + field.name() + "\")");
           }
         }
       } else {
         if (field.fieldType().storeTermVectors()) {
-          throw new IllegalArgumentException("cannot index term vectors when field is not indexed (field=\"" + field.name());
+          throw new IllegalArgumentException("cannot index term vectors when field is not indexed (field=\"" + field.name() + "\")");
         }
         if (field.fieldType().storeTermVectorOffsets()) {
-          throw new IllegalArgumentException("cannot index term vector offsets when field is not indexed (field=\"" + field.name());
+          throw new IllegalArgumentException("cannot index term vector offsets when field is not indexed (field=\"" + field.name() + "\")");
         }
         if (field.fieldType().storeTermVectorPositions()) {
-          throw new IllegalArgumentException("cannot index term vector positions when field is not indexed (field=\"" + field.name());
+          throw new IllegalArgumentException("cannot index term vector positions when field is not indexed (field=\"" + field.name() + "\")");
         }
         if (field.fieldType().storeTermVectorPayloads()) {
-          throw new IllegalArgumentException("cannot index term vector payloads when field is not indexed (field=\"" + field.name());
+          throw new IllegalArgumentException("cannot index term vector payloads when field is not indexed (field=\"" + field.name() + "\")");
         }
       }
     }

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java Tue Aug 13 04:06:18 2013
@@ -26,7 +26,7 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefIterator;
 
 /** Iterator to seek ({@link #seekCeil(BytesRef)}, {@link
- * #seekExact(BytesRef,boolean)}) or step through ({@link
+ * #seekExact(BytesRef)}) or step through ({@link
  * #next} terms to obtain frequency information ({@link
  * #docFreq}), {@link DocsEnum} or {@link
  * DocsAndPositionsEnum} for the current term ({@link
@@ -70,24 +70,17 @@ public abstract class TermsEnum implemen
    *  true if the term is found.  If this returns false, the
    *  enum is unpositioned.  For some codecs, seekExact may
    *  be substantially faster than {@link #seekCeil}. */
-  public boolean seekExact(BytesRef text, boolean useCache) throws IOException {
-    return seekCeil(text, useCache) == SeekStatus.FOUND;
+  public boolean seekExact(BytesRef text) throws IOException {
+    return seekCeil(text) == SeekStatus.FOUND;
   }
 
-  /** Expert: just like {@link #seekCeil(BytesRef)} but allows
-   *  you to control whether the implementation should
-   *  attempt to use its term cache (if it uses one). */
-  public abstract SeekStatus seekCeil(BytesRef text, boolean useCache) throws IOException;
-
   /** Seeks to the specified term, if it exists, or to the
    *  next (ceiling) term.  Returns SeekStatus to
    *  indicate whether exact term was found, a different
    *  term was found, or EOF was hit.  The target term may
    *  be before or after the current term.  If this returns
    *  SeekStatus.END, the enum is unpositioned. */
-  public final SeekStatus seekCeil(BytesRef text) throws IOException {
-    return seekCeil(text, true);
-  }
+  public abstract SeekStatus seekCeil(BytesRef text) throws IOException;
 
   /** Seeks to the specified term by ordinal (position) as
    *  previously returned by {@link #ord}.  The target ord
@@ -117,7 +110,7 @@ public abstract class TermsEnum implemen
    * @param state the {@link TermState}
    * */
   public void seekExact(BytesRef term, TermState state) throws IOException {
-    if (!seekExact(term, true)) {
+    if (!seekExact(term)) {
       throw new IllegalArgumentException("term=" + term + " does not exist");
     }
   }
@@ -226,7 +219,7 @@ public abstract class TermsEnum implemen
    */
   public static final TermsEnum EMPTY = new TermsEnum() {    
     @Override
-    public SeekStatus seekCeil(BytesRef term, boolean useCache) { return SeekStatus.END; }
+    public SeekStatus seekCeil(BytesRef term) { return SeekStatus.END; }
     
     @Override
     public void seekExact(long ord) {}

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Tue Aug 13 04:06:18 2013
@@ -303,13 +303,13 @@ public class FuzzyTermsEnum extends Term
   }
   
   @Override
-  public boolean seekExact(BytesRef text, boolean useCache) throws IOException {
-    return actualEnum.seekExact(text, useCache);
+  public boolean seekExact(BytesRef text) throws IOException {
+    return actualEnum.seekExact(text);
   }
 
   @Override
-  public SeekStatus seekCeil(BytesRef text, boolean useCache) throws IOException {
-    return actualEnum.seekCeil(text, useCache);
+  public SeekStatus seekCeil(BytesRef text) throws IOException {
+    return actualEnum.seekCeil(text);
   }
   
   @Override

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java Tue Aug 13 04:06:18 2013
@@ -430,6 +430,10 @@ public class IndexSearcher {
     if (limit == 0) {
       limit = 1;
     }
+    if (after != null && after.doc >= limit) {
+      throw new IllegalArgumentException("after.doc exceeds the number of documents in that reader: after.doc="
+          + after.doc + " limit=" + limit);
+    }
     nDocs = Math.min(nDocs, limit);
     
     if (executor == null) {
@@ -440,8 +444,7 @@ public class IndexSearcher {
       final ExecutionHelper<TopDocs> runner = new ExecutionHelper<TopDocs>(executor);
     
       for (int i = 0; i < leafSlices.length; i++) { // search each sub
-        runner.submit(
-                      new SearcherCallableNoSort(lock, this, leafSlices[i], weight, after, nDocs, hq));
+        runner.submit(new SearcherCallableNoSort(lock, this, leafSlices[i], weight, after, nDocs, hq));
       }
 
       int totalHits = 0;
@@ -920,7 +923,7 @@ public class IndexSearcher {
    */
   public TermStatistics termStatistics(Term term, TermContext context) throws IOException {
     return new TermStatistics(term.bytes(), context.docFreq(), context.totalTermFreq());
-  };
+  }
   
   /**
    * Returns {@link CollectionStatistics} for a field.

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java Tue Aug 13 04:06:18 2013
@@ -149,7 +149,7 @@ public class MultiPhraseQuery extends Qu
         for (Term term: terms) {
           TermContext termContext = termContexts.get(term);
           if (termContext == null) {
-            termContext = TermContext.build(context, term, true);
+            termContext = TermContext.build(context, term);
             termContexts.put(term, termContext);
           }
           allTermStats.add(searcher.termStatistics(term, termContext));

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java Tue Aug 13 04:06:18 2013
@@ -37,7 +37,6 @@ import org.apache.lucene.search.similari
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ToStringUtils;
 
 /** A Query that matches documents containing a particular sequence of terms.
@@ -218,7 +217,7 @@ public class PhraseQuery extends Query {
       TermStatistics termStats[] = new TermStatistics[terms.size()];
       for (int i = 0; i < terms.size(); i++) {
         final Term term = terms.get(i);
-        states[i] = TermContext.build(context, term, true);
+        states[i] = TermContext.build(context, term);
         termStats[i] = searcher.termStatistics(term, states[i]);
       }
       stats = similarity.computeWeight(getBoost(), searcher.collectionStatistics(field), termStats);
@@ -269,7 +268,7 @@ public class PhraseQuery extends Query {
         // PhraseQuery on a field that did not index
         // positions.
         if (postingsEnum == null) {
-          assert te.seekExact(t.bytes(), false) : "termstate found but no term exists in reader";
+          assert te.seekExact(t.bytes()) : "termstate found but no term exists in reader";
           // term does exist, but has no positions
           throw new IllegalStateException("field \"" + t.field() + "\" was indexed without position data; cannot run PhraseQuery (term=" + t.text() + ")");
         }

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/TermQuery.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/TermQuery.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/TermQuery.java Tue Aug 13 04:06:18 2013
@@ -163,7 +163,7 @@ public class TermQuery extends Query {
     final TermContext termState;
     if (perReaderTermState == null || perReaderTermState.topReaderContext != context) {
       // make TermQuery single-pass if we don't have a PRTS or if the context differs!
-      termState = TermContext.build(context, term, true); // cache term lookups!
+      termState = TermContext.build(context, term);
     } else {
      // PRTS was pre-build for this IS
      termState = this.perReaderTermState;

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java Tue Aug 13 04:06:18 2013
@@ -182,7 +182,7 @@ public class PayloadSpanUtil {
     TreeSet<Term> terms = new TreeSet<Term>();
     query.extractTerms(terms);
     for (Term term : terms) {
-      termContexts.put(term, TermContext.build(context, term, true));
+      termContexts.put(term, TermContext.build(context, term));
     }
     for (AtomicReaderContext atomicReaderContext : context.leaves()) {
       final Spans spans = query.getSpans(atomicReaderContext, atomicReaderContext.reader().getLiveDocs(), termContexts);

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java Tue Aug 13 04:06:18 2013
@@ -22,6 +22,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermContext;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.InPlaceMergeSorter;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -72,13 +73,19 @@ public class NearSpansOrdered extends Sp
   private List<byte[]> matchPayload;
 
   private final Spans[] subSpansByDoc;
-  private final Comparator<Spans> spanDocComparator = new Comparator<Spans>() {
+  // Even though the array is probably almost sorted, InPlaceMergeSorter will likely
+  // perform better since it has a lower overhead than TimSorter for small arrays
+  private final InPlaceMergeSorter sorter = new InPlaceMergeSorter() {
     @Override
-    public int compare(Spans o1, Spans o2) {
-      return o1.doc() - o2.doc();
+    protected void swap(int i, int j) {
+      ArrayUtil.swap(subSpansByDoc, i, j);
+    }
+    @Override
+    protected int compare(int i, int j) {
+      return subSpansByDoc[i].doc() - subSpansByDoc[j].doc();
     }
   };
-  
+
   private SpanNearQuery query;
   private boolean collectPayloads = true;
   
@@ -204,7 +211,7 @@ public class NearSpansOrdered extends Sp
 
   /** Advance the subSpans to the same document */
   private boolean toSameDoc() throws IOException {
-    ArrayUtil.timSort(subSpansByDoc, spanDocComparator);
+    sorter.sort(0, subSpansByDoc.length);
     int firstIndex = 0;
     int maxDoc = subSpansByDoc[subSpansByDoc.length - 1].doc();
     while (subSpansByDoc[firstIndex].doc() != maxDoc) {

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/spans/SpanNotQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/spans/SpanNotQuery.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/spans/SpanNotQuery.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/spans/SpanNotQuery.java Tue Aug 13 04:06:18 2013
@@ -31,16 +31,36 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.Set;
 
-/** Removes matches which overlap with another SpanQuery. */
+/** Removes matches which overlap with another SpanQuery or 
+ * within a x tokens before or y tokens after another SpanQuery. */
 public class SpanNotQuery extends SpanQuery implements Cloneable {
   private SpanQuery include;
   private SpanQuery exclude;
+  private final int pre;
+  private final int post;
 
   /** Construct a SpanNotQuery matching spans from <code>include</code> which
    * have no overlap with spans from <code>exclude</code>.*/
   public SpanNotQuery(SpanQuery include, SpanQuery exclude) {
+     this(include, exclude, 0, 0);
+  }
+
+  
+  /** Construct a SpanNotQuery matching spans from <code>include</code> which
+   * have no overlap with spans from <code>exclude</code> within 
+   * <code>dist</code> tokens of <code>include</code>. */
+  public SpanNotQuery(SpanQuery include, SpanQuery exclude, int dist) {
+     this(include, exclude, dist, dist);
+  }
+  
+  /** Construct a SpanNotQuery matching spans from <code>include</code> which
+   * have no overlap with spans from <code>exclude</code> within 
+   * <code>pre</code> tokens before or <code>post</code> tokens of <code>include</code>. */
+  public SpanNotQuery(SpanQuery include, SpanQuery exclude, int pre, int post) {
     this.include = include;
     this.exclude = exclude;
+    this.pre = (pre >=0) ? pre : 0;
+    this.post = (post >= 0) ? post : 0;
 
     if (!include.getField().equals(exclude.getField()))
       throw new IllegalArgumentException("Clauses must have same field.");
@@ -65,6 +85,10 @@ public class SpanNotQuery extends SpanQu
     buffer.append(include.toString(field));
     buffer.append(", ");
     buffer.append(exclude.toString(field));
+    buffer.append(", ");
+    buffer.append(Integer.toString(pre));
+    buffer.append(", ");
+    buffer.append(Integer.toString(post));
     buffer.append(")");
     buffer.append(ToStringUtils.boost(getBoost()));
     return buffer.toString();
@@ -72,7 +96,8 @@ public class SpanNotQuery extends SpanQu
 
   @Override
   public SpanNotQuery clone() {
-    SpanNotQuery spanNotQuery = new SpanNotQuery((SpanQuery)include.clone(),(SpanQuery) exclude.clone());
+    SpanNotQuery spanNotQuery = new SpanNotQuery((SpanQuery)include.clone(),
+          (SpanQuery) exclude.clone(), pre, post);
     spanNotQuery.setBoost(getBoost());
     return  spanNotQuery;
   }
@@ -98,13 +123,13 @@ public class SpanNotQuery extends SpanQu
 
             while (moreExclude                    // while exclude is before
                    && includeSpans.doc() == excludeSpans.doc()
-                   && excludeSpans.end() <= includeSpans.start()) {
+                   && excludeSpans.end() <= includeSpans.start() - pre) {
               moreExclude = excludeSpans.next();  // increment exclude
             }
 
             if (!moreExclude                      // if no intersection
                 || includeSpans.doc() != excludeSpans.doc()
-                || includeSpans.end() <= excludeSpans.start())
+                || includeSpans.end()+post <= excludeSpans.start())
               break;                              // we found a match
 
             moreInclude = includeSpans.next();    // intersected: keep scanning
@@ -126,13 +151,13 @@ public class SpanNotQuery extends SpanQu
 
           while (moreExclude                      // while exclude is before
                  && includeSpans.doc() == excludeSpans.doc()
-                 && excludeSpans.end() <= includeSpans.start()) {
+                 && excludeSpans.end() <= includeSpans.start()-pre) {
             moreExclude = excludeSpans.next();    // increment exclude
           }
 
           if (!moreExclude                      // if no intersection
                 || includeSpans.doc() != excludeSpans.doc()
-                || includeSpans.end() <= excludeSpans.start())
+                || includeSpans.end()+post <= excludeSpans.start())
             return true;                          // we found a match
 
           return next();                          // scan to next match
@@ -199,23 +224,28 @@ public class SpanNotQuery extends SpanQu
     /** Returns true iff <code>o</code> is equal to this. */
   @Override
   public boolean equals(Object o) {
-    if (this == o) return true;
-    if (!(o instanceof SpanNotQuery)) return false;
+    if (!super.equals(o))
+      return false;
 
     SpanNotQuery other = (SpanNotQuery)o;
     return this.include.equals(other.include)
             && this.exclude.equals(other.exclude)
-            && this.getBoost() == other.getBoost();
+            && this.pre == other.pre 
+            && this.post == other.post;
   }
 
   @Override
   public int hashCode() {
-    int h = include.hashCode();
-    h = (h<<1) | (h >>> 31);  // rotate left
+    int h = super.hashCode();
+    h = Integer.rotateLeft(h, 1);
+    h ^= include.hashCode();
+    h = Integer.rotateLeft(h, 1);
     h ^= exclude.hashCode();
-    h = (h<<1) | (h >>> 31);  // rotate left
-    h ^= Float.floatToRawIntBits(getBoost());
+    h = Integer.rotateLeft(h, 1);
+    h ^= pre;
+    h = Integer.rotateLeft(h, 1);
+    h ^= post;
     return h;
   }
 
-}
+}
\ No newline at end of file

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java Tue Aug 13 04:06:18 2013
@@ -98,7 +98,7 @@ public class SpanTermQuery extends SpanQ
         final Terms terms = fields.terms(term.field());
         if (terms != null) {
           final TermsEnum termsEnum = terms.iterator(null);
-          if (termsEnum.seekExact(term.bytes(), true)) { 
+          if (termsEnum.seekExact(term.bytes())) { 
             state = termsEnum.termState();
           } else {
             state = null;

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java Tue Aug 13 04:06:18 2013
@@ -52,7 +52,7 @@ public class SpanWeight extends Weight {
     final TermStatistics termStats[] = new TermStatistics[terms.size()];
     int i = 0;
     for (Term term : terms) {
-      TermContext state = TermContext.build(context, term, true);
+      TermContext state = TermContext.build(context, term);
       termStats[i] = searcher.termStatistics(term, state);
       termContexts.put(term, state);
       i++;

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/spans/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/spans/package.html?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/spans/package.html (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/search/spans/package.html Tue Aug 13 04:06:18 2013
@@ -38,8 +38,8 @@ and inter-phrase proximity (when constru
 number of other {@link org.apache.lucene.search.spans.SpanQuery}s.</li>
 
 <li>A {@link org.apache.lucene.search.spans.SpanNotQuery SpanNotQuery} removes spans
-matching one {@link org.apache.lucene.search.spans.SpanQuery SpanQuery} which overlap
-another.  This can be used, e.g., to implement within-paragraph
+matching one {@link org.apache.lucene.search.spans.SpanQuery SpanQuery} which overlap (or comes
+near) another.  This can be used, e.g., to implement within-paragraph
 search.</li>
 
 <li>A {@link org.apache.lucene.search.spans.SpanFirstQuery SpanFirstQuery} matches spans

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/BufferedIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/BufferedIndexInput.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/BufferedIndexInput.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/BufferedIndexInput.java Tue Aug 13 04:06:18 2013
@@ -23,7 +23,7 @@ import java.io.IOException;
 /** Base implementation class for buffered {@link IndexInput}. */
 public abstract class BufferedIndexInput extends IndexInput {
 
-  /** Default buffer size set to 1024*/
+  /** Default buffer size set to {@value #BUFFER_SIZE}. */
   public static final int BUFFER_SIZE = 1024;
   
   // The normal read buffer size defaults to 1024, but
@@ -33,7 +33,7 @@ public abstract class BufferedIndexInput
   // BufferedIndexInputs created during merging.  See
   // LUCENE-888 for details.
   /**
-   * A buffer size for merges set to 4096
+   * A buffer size for merges set to {@value #MERGE_BUFFER_SIZE}.
    */
   public static final int MERGE_BUFFER_SIZE = 4096;
 
@@ -115,15 +115,14 @@ public abstract class BufferedIndexInput
 
   @Override
   public final void readBytes(byte[] b, int offset, int len, boolean useBuffer) throws IOException {
-
-    if(len <= (bufferLength-bufferPosition)){
+    int available = bufferLength - bufferPosition;
+    if(len <= available){
       // the buffer contains enough data to satisfy this request
       if(len>0) // to allow b to be null if len is 0...
         System.arraycopy(buffer, bufferPosition, b, offset, len);
       bufferPosition+=len;
     } else {
       // the buffer does not have enough data. First serve all we've got.
-      int available = bufferLength - bufferPosition;
       if(available > 0){
         System.arraycopy(buffer, bufferPosition, b, offset, available);
         offset += available;

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java Tue Aug 13 04:06:18 2013
@@ -31,6 +31,7 @@ import java.util.concurrent.Future;
 
 import org.apache.lucene.util.ThreadInterruptedException;
 import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.IOUtils;
 
 /**
  * Base class for Directory implementations that store index
@@ -111,17 +112,8 @@ import org.apache.lucene.util.Constants;
  */
 public abstract class FSDirectory extends Directory {
 
-  /**
-   * Default read chunk size.  This is a conditional default: on 32bit JVMs, it defaults to 100 MB.  On 64bit JVMs, it's
-   * <code>Integer.MAX_VALUE</code>.
-   *
-   * @see #setReadChunkSize
-   */
-  public static final int DEFAULT_READ_CHUNK_SIZE = Constants.JRE_IS_64BIT ? Integer.MAX_VALUE : 100 * 1024 * 1024;
-
   protected final File directory; // The underlying filesystem directory
   protected final Set<String> staleFiles = synchronizedSet(new HashSet<String>()); // Files written, but not yet sync'ed
-  private int chunkSize = DEFAULT_READ_CHUNK_SIZE; // LUCENE-1566
 
   // returns the canonical version of the directory, creating it if it doesn't exist.
   private static File getCanonicalPath(File file) throws IOException {
@@ -355,68 +347,38 @@ public abstract class FSDirectory extend
   }
 
   /**
-   * Sets the maximum number of bytes read at once from the
-   * underlying file during {@link IndexInput#readBytes}.
-   * The default value is {@link #DEFAULT_READ_CHUNK_SIZE};
-   *
-   * <p> This was introduced due to <a
-   * href="http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6478546">Sun
-   * JVM Bug 6478546</a>, which throws an incorrect
-   * OutOfMemoryError when attempting to read too many bytes
-   * at once.  It only happens on 32bit JVMs with a large
-   * maximum heap size.</p>
-   *
-   * <p>Changes to this value will not impact any
-   * already-opened {@link IndexInput}s.  You should call
-   * this before attempting to open an index on the
-   * directory.</p>
-   *
-   * <p> <b>NOTE</b>: This value should be as large as
-   * possible to reduce any possible performance impact.  If
-   * you still encounter an incorrect OutOfMemoryError,
-   * trying lowering the chunk size.</p>
-   */
-  public final void setReadChunkSize(int chunkSize) {
-    // LUCENE-1566
-    if (chunkSize <= 0) {
-      throw new IllegalArgumentException("chunkSize must be positive");
-    }
-    if (!Constants.JRE_IS_64BIT) {
-      this.chunkSize = chunkSize;
-    }
-  }
-
-  /**
-   * The maximum number of bytes to read at once from the
-   * underlying file during {@link IndexInput#readBytes}.
-   * @see #setReadChunkSize
-   */
-  public final int getReadChunkSize() {
-    // LUCENE-1566
-    return chunkSize;
-  }
-
-  /**
    * Writes output with {@link RandomAccessFile#write(byte[], int, int)}
    */
   protected static class FSIndexOutput extends BufferedIndexOutput {
+    /**
+     * The maximum chunk size is 8192 bytes, because {@link RandomAccessFile} mallocs
+     * a native buffer outside of stack if the write buffer size is larger.
+     */
+    private static final int CHUNK_SIZE = 8192;
+    
     private final FSDirectory parent;
     private final String name;
     private final RandomAccessFile file;
     private volatile boolean isOpen; // remember if the file is open, so that we don't try to close it more than once
     
     public FSIndexOutput(FSDirectory parent, String name) throws IOException {
+      super(CHUNK_SIZE);
       this.parent = parent;
       this.name = name;
       file = new RandomAccessFile(new File(parent.directory, name), "rw");
       isOpen = true;
     }
 
-    /** output methods: */
     @Override
-    public void flushBuffer(byte[] b, int offset, int size) throws IOException {
+    protected void flushBuffer(byte[] b, int offset, int size) throws IOException {
       assert isOpen;
-      file.write(b, offset, size);
+      while (size > 0) {
+        final int toWrite = Math.min(CHUNK_SIZE, size);
+        file.write(b, offset, toWrite);
+        offset += toWrite;
+        size -= toWrite;
+      }
+      assert size == 0;
     }
     
     @Override
@@ -424,21 +386,14 @@ public abstract class FSDirectory extend
       parent.onIndexOutputClosed(name);
       // only close the file if it has not been closed yet
       if (isOpen) {
-        boolean success = false;
+        IOException priorE = null;
         try {
           super.close();
-          success = true;
+        } catch (IOException ioe) {
+          priorE = ioe;
         } finally {
           isOpen = false;
-          if (!success) {
-            try {
-              file.close();
-            } catch (Throwable t) {
-              // Suppress so we don't mask original exception
-            }
-          } else {
-            file.close();
-          }
+          IOUtils.closeWhileHandlingException(priorE, file);
         }
       }
     }

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java Tue Aug 13 04:06:18 2013
@@ -20,6 +20,7 @@ package org.apache.lucene.store;
 import java.io.File;
 import java.io.EOFException;
 import java.io.IOException;
+import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.nio.channels.ClosedChannelException; // javadoc @link
 import java.nio.channels.FileChannel;
@@ -79,7 +80,7 @@ public class NIOFSDirectory extends FSDi
     ensureOpen();
     File path = new File(getDirectory(), name);
     FileChannel fc = FileChannel.open(path.toPath(), StandardOpenOption.READ);
-    return new NIOFSIndexInput("NIOFSIndexInput(path=\"" + path + "\")", fc, context, getReadChunkSize());
+    return new NIOFSIndexInput("NIOFSIndexInput(path=\"" + path + "\")", fc, context);
   }
   
   @Override
@@ -98,7 +99,7 @@ public class NIOFSDirectory extends FSDi
       @Override
       public IndexInput openSlice(String sliceDescription, long offset, long length) {
         return new NIOFSIndexInput("NIOFSIndexInput(" + sliceDescription + " in path=\"" + path + "\" slice=" + offset + ":" + (offset+length) + ")", descriptor, offset,
-            length, BufferedIndexInput.bufferSize(context), getReadChunkSize());
+            length, BufferedIndexInput.bufferSize(context));
       }
     };
   }
@@ -107,12 +108,15 @@ public class NIOFSDirectory extends FSDi
    * Reads bytes with {@link FileChannel#read(ByteBuffer, long)}
    */
   protected static class NIOFSIndexInput extends BufferedIndexInput {
+    /**
+     * The maximum chunk size for reads of 16384 bytes.
+     */
+    private static final int CHUNK_SIZE = 16384;
+    
     /** the file channel we will read from */
     protected final FileChannel channel;
     /** is this instance a clone and hence does not own the file to close it */
     boolean isClone = false;
-    /** maximum read length on a 32bit JVM to prevent incorrect OOM, see LUCENE-1566 */ 
-    protected final int chunkSize;
     /** start offset: non-zero in the slice case */
     protected final long off;
     /** end offset (start+length) */
@@ -120,18 +124,16 @@ public class NIOFSDirectory extends FSDi
     
     private ByteBuffer byteBuf; // wraps the buffer for NIO
 
-    public NIOFSIndexInput(String resourceDesc, FileChannel fc, IOContext context, int chunkSize) throws IOException {
+    public NIOFSIndexInput(String resourceDesc, FileChannel fc, IOContext context) throws IOException {
       super(resourceDesc, context);
       this.channel = fc; 
-      this.chunkSize = chunkSize;
       this.off = 0L;
       this.end = fc.size();
     }
     
-    public NIOFSIndexInput(String resourceDesc, FileChannel fc, long off, long length, int bufferSize, int chunkSize) {
+    public NIOFSIndexInput(String resourceDesc, FileChannel fc, long off, long length, int bufferSize) {
       super(resourceDesc, bufferSize);
       this.channel = fc;
-      this.chunkSize = chunkSize;
       this.off = off;
       this.end = off + length;
       this.isClone = true;
@@ -164,24 +166,18 @@ public class NIOFSDirectory extends FSDi
 
     @Override
     protected void readInternal(byte[] b, int offset, int len) throws IOException {
-
       final ByteBuffer bb;
 
       // Determine the ByteBuffer we should use
-      if (b == buffer && 0 == offset) {
+      if (b == buffer) {
         // Use our own pre-wrapped byteBuf:
         assert byteBuf != null;
-        byteBuf.clear();
-        byteBuf.limit(len);
         bb = byteBuf;
+        byteBuf.clear().position(offset);
       } else {
         bb = ByteBuffer.wrap(b, offset, len);
       }
 
-      int readOffset = bb.position();
-      int readLength = bb.limit() - readOffset;
-      assert readLength == len;
-
       long pos = getFilePointer() + off;
       
       if (pos + len > end) {
@@ -189,30 +185,20 @@ public class NIOFSDirectory extends FSDi
       }
 
       try {
+        int readLength = len;
         while (readLength > 0) {
-          final int limit;
-          if (readLength > chunkSize) {
-            // LUCENE-1566 - work around JVM Bug by breaking
-            // very large reads into chunks
-            limit = readOffset + chunkSize;
-          } else {
-            limit = readOffset + readLength;
+          final int toRead = Math.min(CHUNK_SIZE, readLength);
+          bb.limit(bb.position() + toRead);
+          assert bb.remaining() == toRead;
+          final int i = channel.read(bb, pos);
+          if (i < 0) { // be defensive here, even though we checked before hand, something could have changed
+            throw new EOFException("read past EOF: " + this + " off: " + offset + " len: " + len + " pos: " + pos + " chunkLen: " + toRead + " end: " + end);
           }
-          bb.limit(limit);
-          int i = channel.read(bb, pos);
+          assert i > 0 : "FileChannel.read with non zero-length bb.remaining() must always read at least one byte (FileChannel is in blocking mode, see spec of ReadableByteChannel)";
           pos += i;
-          readOffset += i;
           readLength -= i;
         }
-      } catch (OutOfMemoryError e) {
-        // propagate OOM up and add a hint for 32bit VM Users hitting the bug
-        // with a large chunk size in the fast path.
-        final OutOfMemoryError outOfMemoryError = new OutOfMemoryError(
-              "OutOfMemoryError likely caused by the Sun VM Bug described in "
-              + "https://issues.apache.org/jira/browse/LUCENE-1566; try calling FSDirectory.setReadChunkSize "
-              + "with a value smaller than the current chunk size (" + chunkSize + ")");
-        outOfMemoryError.initCause(e);
-        throw outOfMemoryError;
+        assert readLength == 0;
       } catch (IOException ioe) {
         throw new IOException(ioe.getMessage() + ": " + this, ioe);
       }