You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2014/04/30 09:59:35 UTC

svn commit: r1591209 [3/5] - in /lucene/dev/branches/LUCENE-5622: ./ dev-tools/ dev-tools/idea/solr/contrib/analysis-extras/ dev-tools/idea/solr/contrib/clustering/ dev-tools/idea/solr/contrib/dataimporthandler-extras/ dev-tools/idea/solr/contrib/extra...

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java Wed Apr 30 07:59:31 2014
@@ -26,18 +26,15 @@ import org.apache.lucene.util.BytesRefHa
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.IntBlockPool;
 import org.apache.lucene.util.BytesRefHash.BytesStartArray;
-import org.apache.lucene.util.BytesRefHash.MaxBytesLengthExceededException;
 
-final class TermsHashPerField extends InvertedDocConsumerPerField {
+abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
   private static final int HASH_INIT_SIZE = 4;
 
-  final TermsHashConsumerPerField consumer;
-
   final TermsHash termsHash;
 
   final TermsHashPerField nextPerField;
-  final DocumentsWriterPerThread.DocState docState;
-  final FieldInvertState fieldState;
+  protected final DocumentsWriterPerThread.DocState docState;
+  protected final FieldInvertState fieldState;
   TermToBytesRefAttribute termAtt;
   BytesRef termBytesRef;
 
@@ -49,44 +46,37 @@ final class TermsHashPerField extends In
   final int streamCount;
   final int numPostingInt;
 
-  final FieldInfo fieldInfo;
+  protected final FieldInfo fieldInfo;
 
   final BytesRefHash bytesHash;
 
   ParallelPostingsArray postingsArray;
   private final Counter bytesUsed;
 
-  public TermsHashPerField(DocInverterPerField docInverterPerField, final TermsHash termsHash, final TermsHash nextTermsHash, final FieldInfo fieldInfo) {
+  /** streamCount: how many streams this field stores per term.
+   * E.g. doc(+freq) is 1 stream, prox+offset is a second. */
+
+  public TermsHashPerField(int streamCount, FieldInvertState fieldState, TermsHash termsHash, TermsHashPerField nextPerField, FieldInfo fieldInfo) {
     intPool = termsHash.intPool;
     bytePool = termsHash.bytePool;
     termBytePool = termsHash.termBytePool;
     docState = termsHash.docState;
     this.termsHash = termsHash;
     bytesUsed = termsHash.bytesUsed;
-    fieldState = docInverterPerField.fieldState;
-    this.consumer = termsHash.consumer.addField(this, fieldInfo);
-    PostingsBytesStartArray byteStarts = new PostingsBytesStartArray(this, bytesUsed);
-    bytesHash = new BytesRefHash(termBytePool, HASH_INIT_SIZE, byteStarts);
-    streamCount = consumer.getStreamCount();
+    this.fieldState = fieldState;
+    this.streamCount = streamCount;
     numPostingInt = 2*streamCount;
     this.fieldInfo = fieldInfo;
-    if (nextTermsHash != null)
-      nextPerField = (TermsHashPerField) nextTermsHash.addField(docInverterPerField, fieldInfo);
-    else
-      nextPerField = null;
+    this.nextPerField = nextPerField;
+    PostingsBytesStartArray byteStarts = new PostingsBytesStartArray(this, bytesUsed);
+    bytesHash = new BytesRefHash(termBytePool, HASH_INIT_SIZE, byteStarts);
   }
 
   void reset() {
     bytesHash.clear(false);
-    if (nextPerField != null)
+    if (nextPerField != null) {
       nextPerField.reset();
-  }
-
-  @Override
-  public void abort() {
-    reset();
-    if (nextPerField != null)
-      nextPerField.abort();
+    }
   }
 
   public void initReader(ByteSliceReader reader, int termID, int stream) {
@@ -99,34 +89,17 @@ final class TermsHashPerField extends In
                 ints[upto+stream]);
   }
 
-  /** Collapse the hash table & sort in-place. */
+  int[] sortedTermIDs;
+
+  /** Collapse the hash table & sort in-place; also sets
+   * this.sortedTermIDs to the results */
   public int[] sortPostings() {
-    return bytesHash.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
+    sortedTermIDs = bytesHash.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
+    return sortedTermIDs;
   }
 
-  private boolean doCall;
   private boolean doNextCall;
 
-  @Override
-  void start(IndexableField f) {
-    termAtt = fieldState.attributeSource.getAttribute(TermToBytesRefAttribute.class);
-    termBytesRef = termAtt.getBytesRef();
-    consumer.start(f);
-    if (nextPerField != null) {
-      nextPerField.start(f);
-    }
-  }
-
-  @Override
-  boolean start(IndexableField[] fields, int count) throws IOException {
-    doCall = consumer.start(fields, count);
-    bytesHash.reinit();
-    if (nextPerField != null) {
-      doNextCall = nextPerField.start(fields, count);
-    }
-    return doCall || doNextCall;
-  }
-
   // Secondary entry point (for 2nd & subsequent TermsHash),
   // because token text has already been "interned" into
   // textStart, so we hash by textStart.  term vectors use
@@ -137,8 +110,9 @@ final class TermsHashPerField extends In
       // First time we are seeing this token since we last
       // flushed the hash.
       // Init stream slices
-      if (numPostingInt + intPool.intUpto > IntBlockPool.INT_BLOCK_SIZE)
+      if (numPostingInt + intPool.intUpto > IntBlockPool.INT_BLOCK_SIZE) {
         intPool.nextBuffer();
+      }
 
       if (ByteBlockPool.BYTE_BLOCK_SIZE - bytePool.byteUpto < numPostingInt*ByteBlockPool.FIRST_LEVEL_SIZE) {
         bytePool.nextBuffer();
@@ -156,20 +130,20 @@ final class TermsHashPerField extends In
       }
       postingsArray.byteStarts[termID] = intUptos[intUptoStart];
 
-      consumer.newTerm(termID);
+      newTerm(termID);
 
     } else {
       termID = (-termID)-1;
       int intStart = postingsArray.intStarts[termID];
       intUptos = intPool.buffers[intStart >> IntBlockPool.INT_BLOCK_SHIFT];
       intUptoStart = intStart & IntBlockPool.INT_BLOCK_MASK;
-      consumer.addTerm(termID);
+      addTerm(termID);
     }
   }
 
-  // Primary entry point (for first TermsHash); postings use
-  // this API.
-  @Override
+  /** Called once per inverted token.  This is the primary
+   *  entry point (for first TermsHash); postings use this
+   *  API. */
   void add() throws IOException {
 
     termAtt.fillBytesRef();
@@ -177,27 +151,10 @@ final class TermsHashPerField extends In
     // We are first in the chain so we must "intern" the
     // term text into textStart address
     // Get the text & hash of this term.
-    int termID;
-    try {
-      termID = bytesHash.add(termBytesRef);
-    } catch (MaxBytesLengthExceededException e) {
-      // Term is too large; record this here (can't throw an
-      // exc because DocInverterPerField will then abort the
-      // entire segment) and then throw an exc later in
-      // DocInverterPerField.java.  LengthFilter can always be
-      // used to prune the term before indexing:
-      if (docState.maxTermPrefix == null) {
-        final int saved = termBytesRef.length;
-        try {
-          termBytesRef.length = Math.min(30, DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8);
-          docState.maxTermPrefix = termBytesRef.toString();
-        } finally {
-          termBytesRef.length = saved;
-        }
-      }
-      consumer.skippingLongTerm();
-      return;
-    }
+    int termID = bytesHash.add(termBytesRef);
+      
+    //System.out.println("add term=" + termBytesRef.utf8ToString() + " doc=" + docState.docID + " termID=" + termID);
+
     if (termID >= 0) {// New posting
       bytesHash.byteStart(termID);
       // Init stream slices
@@ -221,18 +178,19 @@ final class TermsHashPerField extends In
       }
       postingsArray.byteStarts[termID] = intUptos[intUptoStart];
 
-      consumer.newTerm(termID);
+      newTerm(termID);
 
     } else {
       termID = (-termID)-1;
-      final int intStart = postingsArray.intStarts[termID];
+      int intStart = postingsArray.intStarts[termID];
       intUptos = intPool.buffers[intStart >> IntBlockPool.INT_BLOCK_SHIFT];
       intUptoStart = intStart & IntBlockPool.INT_BLOCK_MASK;
-      consumer.addTerm(termID);
+      addTerm(termID);
     }
 
-    if (doNextCall)
+    if (doNextCall) {
       nextPerField.add(postingsArray.textStarts[termID]);
+    }
   }
 
   int[] intUptos;
@@ -269,13 +227,6 @@ final class TermsHashPerField extends In
     writeByte(stream, (byte) i);
   }
 
-  @Override
-  void finish() throws IOException {
-    consumer.finish();
-    if (nextPerField != null)
-      nextPerField.finish();
-  }
-
   private static final class PostingsBytesStartArray extends BytesStartArray {
 
     private final TermsHashPerField perField;
@@ -290,7 +241,8 @@ final class TermsHashPerField extends In
     @Override
     public int[] init() {
       if (perField.postingsArray == null) {
-        perField.postingsArray = perField.consumer.createPostingsArray(2);
+        perField.postingsArray = perField.createPostingsArray(2);
+        perField.newPostingsArray();
         bytesUsed.addAndGet(perField.postingsArray.size * perField.postingsArray.bytesPerPosting());
       }
       return perField.postingsArray.textStarts;
@@ -301,15 +253,17 @@ final class TermsHashPerField extends In
       ParallelPostingsArray postingsArray = perField.postingsArray;
       final int oldSize = perField.postingsArray.size;
       postingsArray = perField.postingsArray = postingsArray.grow();
+      perField.newPostingsArray();
       bytesUsed.addAndGet((postingsArray.bytesPerPosting() * (postingsArray.size - oldSize)));
       return postingsArray.textStarts;
     }
 
     @Override
     public int[] clear() {
-      if(perField.postingsArray != null) {
+      if (perField.postingsArray != null) {
         bytesUsed.addAndGet(-(perField.postingsArray.size * perField.postingsArray.bytesPerPosting()));
         perField.postingsArray = null;
+        perField.newPostingsArray();
       }
       return null;
     }
@@ -318,7 +272,47 @@ final class TermsHashPerField extends In
     public Counter bytesUsed() {
       return bytesUsed;
     }
+  }
+
+  @Override
+  public int compareTo(TermsHashPerField other) {
+    return fieldInfo.name.compareTo(other.fieldInfo.name);
+  }
+
+  /** Finish adding all instances of this field to the
+   *  current document. */
+  void finish() throws IOException {
+    if (nextPerField != null) {
+      nextPerField.finish();
+    }
+  }
 
+  /** Start adding a new field instance; first is true if
+   *  this is the first time this field name was seen in the
+   *  document. */
+  boolean start(IndexableField field, boolean first) {
+    termAtt = fieldState.termAttribute;
+    // EmptyTokenStream can have null term att
+    if (termAtt != null) {
+      termBytesRef = termAtt.getBytesRef();
+    }
+    if (nextPerField != null) {
+      doNextCall = nextPerField.start(field, first);
+    }
+
+    return true;
   }
 
+  /** Called when a term is seen for the first time. */
+  abstract void newTerm(int termID) throws IOException;
+
+  /** Called when a previously seen term is seen again. */
+  abstract void addTerm(int termID) throws IOException;
+
+  /** Called when the postings array is initialized or
+   *  resized. */
+  abstract void newPostingsArray();
+
+  /** Creates a new postings array of the specified size. */
+  abstract ParallelPostingsArray createPostingsArray(int size);
 }

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/search/Weight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/search/Weight.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/search/Weight.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/search/Weight.java Wed Apr 30 07:59:31 2014
@@ -170,7 +170,11 @@ public abstract class Weight {
         return scoreRange(collector, scorer, doc, max);
       }
     }
-    
+
+    /** Specialized method to bulk-score a range of hits; we
+     *  separate this from {@link #scoreAll} to help out
+     *  hotspot.
+     *  See <a href="https://issues.apache.org/jira/browse/LUCENE-5487">LUCENE-5487</a> */
     static boolean scoreRange(LeafCollector collector, Scorer scorer, int currentDoc, int end) throws IOException {
       while (currentDoc < end) {
         collector.collect(currentDoc);
@@ -179,6 +183,10 @@ public abstract class Weight {
       return currentDoc != DocIdSetIterator.NO_MORE_DOCS;
     }
     
+    /** Specialized method to bulk-score all hits; we
+     *  separate this from {@link #scoreRange} to help out
+     *  hotspot.
+     *  See <a href="https://issues.apache.org/jira/browse/LUCENE-5487">LUCENE-5487</a> */
     static void scoreAll(LeafCollector collector, Scorer scorer) throws IOException {
       int doc;
       while ((doc = scorer.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/store/NativeFSLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/store/NativeFSLockFactory.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/store/NativeFSLockFactory.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/store/NativeFSLockFactory.java Wed Apr 30 07:59:31 2014
@@ -123,14 +123,10 @@ class NativeFSLock extends Lock {
     path = new File(lockDir, lockFileName);
   }
 
-  private synchronized boolean lockExists() {
-    return lock != null;
-  }
-
   @Override
   public synchronized boolean obtain() throws IOException {
 
-    if (lockExists()) {
+    if (lock != null) {
       // Our instance is already locked:
       return false;
     }
@@ -150,7 +146,7 @@ class NativeFSLock extends Lock {
     boolean success = false;
     try {
       lock = channel.tryLock();
-      success = true;
+      success = lock != null;
     } catch (IOException | OverlappingFileLockException e) {
       // At least on OS X, we will sometimes get an
       // intermittent "Permission Denied" IOException,
@@ -171,39 +167,20 @@ class NativeFSLock extends Lock {
         }
       }
     }
-    return lockExists();
+    return lock != null;
   }
 
   @Override
   public synchronized void close() throws IOException {
-    if (lockExists()) {
-      try {
+    try {
+      if (lock != null) {
         lock.release();
-      } finally {
         lock = null;
-        try {
-          channel.close();
-        } finally {
-          channel = null;
-        }
       }
-    } else {
-      // if we don't hold the lock, and somebody still called release(), for
-      // example as a result of calling IndexWriter.unlock(), we should attempt
-      // to obtain the lock and release it. If the obtain fails, it means the
-      // lock cannot be released, and we should throw a proper exception rather
-      // than silently failing/not doing anything.
-      boolean obtained = false;
-      try {
-        if (!(obtained = obtain())) {
-          throw new LockReleaseFailedException(
-              "Cannot forcefully unlock a NativeFSLock which is held by another indexer component: "
-                  + path);
-        }
-      } finally {
-        if (obtained) {
-          close();
-        }
+    } finally {
+      if (channel != null) {
+        channel.close();
+        channel = null;
       }
     }
   }
@@ -213,7 +190,7 @@ class NativeFSLock extends Lock {
     // The test for is isLocked is not directly possible with native file locks:
     
     // First a shortcut, if a lock reference in this instance is available
-    if (lockExists()) return true;
+    if (lock != null) return true;
     
     // Look if lock file is present; if not, there can definitely be no lock!
     if (!path.exists()) return false;

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java Wed Apr 30 07:59:31 2014
@@ -171,7 +171,7 @@ public class RAMDirectory extends BaseDi
       existing.directory = null;
     }
     fileMap.put(name, file);
-    return new RAMOutputStream(file);
+    return new RAMOutputStream(file, true);
   }
 
   /**

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java Wed Apr 30 07:59:31 2014
@@ -38,20 +38,25 @@ public class RAMOutputStream extends Ind
   private long bufferStart;
   private int bufferLength;
   
-  private Checksum crc = new BufferedChecksum(new CRC32());
+  private final Checksum crc;
 
   /** Construct an empty output buffer. */
   public RAMOutputStream() {
-    this(new RAMFile());
+    this(new RAMFile(), false);
   }
 
-  public RAMOutputStream(RAMFile f) {
+  public RAMOutputStream(RAMFile f, boolean checksum) {
     file = f;
 
     // make sure that we switch to the
     // first needed buffer lazily
     currentBufferIndex = -1;
     currentBuffer = null;
+    if (checksum) {
+      crc = new BufferedChecksum(new CRC32());
+    } else {
+      crc = null;
+    }
   }
 
   /** Copy the current contents of this buffer to the named output. */
@@ -99,7 +104,9 @@ public class RAMOutputStream extends Ind
     bufferStart = 0;
     bufferLength = 0;
     file.setLength(0);
-    crc.reset();
+    if (crc != null) {
+      crc.reset();
+    }
   }
 
   @Override
@@ -113,14 +120,18 @@ public class RAMOutputStream extends Ind
       currentBufferIndex++;
       switchCurrentBuffer();
     }
-    crc.update(b);
+    if (crc != null) {
+      crc.update(b);
+    }
     currentBuffer[bufferPosition++] = b;
   }
 
   @Override
   public void writeBytes(byte[] b, int offset, int len) throws IOException {
     assert b != null;
-    crc.update(b, offset, len);
+    if (crc != null) {
+      crc.update(b, offset, len);
+    }
     while (len > 0) {
       if (bufferPosition ==  bufferLength) {
         currentBufferIndex++;
@@ -171,6 +182,10 @@ public class RAMOutputStream extends Ind
 
   @Override
   public long getChecksum() throws IOException {
-    return crc.getValue();
+    if (crc == null) {
+      throw new IllegalStateException("internal RAMOutputStream created with checksum disabled");
+    } else {
+      return crc.getValue();
+    }
   }
 }

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/store/SimpleFSLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/store/SimpleFSLockFactory.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/store/SimpleFSLockFactory.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/store/SimpleFSLockFactory.java Wed Apr 30 07:59:31 2014
@@ -132,7 +132,17 @@ class SimpleFSLock extends Lock {
       throw new IOException("Found regular file where directory expected: " + 
                             lockDir.getAbsolutePath());
     }
-    return lockFile.createNewFile();
+    
+    try {
+      return lockFile.createNewFile();
+    } catch (IOException ioe) {
+      // On Windows, on concurrent createNewFile, the 2nd process gets "access denied".
+      // In that case, the lock was not aquired successfully, so return false.
+      // We record the failure reason here; the obtain with timeout (usually the
+      // one calling us) will use this as "root cause" if it fails to get the lock.
+      failureReason = ioe;
+      return false;
+    }
   }
 
   @Override

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/util/AttributeSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/util/AttributeSource.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/util/AttributeSource.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/util/AttributeSource.java Wed Apr 30 07:59:31 2014
@@ -287,22 +287,20 @@ public class AttributeSource {
   }
 
   /**
-   * The caller must pass in a Class&lt;? extends Attribute&gt; value. 
    * Returns the instance of the passed in Attribute contained in this AttributeSource
+   * <p>
+   * The caller must pass in a Class&lt;? extends Attribute&gt; value. 
    * 
-   * @throws IllegalArgumentException if this AttributeSource does not contain the
-   *         Attribute. It is recommended to always use {@link #addAttribute} even in consumers
-   *         of TokenStreams, because you cannot know if a specific TokenStream really uses
-   *         a specific Attribute. {@link #addAttribute} will automatically make the attribute
-   *         available. If you want to only use the attribute, if it is available (to optimize
+   * @return instance of the passed in Attribute, or {@code null} if this AttributeSource 
+   *         does not contain the Attribute. It is recommended to always use 
+   *         {@link #addAttribute} even in consumers  of TokenStreams, because you cannot 
+   *         know if a specific TokenStream really uses a specific Attribute. 
+   *         {@link #addAttribute} will automatically make the attribute available. 
+   *         If you want to only use the attribute, if it is available (to optimize
    *         consuming), use {@link #hasAttribute}.
    */
   public final <T extends Attribute> T getAttribute(Class<T> attClass) {
-    AttributeImpl attImpl = attributes.get(attClass);
-    if (attImpl == null) {
-      throw new IllegalArgumentException("This AttributeSource does not have the attribute '" + attClass.getName() + "'.");
-    }
-    return attClass.cast(attImpl);
+    return attClass.cast(attributes.get(attClass));
   }
     
   private State getCurrentState() {

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java Wed Apr 30 07:59:31 2014
@@ -80,8 +80,7 @@ public final class BytesRefHash {
   /**
    * Creates a new {@link BytesRefHash}
    */
-  public BytesRefHash(ByteBlockPool pool, int capacity,
-      BytesStartArray bytesStartArray) {
+  public BytesRefHash(ByteBlockPool pool, int capacity, BytesStartArray bytesStartArray) {
     hashSize = capacity;
     hashHalfSize = hashSize >> 1;
     hashMask = hashSize - 1;

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java Wed Apr 30 07:59:31 2014
@@ -17,12 +17,16 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
+import java.io.IOException;
+
 import org.apache.lucene.analysis.NumericTokenStream;
 import org.apache.lucene.document.DoubleField; // javadocs
 import org.apache.lucene.document.FloatField; // javadocs
 import org.apache.lucene.document.IntField; // javadocs
 import org.apache.lucene.document.LongField; // javadocs
+import org.apache.lucene.index.FilterAtomicReader;
 import org.apache.lucene.index.FilteredTermsEnum;
+import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.NumericRangeFilter;
 import org.apache.lucene.search.NumericRangeQuery; // for javadocs
@@ -464,14 +468,15 @@ public final class NumericUtils {
    *         terms with a shift value of <tt>0</tt>.
    */
   public static TermsEnum filterPrefixCodedLongs(TermsEnum termsEnum) {
-    return new FilteredTermsEnum(termsEnum, false) {
+    return new SeekingNumericFilteredTermsEnum(termsEnum) {
+
       @Override
       protected AcceptStatus accept(BytesRef term) {
         return NumericUtils.getPrefixCodedLongShift(term) == 0 ? AcceptStatus.YES : AcceptStatus.END;
       }
     };
   }
-  
+
   /**
    * Filters the given {@link TermsEnum} by accepting only prefix coded 32 bit
    * terms with a shift value of <tt>0</tt>.
@@ -482,7 +487,7 @@ public final class NumericUtils {
    *         terms with a shift value of <tt>0</tt>.
    */
   public static TermsEnum filterPrefixCodedInts(TermsEnum termsEnum) {
-    return new FilteredTermsEnum(termsEnum, false) {
+    return new SeekingNumericFilteredTermsEnum(termsEnum) {
       
       @Override
       protected AcceptStatus accept(BytesRef term) {
@@ -490,5 +495,85 @@ public final class NumericUtils {
       }
     };
   }
+
+  /** Just like FilteredTermsEnum, except it adds a limited
+   *  seekCeil implementation that only works with {@link
+   *  #filterPrefixCodedInts} and {@link
+   *  #filterPrefixCodedLongs}. */
+  private static abstract class SeekingNumericFilteredTermsEnum extends FilteredTermsEnum {
+    public SeekingNumericFilteredTermsEnum(final TermsEnum tenum) {
+      super(tenum, false);
+    }
+
+    @Override
+    @SuppressWarnings("fallthrough")
+    public SeekStatus seekCeil(BytesRef term) throws IOException {
+
+      // NOTE: This is not general!!  It only handles YES
+      // and END, because that's all we need for the numeric
+      // case here
+
+      SeekStatus status = tenum.seekCeil(term);
+      if (status == SeekStatus.END) {
+        return SeekStatus.END;
+      }
+
+      actualTerm = tenum.term();
+
+      if (accept(actualTerm) == AcceptStatus.YES) {
+        return status;
+      } else {
+        return SeekStatus.END;
+      }
+    }
+  }
+
+  private static Terms intTerms(Terms terms) {
+    return new FilterAtomicReader.FilterTerms(terms) {
+        @Override
+        public TermsEnum iterator(TermsEnum reuse) throws IOException {
+          return filterPrefixCodedInts(in.iterator(reuse));
+        }
+      };
+  }
+
+  private static Terms longTerms(Terms terms) {
+    return new FilterAtomicReader.FilterTerms(terms) {
+        @Override
+        public TermsEnum iterator(TermsEnum reuse) throws IOException {
+          return filterPrefixCodedLongs(in.iterator(reuse));
+        }
+      };
+  }
+    
+  /** Returns the minimum int value indexed into this
+   *  numeric field. */
+  public static int getMinInt(Terms terms) throws IOException {
+    // All shift=0 terms are sorted first, so we don't need
+    // to filter the incoming terms; we can just get the
+    // min: 
+    return NumericUtils.prefixCodedToInt(terms.getMin());
+  }
+
+  /** Returns the maximum int value indexed into this
+   *  numeric field. */
+  public static int getMaxInt(Terms terms) throws IOException {
+    return NumericUtils.prefixCodedToInt(intTerms(terms).getMax());
+  }
+
+  /** Returns the minimum long value indexed into this
+   *  numeric field. */
+  public static long getMinLong(Terms terms) throws IOException {
+    // All shift=0 terms are sorted first, so we don't need
+    // to filter the incoming terms; we can just get the
+    // min: 
+    return NumericUtils.prefixCodedToLong(terms.getMin());
+  }
+
+  /** Returns the maximum long value indexed into this
+   *  numeric field. */
+  public static long getMaxLong(Terms terms) throws IOException {
+    return NumericUtils.prefixCodedToLong(longTerms(terms).getMax());
+  }
   
 }

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java Wed Apr 30 07:59:31 2014
@@ -207,12 +207,8 @@ public class QueryBuilder {
       buffer = new CachingTokenFilter(source);
       buffer.reset();
 
-      if (buffer.hasAttribute(TermToBytesRefAttribute.class)) {
-        termAtt = buffer.getAttribute(TermToBytesRefAttribute.class);
-      }
-      if (buffer.hasAttribute(PositionIncrementAttribute.class)) {
-        posIncrAtt = buffer.getAttribute(PositionIncrementAttribute.class);
-      }
+      termAtt = buffer.getAttribute(TermToBytesRefAttribute.class);
+      posIncrAtt = buffer.getAttribute(PositionIncrementAttribute.class);
 
       if (termAtt != null) {
         try {

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java Wed Apr 30 07:59:31 2014
@@ -31,10 +31,12 @@ public class TestNumericTokenStream exte
 
   public void testLongStream() throws Exception {
     final NumericTokenStream stream=new NumericTokenStream().setLongValue(lvalue);
-    // use getAttribute to test if attributes really exist, if not an IAE will be throwed
     final TermToBytesRefAttribute bytesAtt = stream.getAttribute(TermToBytesRefAttribute.class);
+    assertNotNull(bytesAtt);
     final TypeAttribute typeAtt = stream.getAttribute(TypeAttribute.class);
+    assertNotNull(typeAtt);
     final NumericTokenStream.NumericTermAttribute numericAtt = stream.getAttribute(NumericTokenStream.NumericTermAttribute.class);
+    assertNotNull(numericAtt);
     final BytesRef bytes = bytesAtt.getBytesRef();
     stream.reset();
     assertEquals(64, numericAtt.getValueSize());
@@ -53,10 +55,12 @@ public class TestNumericTokenStream exte
 
   public void testIntStream() throws Exception {
     final NumericTokenStream stream=new NumericTokenStream().setIntValue(ivalue);
-    // use getAttribute to test if attributes really exist, if not an IAE will be throwed
     final TermToBytesRefAttribute bytesAtt = stream.getAttribute(TermToBytesRefAttribute.class);
+    assertNotNull(bytesAtt);
     final TypeAttribute typeAtt = stream.getAttribute(TypeAttribute.class);
+    assertNotNull(typeAtt);
     final NumericTokenStream.NumericTermAttribute numericAtt = stream.getAttribute(NumericTokenStream.NumericTermAttribute.class);
+    assertNotNull(numericAtt);
     final BytesRef bytes = bytesAtt.getBytesRef();
     stream.reset();
     assertEquals(32, numericAtt.getValueSize());

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/document/TestDocument.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/document/TestDocument.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/document/TestDocument.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/document/TestDocument.java Wed Apr 30 07:59:31 2014
@@ -131,7 +131,7 @@ public class TestDocument extends Lucene
     assertEquals(0, doc.getFields().size());
   }
 
-  public void testConstructorExceptions() {
+  public void testConstructorExceptions() throws Exception {
     FieldType ft = new FieldType();
     ft.setStored(true);
     new Field("name", "value", ft); // okay
@@ -142,16 +142,23 @@ public class TestDocument extends Lucene
     } catch (IllegalArgumentException e) {
       // expected exception
     }
+
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
     new Field("name", "value", ft); // okay
+    Document doc = new Document();
+    FieldType ft2 = new FieldType();
+    ft2.setStored(true);
+    ft2.setStoreTermVectors(true);
+    doc.add(new Field("name", "value", ft2));
     try {
-      FieldType ft2 = new FieldType();
-      ft2.setStored(true);
-      ft2.setStoreTermVectors(true);
-      new Field("name", "value", ft2);
+      w.addDocument(doc);
       fail();
     } catch (IllegalArgumentException e) {
       // expected exception
     }
+    w.close();
+    dir.close();
   }
 
   public void testClearDocument() {

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java Wed Apr 30 07:59:31 2014
@@ -31,11 +31,10 @@ import org.apache.lucene.analysis.Canned
  */
 public final class BinaryTokenStream extends TokenStream {
   private final ByteTermAttribute bytesAtt = addAttribute(ByteTermAttribute.class);
-  private final BytesRef bytes;
   private boolean available = true;
   
   public BinaryTokenStream(BytesRef bytes) {
-    this.bytes = bytes;
+    bytesAtt.setBytesRef(bytes);
   }
   
   @Override
@@ -43,7 +42,6 @@ public final class BinaryTokenStream ext
     if (available) {
       clearAttributes();
       available = false;
-      bytesAtt.setBytesRef(bytes);
       return true;
     }
     return false;

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java Wed Apr 30 07:59:31 2014
@@ -279,21 +279,14 @@ public class TestDocumentWriter extends 
 
     IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(
         TEST_VERSION_CURRENT, new MockAnalyzer(random())));
-    writer.addDocument(doc);
+    try {
+      writer.addDocument(doc);
+      fail("did not hit exception");
+    } catch (IllegalArgumentException iae) {
+      assertEquals("all instances of a given field name must have the same term vectors settings (storeTermVectors changed for field=\"f1\")",
+                   iae.getMessage());
+    }
     writer.shutdown();
-
-    TestUtil.checkIndex(dir);
-
-    IndexReader reader = DirectoryReader.open(dir);
-    // f1
-    Terms tfv1 = reader.getTermVectors(0).terms("f1");
-    assertNotNull(tfv1);
-    assertEquals("the 'with_tv' setting should rule!",2,tfv1.size());
-    // f2
-    Terms tfv2 = reader.getTermVectors(0).terms("f2");
-    assertNotNull(tfv2);
-    assertEquals("the 'with_tv' setting should rule!",2,tfv2.size());
-    reader.close();
   }
 
   /**

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java Wed Apr 30 07:59:31 2014
@@ -546,14 +546,17 @@ public class TestIndexWriterExceptions e
       if (doFail) {
         StackTraceElement[] trace = new Exception().getStackTrace();
         boolean sawFlush = false;
+        boolean sawFinishDocument = false;
         for (int i = 0; i < trace.length; i++) {
           if ("flush".equals(trace[i].getMethodName())) {
             sawFlush = true;
-            break;
+          }
+          if ("finishDocument".equals(trace[i].getMethodName())) {
+            sawFinishDocument = true;
           }
         }
 
-        if (sawFlush && count++ >= 30) {
+        if (sawFlush && sawFinishDocument == false && count++ >= 30) {
           doFail = false;
           throw new IOException("now failing during flush");
         }

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java Wed Apr 30 07:59:31 2014
@@ -427,7 +427,7 @@ public class TestIndexWriterWithThreads 
       if (doFail) {
         StackTraceElement[] trace = new Exception().getStackTrace();
         for (int i = 0; i < trace.length; i++) {
-          if ("flush".equals(trace[i].getMethodName()) && "org.apache.lucene.index.DocFieldProcessor".equals(trace[i].getClassName())) {
+          if ("flush".equals(trace[i].getMethodName()) && DefaultIndexingChain.class.getName().equals(trace[i].getClassName())) {
             if (onlyOnce)
               doFail = false;
             //System.out.println(Thread.currentThread().getName() + ": NOW FAIL: onlyOnce=" + onlyOnce);

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java Wed Apr 30 07:59:31 2014
@@ -20,11 +20,14 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.io.Reader;
 import java.io.StringReader;
+import java.util.Collections;
 import java.util.Iterator;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
@@ -378,4 +381,63 @@ public class TestIndexableField extends 
     r.close();
     dir.close();
   }
+
+  private static class CustomField implements StorableField {
+    @Override
+    public BytesRef binaryValue() {
+      return null;
+    }
+
+    @Override
+    public String stringValue() {
+      return "foobar";
+    }
+
+    @Override
+    public Reader readerValue() {
+      return null;
+    }
+
+    @Override
+    public Number numericValue() {
+      return null;
+    }
+
+    @Override
+    public String name() {
+      return "field";
+    }
+
+    @Override
+    public IndexableFieldType fieldType() {
+      FieldType ft = new FieldType(StoredField.TYPE);
+      ft.setStoreTermVectors(true);
+      ft.freeze();
+      return ft;
+    }
+  }
+
+  // LUCENE-5611
+  public void testNotIndexedTermVectors() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    try {
+      w.addDocument(
+                    new IndexDocument() {
+                      @Override
+                      public Iterable<IndexableField> indexableFields() {
+                        return Collections.emptyList();
+                      }
+                      @Override
+                      public Iterable<StorableField> storableFields() {
+                        return Collections.<StorableField>singletonList(new CustomField());
+                      }
+                    });
+      fail("didn't hit exception");
+    } catch (IllegalArgumentException iae) {
+      // expected
+    }
+    w.close();
+    dir.close();
+  }
 }

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestParallelReaderEmptyIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestParallelReaderEmptyIndex.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestParallelReaderEmptyIndex.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestParallelReaderEmptyIndex.java Wed Apr 30 07:59:31 2014
@@ -98,7 +98,7 @@ public class TestParallelReaderEmptyInde
       doc.add(newField("test", "", customType));
       idField.setStringValue("1");
       iw.addDocument(doc);
-      doc.add(newTextField("test", "", Field.Store.NO));
+      doc.add(newField("test", "", customType));
       idField.setStringValue("2");
       iw.addDocument(doc);
       iw.shutdown();

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java Wed Apr 30 07:59:31 2014
@@ -639,8 +639,8 @@ public class TestStressIndexing2 extends
           int freq1 = dpEnum1.freq();
           int freq2 = dpEnum2.freq();
           assertEquals(freq1, freq2);
-          OffsetAttribute offsetAtt1 = dpEnum1.attributes().hasAttribute(OffsetAttribute.class) ? dpEnum1.attributes().getAttribute(OffsetAttribute.class) : null;
-          OffsetAttribute offsetAtt2 = dpEnum2.attributes().hasAttribute(OffsetAttribute.class) ? dpEnum2.attributes().getAttribute(OffsetAttribute.class) : null;
+          OffsetAttribute offsetAtt1 = dpEnum1.attributes().getAttribute(OffsetAttribute.class);
+          OffsetAttribute offsetAtt2 = dpEnum2.attributes().getAttribute(OffsetAttribute.class);
 
           if (offsetAtt1 != null) {
             assertNotNull(offsetAtt2);
@@ -773,24 +773,39 @@ public class TestStressIndexing2 extends
       Field idField =  newField("id", idString, customType1);
       fields.add(idField);
 
+      Map<String,FieldType> tvTypes = new HashMap<>();
+
       int nFields = nextInt(maxFields);
       for (int i=0; i<nFields; i++) {
 
-        FieldType customType = new FieldType();
-        switch (nextInt(4)) {
-        case 0:
-          break;
-        case 1:
-          customType.setStoreTermVectors(true);
-          break;
-        case 2:
-          customType.setStoreTermVectors(true);
-          customType.setStoreTermVectorPositions(true);
-          break;
-        case 3:
-          customType.setStoreTermVectors(true);
-          customType.setStoreTermVectorOffsets(true);
-          break;
+        String fieldName = "f" + nextInt(100);
+        FieldType customType;
+
+        // Use the same term vector settings if we already
+        // added this field to the doc:
+        FieldType oldTVType = tvTypes.get(fieldName);
+        if (oldTVType != null) {
+          customType = new FieldType(oldTVType);
+        } else {
+          customType = new FieldType();
+          switch (nextInt(4)) {
+          case 0:
+            break;
+          case 1:
+            customType.setStoreTermVectors(true);
+            break;
+          case 2:
+            customType.setStoreTermVectors(true);
+            customType.setStoreTermVectorPositions(true);
+            break;
+          case 3:
+            customType.setStoreTermVectors(true);
+            customType.setStoreTermVectorOffsets(true);
+            break;
+          }
+          FieldType newType = new FieldType(customType);
+          newType.freeze();
+          tvTypes.put(fieldName, newType);
         }
         
         switch (nextInt(4)) {
@@ -798,26 +813,30 @@ public class TestStressIndexing2 extends
             customType.setStored(true);
             customType.setOmitNorms(true);
             customType.setIndexed(true);
-            fields.add(newField("f" + nextInt(100), getString(1), customType));
+            customType.freeze();
+            fields.add(newField(fieldName, getString(1), customType));
             break;
           case 1:
             customType.setIndexed(true);
             customType.setTokenized(true);
-            fields.add(newField("f" + nextInt(100), getString(0), customType));
+            customType.freeze();
+            fields.add(newField(fieldName, getString(0), customType));
             break;
           case 2:
             customType.setStored(true);
             customType.setStoreTermVectors(false);
             customType.setStoreTermVectorOffsets(false);
             customType.setStoreTermVectorPositions(false);
-            fields.add(newField("f" + nextInt(100), getString(0), customType));
+            customType.freeze();
+            fields.add(newField(fieldName, getString(0), customType));
             break;
           case 3:
             customType.setStored(true);
             customType.setIndexed(true);
             customType.setTokenized(true);
-            fields.add(newField("f" + nextInt(100), getString(bigFieldSize), customType));
-            break;          
+            customType.freeze();
+            fields.add(newField(fieldName, getString(bigFieldSize), customType));
+            break;
         }
       }
 
@@ -872,8 +891,7 @@ public class TestStressIndexing2 extends
           }
         }
       } catch (Throwable e) {
-        e.printStackTrace();
-        Assert.fail(e.toString());
+        throw new RuntimeException(e);
       }
 
       synchronized (this) {

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java Wed Apr 30 07:59:31 2014
@@ -29,6 +29,7 @@ import org.apache.lucene.codecs.TermVect
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.StoredField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.Directory;
@@ -109,6 +110,7 @@ public class TestTermVectorsReader exten
       }
       else if (!testFieldsStorePos[i] && testFieldsStoreOff[i]) {
         customType.setStoreTermVectors(true);
+        customType.setStoreTermVectorPositions(true);
         customType.setStoreTermVectorOffsets(true);
       }
       else {
@@ -333,7 +335,9 @@ public class TestTermVectorsReader exten
 
   public void testIllegalIndexableField() throws Exception {
     Directory dir = newDirectory();
-    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    MockAnalyzer a = new MockAnalyzer(random());
+    a.setEnableChecks(false);
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, a);
     FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
     ft.setStoreTermVectors(true);
     ft.setStoreTermVectorPayloads(true);
@@ -386,6 +390,67 @@ public class TestTermVectorsReader exten
       assertEquals("cannot index term vector payloads when term vectors are not indexed (field=\"field\")", iae.getMessage());
     }
 
+    ft = new FieldType(TextField.TYPE_NOT_STORED);
+    ft.setStoreTermVectors(true);
+    ft.setStoreTermVectorPayloads(true);
+    doc = new Document();
+    doc.add(new Field("field", "value", ft));
+    try {
+      w.addDocument(doc);
+      fail("did not hit exception");
+    } catch (IllegalArgumentException iae) {
+      // Expected
+      assertEquals("cannot index term vector payloads without term vector positions (field=\"field\")", iae.getMessage());
+    }
+
+    ft = new FieldType(StoredField.TYPE);
+    ft.setStoreTermVectors(true);
+    doc = new Document();
+    doc.add(new Field("field", "value", ft));
+    try {
+      w.addDocument(doc);
+      fail("did not hit exception");
+    } catch (IllegalArgumentException iae) {
+      // Expected
+      assertEquals("cannot store term vectors for a field that is not indexed (field=\"field\")", iae.getMessage());
+    }
+
+    ft = new FieldType(StoredField.TYPE);
+    ft.setStoreTermVectorPositions(true);
+    doc = new Document();
+    doc.add(new Field("field", "value", ft));
+    try {
+      w.addDocument(doc);
+      fail("did not hit exception");
+    } catch (IllegalArgumentException iae) {
+      // Expected
+      assertEquals("cannot store term vector positions for a field that is not indexed (field=\"field\")", iae.getMessage());
+    }
+
+    ft = new FieldType(StoredField.TYPE);
+    ft.setStoreTermVectorOffsets(true);
+    doc = new Document();
+    doc.add(new Field("field", "value", ft));
+    try {
+      w.addDocument(doc);
+      fail("did not hit exception");
+    } catch (IllegalArgumentException iae) {
+      // Expected
+      assertEquals("cannot store term vector offsets for a field that is not indexed (field=\"field\")", iae.getMessage());
+    }
+
+    ft = new FieldType(StoredField.TYPE);
+    ft.setStoreTermVectorPayloads(true);
+    doc = new Document();
+    doc.add(new Field("field", "value", ft));
+    try {
+      w.addDocument(doc);
+      fail("did not hit exception");
+    } catch (IllegalArgumentException iae) {
+      // Expected
+      assertEquals("cannot store term vector payloads for a field that is not indexed (field=\"field\")", iae.getMessage());
+    }
+
     w.shutdown();
     
     dir.close();

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsWriter.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsWriter.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsWriter.java Wed Apr 30 07:59:31 2014
@@ -534,6 +534,7 @@ public class TestTermVectorsWriter exten
 
     FieldType customType = new FieldType(StringField.TYPE_NOT_STORED);
     customType.setStoreTermVectors(true);
+    document = new Document();
     document.add(newField("tvtest", "a b c", customType));
     iw.addDocument(document);
     // Make 2nd segment
@@ -567,6 +568,7 @@ public class TestTermVectorsWriter exten
     FieldType customType2 = new FieldType(StringField.TYPE_NOT_STORED);
     customType2.setStoreTermVectors(true);
     document.add(newField("tvtest", "a b c", customType2));
+    document = new Document();
     iw.addDocument(document);
     // Make 2nd segment
     iw.commit();

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java Wed Apr 30 07:59:31 2014
@@ -26,15 +26,10 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.English;
 import org.apache.lucene.util.IOUtils;
@@ -116,44 +111,14 @@ public class TestTermVectors extends Luc
     doc.add(newField("field", "one", ft3));
     doc.add(newField("field", "one", ft4));
     doc.add(newField("field", "one", ft5));
-    writer.addDocument(doc);
-    IndexReader reader = writer.getReader();
-    writer.shutdown();
-
-    IndexSearcher searcher = newSearcher(reader);
-
-    Query query = new TermQuery(new Term("field", "one"));
-    ScoreDoc[] hits = searcher.search(query, null, 1000).scoreDocs;
-    assertEquals(1, hits.length);
-
-    Fields vectors = searcher.reader.getTermVectors(hits[0].doc);
-    assertNotNull(vectors);
-    assertEquals(1, vectors.size());
-    Terms vector = vectors.terms("field");
-    assertNotNull(vector);
-    assertEquals(1, vector.size());
-    TermsEnum termsEnum = vector.iterator(null);
-    assertNotNull(termsEnum.next());
-    assertEquals("one", termsEnum.term().utf8ToString());
-    assertEquals(5, termsEnum.totalTermFreq());
-    DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null);
-    assertNotNull(dpEnum);
-    assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
-    assertEquals(5, dpEnum.freq());
-    for(int i=0;i<5;i++) {
-      assertEquals(i, dpEnum.nextPosition());
-    }
-
-    dpEnum = termsEnum.docsAndPositions(null, dpEnum);
-    assertNotNull(dpEnum);
-    assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
-    assertEquals(5, dpEnum.freq());
-    for(int i=0;i<5;i++) {
-      dpEnum.nextPosition();
-      assertEquals(4*i, dpEnum.startOffset());
-      assertEquals(4*i+3, dpEnum.endOffset());
+    try {
+      writer.addDocument(doc);
+      fail("should have hit exception");
+    } catch (IllegalArgumentException iae) {
+      assertEquals("all instances of a given field name must have the same term vectors settings (storeTermVectors changed for field=\"field\")",
+                   iae.getMessage());
     }
-    reader.close();
+    writer.shutdown();
   }
 
   private IndexWriter createWriter(Directory dir) throws IOException {

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/store/TestHugeRamFile.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/store/TestHugeRamFile.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/store/TestHugeRamFile.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/store/TestHugeRamFile.java Wed Apr 30 07:59:31 2014
@@ -54,7 +54,7 @@ public class TestHugeRamFile extends Luc
   public void testHugeFile() throws IOException {
     DenseRAMFile f = new DenseRAMFile();
     // output part
-    RAMOutputStream out = new RAMOutputStream(f);
+    RAMOutputStream out = new RAMOutputStream(f, true);
     byte b1[] = new byte[RAMOutputStream.BUFFER_SIZE];
     byte b2[] = new byte[RAMOutputStream.BUFFER_SIZE / 3];
     for (int i = 0; i < b1.length; i++) {

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java Wed Apr 30 07:59:31 2014
@@ -214,25 +214,6 @@ public class TestLockFactory extends Luc
       }
     }
 
-    public void testNativeFSLockReleaseByOtherLock() throws IOException {
-      NativeFSLockFactory f = new NativeFSLockFactory(createTempDir(LuceneTestCase.getTestClass().getSimpleName()));
-
-      f.setLockPrefix("test");
-      Lock l = f.makeLock("commit");
-      Lock l2 = f.makeLock("commit");
-
-      assertTrue("failed to obtain lock", l.obtain());
-      try {
-        assertTrue(l2.isLocked());
-        l2.close();
-        fail("should not have reached here. LockReleaseFailedException should have been thrown");
-      } catch (LockReleaseFailedException e) {
-        // expected
-      } finally {
-        l.close();
-      }
-    }
-
     // Verify: NativeFSLockFactory assigns null as lockPrefix if the lockDir is inside directory
     public void testNativeFSLockFactoryPrefix() throws IOException {
 

Modified: lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/util/TestAttributeSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/util/TestAttributeSource.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/util/TestAttributeSource.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/core/src/test/org/apache/lucene/util/TestAttributeSource.java Wed Apr 30 07:59:31 2014
@@ -90,7 +90,9 @@ public class TestAttributeSource extends
     assertFalse("No more attributes", it.hasNext());
     
     final FlagsAttribute flagsAtt2 = clone.getAttribute(FlagsAttribute.class);
+    assertNotNull(flagsAtt2);
     final TypeAttribute typeAtt2 = clone.getAttribute(TypeAttribute.class);
+    assertNotNull(typeAtt2);
     assertNotSame("FlagsAttribute of original and clone must be different instances", flagsAtt2, flagsAtt);
     assertNotSame("TypeAttribute of original and clone must be different instances", typeAtt2, typeAtt);
     assertEquals("FlagsAttribute of original and clone must be equal", flagsAtt2, flagsAtt);

Modified: lucene/dev/branches/LUCENE-5622/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingAtomicReader.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingAtomicReader.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingAtomicReader.java Wed Apr 30 07:59:31 2014
@@ -582,7 +582,7 @@ public class SortingAtomicReader extends
         file = new RAMFile();
         sorter = new DocOffsetSorter(maxDoc);
       }
-      final IndexOutput out = new RAMOutputStream(file);
+      final IndexOutput out = new RAMOutputStream(file, false);
       int doc;
       int i = 0;
       while ((doc = in.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {

Modified: lucene/dev/branches/LUCENE-5622/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java Wed Apr 30 07:59:31 2014
@@ -208,11 +208,11 @@ public class ComplexPhraseQueryParser ex
    */
   static class ComplexPhraseQuery extends Query {
 
-    String field;
+    final String field;
 
-    String phrasedQueryStringContents;
+    final String phrasedQueryStringContents;
 
-    int slopFactor;
+    final int slopFactor;
 
     private final boolean inOrder;
 
@@ -394,6 +394,7 @@ public class ComplexPhraseQueryParser ex
           + ((phrasedQueryStringContents == null) ? 0
               : phrasedQueryStringContents.hashCode());
       result = prime * result + slopFactor;
+      result = prime * result + (inOrder ? 1 : 0);
       return result;
     }
 
@@ -422,7 +423,7 @@ public class ComplexPhraseQueryParser ex
         return false;
       if (slopFactor != other.slopFactor)
         return false;
-      return true;
+      return inOrder == other.inOrder;
     }
   }
 }

Modified: lucene/dev/branches/LUCENE-5622/lucene/queryparser/src/test/org/apache/lucene/queryparser/complexPhrase/TestComplexPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/queryparser/src/test/org/apache/lucene/queryparser/complexPhrase/TestComplexPhraseQuery.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/queryparser/src/test/org/apache/lucene/queryparser/complexPhrase/TestComplexPhraseQuery.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/queryparser/src/test/org/apache/lucene/queryparser/complexPhrase/TestComplexPhraseQuery.java Wed Apr 30 07:59:31 2014
@@ -139,6 +139,31 @@ public class TestComplexPhraseQuery exte
     checkMatches("+role:developer +name:jack*", "");
     checkMatches("name:\"john smith\"~2 AND role:designer AND id:3", "3");
   }
+
+  public void testHashcodeEquals() throws Exception {
+    ComplexPhraseQueryParser qp = new ComplexPhraseQueryParser(TEST_VERSION_CURRENT, defaultFieldName, analyzer);
+    qp.setInOrder(true);
+    qp.setFuzzyPrefixLength(1);
+
+    String qString = "\"aaa* bbb*\"";
+
+    Query q = qp.parse(qString);
+    Query q2 = qp.parse(qString);
+
+    assertEquals(q.hashCode(), q2.hashCode());
+    assertEquals(q, q2);
+
+    qp.setInOrder(false); // SOLR-6011
+
+    q2 = qp.parse(qString);
+
+    // although the general contract of hashCode can't guarantee different values, if we only change one thing
+    // about a single query, it normally should result in a different value (and will with the current
+    // implementation in ComplexPhraseQuery)
+    assertTrue(q.hashCode() != q2.hashCode());
+    assertTrue(!q.equals(q2));
+    assertTrue(!q2.equals(q));
+  }
   
   @Override
   public void setUp() throws Exception {

Modified: lucene/dev/branches/LUCENE-5622/lucene/replicator/ivy.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/replicator/ivy.xml?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/replicator/ivy.xml (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/replicator/ivy.xml Wed Apr 30 07:59:31 2014
@@ -16,9 +16,6 @@
    specific language governing permissions and limitations
    under the License.    
 -->
-<!DOCTYPE ivy-module [
-  <!ENTITY jetty.version "8.1.10.v20130312">
-]>
 <ivy-module version="2.0">
   <info organisation="org.apache.lucene" module="replicator"/>
 

Modified: lucene/dev/branches/LUCENE-5622/lucene/replicator/src/java/org/apache/lucene/replicator/http/HttpClientBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/replicator/src/java/org/apache/lucene/replicator/http/HttpClientBase.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/replicator/src/java/org/apache/lucene/replicator/http/HttpClientBase.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/replicator/src/java/org/apache/lucene/replicator/http/HttpClientBase.java Wed Apr 30 07:59:31 2014
@@ -29,12 +29,12 @@ import org.apache.http.HttpEntity;
 import org.apache.http.HttpResponse;
 import org.apache.http.HttpStatus;
 import org.apache.http.StatusLine;
-import org.apache.http.client.HttpClient;
+import org.apache.http.client.config.RequestConfig;
 import org.apache.http.client.methods.HttpGet;
 import org.apache.http.client.methods.HttpPost;
-import org.apache.http.conn.ClientConnectionManager;
-import org.apache.http.impl.client.DefaultHttpClient;
-import org.apache.http.params.HttpConnectionParams;
+import org.apache.http.conn.HttpClientConnectionManager;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClientBuilder;
 import org.apache.http.util.EntityUtils;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.util.IOUtils;
@@ -46,18 +46,10 @@ import org.apache.lucene.util.IOUtils;
  * */
 public abstract class HttpClientBase implements Closeable {
   
-  /**
-   * Default connection timeout for this client, in milliseconds.
-   * 
-   * @see #setConnectionTimeout(int)
-   */
+  /** Default connection timeout for this client, in milliseconds. */
   public static final int DEFAULT_CONNECTION_TIMEOUT = 1000;
   
-  /**
-   * Default socket timeout for this client, in milliseconds.
-   * 
-   * @see #setSoTimeout(int)
-   */
+  /** Default socket timeout for this client, in milliseconds. */
   public static final int DEFAULT_SO_TIMEOUT = 60000;
   
   // TODO compression?
@@ -67,38 +59,29 @@ public abstract class HttpClientBase imp
   
   private volatile boolean closed = false;
   
-  private final HttpClient httpc;
+  private final CloseableHttpClient httpc;
+  private final RequestConfig defaultConfig;
   
   /**
-   * @param conMgr connection manager to use for this http client.
-   *        <b>NOTE:</b>The provided {@link ClientConnectionManager} will not be
-   *        {@link ClientConnectionManager#shutdown()} by this class.
+   * @param conMgr
+   *          connection manager to use for this http client. <b>NOTE:</b>The
+   *          provided {@link HttpClientConnectionManager} will not be
+   *          {@link HttpClientConnectionManager#shutdown()} by this class.
+   * @param defaultConfig
+   *          the default {@link RequestConfig} to set on the client. If
+   *          {@code null} a default config is created w/ the default connection
+   *          and socket timeouts.
    */
-  protected HttpClientBase(String host, int port, String path, ClientConnectionManager conMgr) {
+  protected HttpClientBase(String host, int port, String path, HttpClientConnectionManager conMgr, RequestConfig defaultConfig) {
     url = normalizedURL(host, port, path);
-    httpc = new DefaultHttpClient(conMgr);
-    setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
-    setSoTimeout(DEFAULT_SO_TIMEOUT);
-  }
-  
-  /**
-   * Set the connection timeout for this client, in milliseconds. This setting
-   * is used to modify {@link HttpConnectionParams#setConnectionTimeout}.
-   * 
-   * @param timeout timeout to set, in millisecopnds
-   */
-  public void setConnectionTimeout(int timeout) {
-    HttpConnectionParams.setConnectionTimeout(httpc.getParams(), timeout);
-  }
-  
-  /**
-   * Set the socket timeout for this client, in milliseconds. This setting
-   * is used to modify {@link HttpConnectionParams#setSoTimeout}.
-   * 
-   * @param timeout timeout to set, in millisecopnds
-   */
-  public void setSoTimeout(int timeout) {
-    HttpConnectionParams.setSoTimeout(httpc.getParams(), timeout);
+    if (defaultConfig == null) {
+      this.defaultConfig = RequestConfig.custom()
+          .setConnectionRequestTimeout(DEFAULT_CONNECTION_TIMEOUT)
+          .setSocketTimeout(DEFAULT_SO_TIMEOUT).build();
+    } else {
+      this.defaultConfig = defaultConfig;
+    }
+    httpc = HttpClientBuilder.create().setConnectionManager(conMgr).setDefaultRequestConfig(this.defaultConfig).build();
   }
   
   /** Throws {@link AlreadyClosedException} if this client is already closed. */
@@ -285,6 +268,7 @@ public abstract class HttpClientBase imp
   
   @Override
   public void close() throws IOException {
+    httpc.close();
     closed = true;
   }
   

Modified: lucene/dev/branches/LUCENE-5622/lucene/replicator/src/java/org/apache/lucene/replicator/http/HttpReplicator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/replicator/src/java/org/apache/lucene/replicator/http/HttpReplicator.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/replicator/src/java/org/apache/lucene/replicator/http/HttpReplicator.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/replicator/src/java/org/apache/lucene/replicator/http/HttpReplicator.java Wed Apr 30 07:59:31 2014
@@ -23,7 +23,7 @@ import java.io.InputStream;
 import java.util.concurrent.Callable;
 
 import org.apache.http.HttpResponse;
-import org.apache.http.conn.ClientConnectionManager;
+import org.apache.http.conn.HttpClientConnectionManager;
 import org.apache.lucene.replicator.Replicator;
 import org.apache.lucene.replicator.Revision;
 import org.apache.lucene.replicator.SessionToken;
@@ -38,8 +38,8 @@ import org.apache.lucene.replicator.http
 public class HttpReplicator extends HttpClientBase implements Replicator {
   
   /** Construct with specified connection manager. */
-  public HttpReplicator(String host, int port, String path, ClientConnectionManager conMgr) {
-    super(host, port, path, conMgr);
+  public HttpReplicator(String host, int port, String path, HttpClientConnectionManager conMgr) {
+    super(host, port, path, conMgr, null);
   }
   
   @Override

Modified: lucene/dev/branches/LUCENE-5622/lucene/replicator/src/test/org/apache/lucene/replicator/IndexAndTaxonomyRevisionTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/replicator/src/test/org/apache/lucene/replicator/IndexAndTaxonomyRevisionTest.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/replicator/src/test/org/apache/lucene/replicator/IndexAndTaxonomyRevisionTest.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/replicator/src/test/org/apache/lucene/replicator/IndexAndTaxonomyRevisionTest.java Wed Apr 30 07:59:31 2014
@@ -141,6 +141,7 @@ public class IndexAndTaxonomyRevisionTes
       Revision rev = new IndexAndTaxonomyRevision(indexWriter, taxoWriter);
       for (Entry<String,List<RevisionFile>> e : rev.getSourceFiles().entrySet()) {
         String source = e.getKey();
+        @SuppressWarnings("resource") // silly, both directories are closed in the end
         Directory dir = source.equals(IndexAndTaxonomyRevision.INDEX_SOURCE) ? indexDir : taxoDir;
         for (RevisionFile file : e.getValue()) {
           IndexInput src = dir.openInput(file.fileName, IOContext.READONCE);

Modified: lucene/dev/branches/LUCENE-5622/lucene/replicator/src/test/org/apache/lucene/replicator/ReplicatorTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/replicator/src/test/org/apache/lucene/replicator/ReplicatorTestCase.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/replicator/src/test/org/apache/lucene/replicator/ReplicatorTestCase.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/replicator/src/test/org/apache/lucene/replicator/ReplicatorTestCase.java Wed Apr 30 07:59:31 2014
@@ -19,8 +19,8 @@ package org.apache.lucene.replicator;
 
 import java.util.Random;
 
-import org.apache.http.conn.ClientConnectionManager;
-import org.apache.http.impl.conn.PoolingClientConnectionManager;
+import org.apache.http.conn.HttpClientConnectionManager;
+import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.eclipse.jetty.server.Connector;
@@ -38,7 +38,7 @@ import org.junit.AfterClass;
 @SuppressCodecs("Lucene3x")
 public abstract class ReplicatorTestCase extends LuceneTestCase {
   
-  private static ClientConnectionManager clientConnectionManager;
+  private static HttpClientConnectionManager clientConnectionManager;
   
   @AfterClass
   public static void afterClassReplicatorTestCase() throws Exception {
@@ -144,15 +144,15 @@ public abstract class ReplicatorTestCase
   }
   
   /**
-   * Returns a {@link ClientConnectionManager}.
+   * Returns a {@link HttpClientConnectionManager}.
    * <p>
-   * <b>NOTE:</b> do not {@link ClientConnectionManager#shutdown()} this
+   * <b>NOTE:</b> do not {@link HttpClientConnectionManager#shutdown()} this
    * connection manager, it will be shutdown automatically after all tests have
    * finished.
    */
-  public static synchronized ClientConnectionManager getClientConnectionManager() {
+  public static synchronized HttpClientConnectionManager getClientConnectionManager() {
     if (clientConnectionManager == null) {
-      PoolingClientConnectionManager ccm = new PoolingClientConnectionManager();
+      PoolingHttpClientConnectionManager ccm = new PoolingHttpClientConnectionManager();
       ccm.setDefaultMaxPerRoute(128);
       ccm.setMaxTotal(128);
       clientConnectionManager = ccm;

Modified: lucene/dev/branches/LUCENE-5622/lucene/replicator/src/test/org/apache/lucene/replicator/http/HttpReplicatorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/replicator/src/test/org/apache/lucene/replicator/http/HttpReplicatorTest.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/replicator/src/test/org/apache/lucene/replicator/http/HttpReplicatorTest.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/replicator/src/test/org/apache/lucene/replicator/http/HttpReplicatorTest.java Wed Apr 30 07:59:31 2014
@@ -21,7 +21,7 @@ import java.io.File;
 import java.io.IOException;
 import java.util.Collections;
 
-import org.apache.http.impl.conn.BasicClientConnectionManager;
+import org.apache.http.impl.conn.BasicHttpClientConnectionManager;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
@@ -130,7 +130,7 @@ public class HttpReplicatorTest extends 
   public void testServerErrors() throws Exception {
     // tests the behaviour of the client when the server sends an error
     // must use BasicClientConnectionManager to test whether the client is closed correctly
-    BasicClientConnectionManager conMgr = new BasicClientConnectionManager();
+    BasicHttpClientConnectionManager conMgr = new BasicHttpClientConnectionManager();
     Replicator replicator = new HttpReplicator(host, port, ReplicationService.REPLICATION_CONTEXT + "/s1", conMgr);
     ReplicationClient client = new ReplicationClient(replicator, new IndexReplicationHandler(handlerIndexDir, null), 
         new PerSessionDirectoryFactory(clientWorkDir));

Modified: lucene/dev/branches/LUCENE-5622/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java Wed Apr 30 07:59:31 2014
@@ -685,11 +685,11 @@ public abstract class BaseTokenStreamTes
     int remainder = random.nextInt(10);
     Reader reader = new StringReader(text);
     TokenStream ts = a.tokenStream("dummy", useCharFilter ? new MockCharFilter(reader, remainder) : reader);
-    CharTermAttribute termAtt = ts.hasAttribute(CharTermAttribute.class) ? ts.getAttribute(CharTermAttribute.class) : null;
-    OffsetAttribute offsetAtt = ts.hasAttribute(OffsetAttribute.class) ? ts.getAttribute(OffsetAttribute.class) : null;
-    PositionIncrementAttribute posIncAtt = ts.hasAttribute(PositionIncrementAttribute.class) ? ts.getAttribute(PositionIncrementAttribute.class) : null;
-    PositionLengthAttribute posLengthAtt = ts.hasAttribute(PositionLengthAttribute.class) ? ts.getAttribute(PositionLengthAttribute.class) : null;
-    TypeAttribute typeAtt = ts.hasAttribute(TypeAttribute.class) ? ts.getAttribute(TypeAttribute.class) : null;
+    CharTermAttribute termAtt = ts.getAttribute(CharTermAttribute.class);
+    OffsetAttribute offsetAtt = ts.getAttribute(OffsetAttribute.class);
+    PositionIncrementAttribute posIncAtt = ts.getAttribute(PositionIncrementAttribute.class);
+    PositionLengthAttribute posLengthAtt = ts.getAttribute(PositionLengthAttribute.class);
+    TypeAttribute typeAtt = ts.getAttribute(TypeAttribute.class);
     List<String> tokens = new ArrayList<>();
     List<String> types = new ArrayList<>();
     List<Integer> positions = new ArrayList<>();

Modified: lucene/dev/branches/LUCENE-5622/lucene/test-framework/src/java/org/apache/lucene/analysis/ValidatingTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/test-framework/src/java/org/apache/lucene/analysis/ValidatingTokenFilter.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/test-framework/src/java/org/apache/lucene/analysis/ValidatingTokenFilter.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/test-framework/src/java/org/apache/lucene/analysis/ValidatingTokenFilter.java Wed Apr 30 07:59:31 2014
@@ -25,7 +25,6 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
-import org.apache.lucene.util.Attribute;
 
 // TODO: rename to OffsetsXXXTF?  ie we only validate
 // offsets (now anyway...)
@@ -47,23 +46,14 @@ public final class ValidatingTokenFilter
   private final Map<Integer,Integer> posToStartOffset = new HashMap<>();
   private final Map<Integer,Integer> posToEndOffset = new HashMap<>();
 
-  private final PositionIncrementAttribute posIncAtt = getAttrIfExists(PositionIncrementAttribute.class);
-  private final PositionLengthAttribute posLenAtt = getAttrIfExists(PositionLengthAttribute.class);
-  private final OffsetAttribute offsetAtt = getAttrIfExists(OffsetAttribute.class);
-  private final CharTermAttribute termAtt = getAttrIfExists(CharTermAttribute.class);
+  private final PositionIncrementAttribute posIncAtt = getAttribute(PositionIncrementAttribute.class);
+  private final PositionLengthAttribute posLenAtt = getAttribute(PositionLengthAttribute.class);
+  private final OffsetAttribute offsetAtt = getAttribute(OffsetAttribute.class);
+  private final CharTermAttribute termAtt = getAttribute(CharTermAttribute.class);
   private final boolean offsetsAreCorrect;
 
   private final String name;
 
-  // Returns null if the attr wasn't already added
-  private <A extends Attribute> A getAttrIfExists(Class<A> att) {
-    if (hasAttribute(att)) {
-      return getAttribute(att);
-    } else {
-      return null;
-    }
-  }
-
   /** The name arg is used to identify this stage when
    *  throwing exceptions (useful if you have more than one
    *  instance in your chain). */

Modified: lucene/dev/branches/LUCENE-5622/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java Wed Apr 30 07:59:31 2014
@@ -90,7 +90,6 @@ public class AssertingStoredFieldsFormat
   static class AssertingStoredFieldsWriter extends StoredFieldsWriter {
     private final StoredFieldsWriter in;
     private int numWritten;
-    private int fieldCount;
     private Status docStatus;
     
     AssertingStoredFieldsWriter(StoredFieldsWriter in) {
@@ -99,11 +98,9 @@ public class AssertingStoredFieldsFormat
     }
 
     @Override
-    public void startDocument(int numStoredFields) throws IOException {
+    public void startDocument() throws IOException {
       assert docStatus != Status.STARTED;
-      in.startDocument(numStoredFields);
-      assert fieldCount == 0;
-      fieldCount = numStoredFields;
+      in.startDocument();
       numWritten++;
       docStatus = Status.STARTED;
     }
@@ -111,7 +108,6 @@ public class AssertingStoredFieldsFormat
     @Override
     public void finishDocument() throws IOException {
       assert docStatus == Status.STARTED;
-      assert fieldCount == 0;
       in.finishDocument();
       docStatus = Status.FINISHED;
     }
@@ -120,8 +116,6 @@ public class AssertingStoredFieldsFormat
     public void writeField(FieldInfo info, StorableField field) throws IOException {
       assert docStatus == Status.STARTED;
       in.writeField(info, field);
-      assert fieldCount > 0;
-      fieldCount--;
     }
 
     @Override
@@ -133,7 +127,6 @@ public class AssertingStoredFieldsFormat
     public void finish(FieldInfos fis, int numDocs) throws IOException {
       assert docStatus == (numDocs > 0 ? Status.FINISHED : Status.UNDEFINED);
       in.finish(fis, numDocs);
-      assert fieldCount == 0;
       assert numDocs == numWritten;
     }
 

Modified: lucene/dev/branches/LUCENE-5622/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java Wed Apr 30 07:59:31 2014
@@ -91,6 +91,20 @@ public class AssertingAtomicReader exten
     }
 
     @Override
+    public BytesRef getMin() throws IOException {
+      BytesRef v = in.getMin();
+      assert v == null || v.isValid();
+      return v;
+    }
+
+    @Override
+    public BytesRef getMax() throws IOException {
+      BytesRef v = in.getMax();
+      assert v == null || v.isValid();
+      return v;
+    }
+
+    @Override
     public TermsEnum iterator(TermsEnum reuse) throws IOException {
       // TODO: should we give this thing a random to be super-evil,
       // and randomly *not* unwrap?

Modified: lucene/dev/branches/LUCENE-5622/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5622/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1591209&r1=1591208&r2=1591209&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5622/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/branches/LUCENE-5622/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java Wed Apr 30 07:59:31 2014
@@ -79,7 +79,7 @@ public class MockDirectoryWrapper extend
   private Set<String> unSyncedFiles;
   private Set<String> createdFiles;
   private Set<String> openFilesForWrite = new HashSet<>();
-  Set<String> openLocks = Collections.synchronizedSet(new HashSet<String>());
+  Map<String,Exception> openLocks = Collections.synchronizedMap(new HashMap<String,Exception>());
   volatile boolean crashed;
   private ThrottledIndexOutput throttledOutput;
   private Throttling throttling = Throttling.SOMETIMES;
@@ -655,14 +655,20 @@ public class MockDirectoryWrapper extend
       // print the first one as its very verbose otherwise
       Exception cause = null;
       Iterator<Exception> stacktraces = openFileHandles.values().iterator();
-      if (stacktraces.hasNext())
+      if (stacktraces.hasNext()) {
         cause = stacktraces.next();
+      }
       // RuntimeException instead of IOException because
       // super() does not throw IOException currently:
       throw new RuntimeException("MockDirectoryWrapper: cannot close: there are still open files: " + openFiles, cause);
     }
     if (openLocks.size() > 0) {
-      throw new RuntimeException("MockDirectoryWrapper: cannot close: there are still open locks: " + openLocks);
+      Exception cause = null;
+      Iterator<Exception> stacktraces = openLocks.values().iterator();
+      if (stacktraces.hasNext()) {
+        cause = stacktraces.next();
+      }
+      throw new RuntimeException("MockDirectoryWrapper: cannot close: there are still open locks: " + openLocks, cause);
     }
 
     isOpen = false;