You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2015/06/04 21:24:08 UTC

svn commit: r1683615 [6/12] - in /lucene/dev/branches/LUCENE-6481: ./ dev-tools/ dev-tools/idea/.idea/ dev-tools/maven/solr/core/src/test/ dev-tools/scripts/ lucene/ lucene/codecs/ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/ lucene/core...

Modified: lucene/dev/branches/LUCENE-6481/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/geo3d/PlaneTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/geo3d/PlaneTest.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/geo3d/PlaneTest.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/geo3d/PlaneTest.java Thu Jun  4 19:24:05 2015
@@ -30,7 +30,7 @@ public class PlaneTest {
 
   @Test
   public void testIdenticalPlanes() {
-    final GeoPoint p = new GeoPoint(0.123, -0.456);
+    final GeoPoint p = new GeoPoint(PlanetModel.SPHERE, 0.123, -0.456);
     final Plane plane1 = new Plane(p, 0.0);
     final Plane plane2 = new Plane(p, 0.0);
     assertTrue(plane1.isNumericallyIdentical(plane2));

Modified: lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java Thu Jun  4 19:24:05 2015
@@ -17,22 +17,10 @@ package org.apache.lucene.search.suggest
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.Map;
-import java.util.Set;
-
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.AnalyzerWrapper;
-import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.TokenStreamToAutomaton;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.Operations;
-import org.apache.lucene.util.automaton.Transition;
 
 /**
  * Wraps an {@link org.apache.lucene.analysis.Analyzer}
@@ -40,15 +28,15 @@ import org.apache.lucene.util.automaton.
  * (e.g. preserving token separators, preserving position increments while converting
  * a token stream to an automaton)
  * <p>
- * Can be used to index {@link SuggestField}
- * and as a query analyzer to {@link SuggestIndexSearcher}
+ * Can be used to index {@link SuggestField} and {@link ContextSuggestField}
+ * and as a query analyzer to {@link PrefixCompletionQuery} amd {@link FuzzyCompletionQuery}
  * <p>
- * NOTE: In most cases, index and query analyzer should have same values for {@link #preservePositionIncrements}
- * and {@link #preserveSep}
+ * NOTE: In most cases, index and query analyzer should have same values for {@link #preservePositionIncrements()}
+ * and {@link #preserveSep()}
  *
  * @lucene.experimental
  */
-public class CompletionAnalyzer extends AnalyzerWrapper {
+public final class CompletionAnalyzer extends AnalyzerWrapper {
 
   /**
    * Represents the separation between tokens, if
@@ -64,7 +52,7 @@ public class CompletionAnalyzer extends
    */
   final static int HOLE_CHARACTER = TokenStreamToAutomaton.HOLE;
 
-  final static int DEFAULT_MAX_GRAPH_EXPANSIONS = -1;
+  final static int DEFAULT_MAX_GRAPH_EXPANSIONS = Operations.DEFAULT_MAX_DETERMINIZED_STATES;
   final static boolean DEFAULT_PRESERVE_SEP = true;
   final static boolean DEFAULT_PRESERVE_POSITION_INCREMENTS = true;
 
@@ -133,6 +121,22 @@ public class CompletionAnalyzer extends
     this(analyzer, DEFAULT_PRESERVE_SEP, DEFAULT_PRESERVE_POSITION_INCREMENTS, maxGraphExpansions);
   }
 
+  /**
+   * Returns true if separation between tokens are preserved when converting
+   * the token stream to an automaton
+   */
+  public boolean preserveSep() {
+    return preserveSep;
+  }
+
+  /**
+   * Returns true if position increments are preserved when converting
+   * the token stream to an automaton
+   */
+  public boolean preservePositionIncrements() {
+    return preservePositionIncrements;
+  }
+
   @Override
   protected Analyzer getWrappedAnalyzer(String fieldName) {
     return analyzer;
@@ -141,33 +145,7 @@ public class CompletionAnalyzer extends
   @Override
   protected TokenStreamComponents wrapComponents(String fieldName, TokenStreamComponents components) {
     CompletionTokenStream tokenStream = new CompletionTokenStream(components.getTokenStream(),
-        preserveSep, preservePositionIncrements, SEP_LABEL, maxGraphExpansions);
+        preserveSep, preservePositionIncrements, maxGraphExpansions);
     return new TokenStreamComponents(components.getTokenizer(), tokenStream);
   }
-
-  /**
-   * Converts <code>key</code> to an automaton using
-   * {@link #preservePositionIncrements}, {@link #preserveSep}
-   * and {@link #maxGraphExpansions}
-   */
-  public Automaton toAutomaton(String field, CharSequence key) throws IOException {
-    for (int i = 0; i < key.length(); i++) {
-      switch (key.charAt(i)) {
-        case HOLE_CHARACTER:
-          throw new IllegalArgumentException("lookup key cannot contain HOLE character U+001E; this character is reserved");
-        case SEP_LABEL:
-          throw new IllegalArgumentException("lookup key cannot contain unit separator character U+001F; this character is reserved");
-        default:
-          break;
-      }
-    }
-
-    try (TokenStream tokenStream = analyzer.tokenStream(field, key.toString())) {
-      try(CompletionTokenStream stream = new CompletionTokenStream(tokenStream,
-          preserveSep, preservePositionIncrements, SEP_LABEL, maxGraphExpansions)) {
-        return stream.toAutomaton(tokenStream);
-      }
-    }
-  }
-
 }

Modified: lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsConsumer.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsConsumer.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsConsumer.java Thu Jun  4 19:24:05 2015
@@ -58,7 +58,7 @@ import static org.apache.lucene.search.s
 final class CompletionFieldsConsumer extends FieldsConsumer {
 
   private final String delegatePostingsFormatName;
-  private final Map<String, Long> seenFields = new HashMap<>();
+  private final Map<String, CompletionMetaData> seenFields = new HashMap<>();
   private final SegmentWriteState state;
   private IndexOutput dictOut;
   private FieldsConsumer delegateFieldsConsumer;
@@ -98,7 +98,10 @@ final class CompletionFieldsConsumer ext
       // store lookup, if needed
       long filePointer = dictOut.getFilePointer();
       if (termWriter.finish(dictOut)) {
-        seenFields.put(field, filePointer);
+        seenFields.put(field, new CompletionMetaData(filePointer,
+            termWriter.minWeight,
+            termWriter.maxWeight,
+            termWriter.type));
       }
     }
   }
@@ -124,10 +127,14 @@ final class CompletionFieldsConsumer ext
       // write # of seen fields
       indexOut.writeVInt(seenFields.size());
       // write field numbers and dictOut offsets
-      for (Map.Entry<String, Long> seenField : seenFields.entrySet()) {
+      for (Map.Entry<String, CompletionMetaData> seenField : seenFields.entrySet()) {
         FieldInfo fieldInfo = state.fieldInfos.fieldInfo(seenField.getKey());
         indexOut.writeVInt(fieldInfo.number);
-        indexOut.writeVLong(seenField.getValue());
+        CompletionMetaData metaData = seenField.getValue();
+        indexOut.writeVLong(metaData.filePointer);
+        indexOut.writeVLong(metaData.minWeight);
+        indexOut.writeVLong(metaData.maxWeight);
+        indexOut.writeByte(metaData.type);
       }
       CodecUtil.writeFooter(indexOut);
       CodecUtil.writeFooter(dictOut);
@@ -140,17 +147,36 @@ final class CompletionFieldsConsumer ext
     }
   }
 
+  private static class CompletionMetaData {
+    private final long filePointer;
+    private final long minWeight;
+    private final long maxWeight;
+    private final byte type;
+
+    private CompletionMetaData(long filePointer, long minWeight, long maxWeight, byte type) {
+      this.filePointer = filePointer;
+      this.minWeight = minWeight;
+      this.maxWeight = maxWeight;
+      this.type = type;
+    }
+  }
+
   // builds an FST based on the terms written
   private static class CompletionTermWriter {
 
     private PostingsEnum postingsEnum = null;
     private int docCount = 0;
+    private long maxWeight = 0;
+    private long minWeight = Long.MAX_VALUE;
+    private byte type;
+    private boolean first;
 
     private final BytesRefBuilder scratch = new BytesRefBuilder();
     private final NRTSuggesterBuilder builder;
 
     public CompletionTermWriter() {
       builder = new NRTSuggesterBuilder();
+      first = true;
     }
 
     /**
@@ -160,6 +186,9 @@ final class CompletionFieldsConsumer ext
     public boolean finish(IndexOutput output) throws IOException {
       boolean stored = builder.store(output);
       assert stored || docCount == 0 : "the FST is null but docCount is != 0 actual value: [" + docCount + "]";
+      if (docCount == 0) {
+        minWeight = 0;
+      }
       return stored;
     }
 
@@ -181,7 +210,17 @@ final class CompletionFieldsConsumer ext
           scratch.grow(len);
           scratch.setLength(len);
           input.readBytes(scratch.bytes(), 0, scratch.length());
-          builder.addEntry(docID, scratch.get(), input.readVLong() - 1);
+          long weight = input.readVInt() - 1;
+          maxWeight = Math.max(maxWeight, weight);
+          minWeight = Math.min(minWeight, weight);
+          byte type = input.readByte();
+          if (first) {
+            this.type = type;
+            first = false;
+          } else if (this.type != type) {
+            throw new IllegalArgumentException("single field name has mixed types");
+          }
+          builder.addEntry(docID, scratch.get(), weight);
         }
         docFreq++;
         docCount = Math.max(docCount, docFreq + 1);

Modified: lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java Thu Jun  4 19:24:05 2015
@@ -30,7 +30,6 @@ import org.apache.lucene.codecs.CodecUti
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FilterLeafReader;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.Terms;
@@ -98,9 +97,12 @@ final class CompletionFieldsProducer ext
       for (int i = 0; i < numFields; i++) {
         int fieldNumber = index.readVInt();
         long offset = index.readVLong();
+        long minWeight = index.readVLong();
+        long maxWeight = index.readVLong();
+        byte type = index.readByte();
         FieldInfo fieldInfo = state.fieldInfos.fieldInfo(fieldNumber);
         // we don't load the FST yet
-        readers.put(fieldInfo.name, new CompletionsTermsReader(offset));
+        readers.put(fieldInfo.name, new CompletionsTermsReader(dictIn, offset, minWeight, maxWeight, type));
       }
       CodecUtil.checkFooter(index);
       success = true;
@@ -161,7 +163,11 @@ final class CompletionFieldsProducer ext
 
   @Override
   public Terms terms(String field) throws IOException {
-    return new CompletionTerms(delegateFieldsProducer.terms(field), readers.get(field));
+    Terms terms = delegateFieldsProducer.terms(field) ;
+    if (terms == null) {
+      return null;
+    }
+    return new CompletionTerms(terms, readers.get(field));
   }
 
   @Override
@@ -169,60 +175,4 @@ final class CompletionFieldsProducer ext
     return readers.size();
   }
 
-  private class CompletionsTermsReader implements Accountable {
-    private final long offset;
-    private NRTSuggester suggester;
-
-    public CompletionsTermsReader(long offset) throws IOException {
-      assert offset >= 0l && offset < dictIn.length();
-      this.offset = offset;
-    }
-
-    public synchronized NRTSuggester suggester() throws IOException {
-      if (suggester == null) {
-        try (IndexInput dictClone = dictIn.clone()) { // let multiple fields load concurrently
-          dictClone.seek(offset);
-          suggester = NRTSuggester.load(dictClone);
-        }
-      }
-      return suggester;
-    }
-
-    @Override
-    public long ramBytesUsed() {
-      return (suggester != null) ? suggester.ramBytesUsed() : 0;
-    }
-
-    @Override
-    public Collection<Accountable> getChildResources() {
-      return Collections.emptyList();
-    }
-  }
-
-  /**
-   * Thin wrapper over {@link org.apache.lucene.index.Terms} with
-   * a {@link NRTSuggester}
-   */
-  public static class CompletionTerms extends FilterLeafReader.FilterTerms {
-
-    private final CompletionsTermsReader reader;
-
-    public CompletionTerms(Terms in, CompletionsTermsReader reader) {
-      super(in);
-      this.reader = reader;
-    }
-
-    /**
-     * Returns a {@link NRTSuggester} for the field
-     * or <code>null</code> if no FST
-     * was indexed for this field
-     */
-    public NRTSuggester suggester() throws IOException {
-      if (reader == null) {
-        return null;
-      }
-      return reader.suggester();
-    }
-  }
-
 }

Modified: lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java Thu Jun  4 19:24:05 2015
@@ -68,9 +68,12 @@ import org.apache.lucene.util.fst.FST;
  *   <li>CompletionIndex (.cmp) --&gt; Header, NumSuggestFields, Entry<sup>NumSuggestFields</sup>, Footer</li>
  *   <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
  *   <li>NumSuggestFields --&gt; {@link DataOutput#writeVInt Uint32}</li>
- *   <li>Entry --&gt; FieldNumber, CompletionDictionaryOffset</li>
+ *   <li>Entry --&gt; FieldNumber, CompletionDictionaryOffset, MinWeight, MaxWeight, Type</li>
  *   <li>FieldNumber --&gt; {@link DataOutput#writeVInt Uint32}</li>
  *   <li>CompletionDictionaryOffset --&gt; {@link DataOutput#writeVLong  Uint64}</li>
+ *   <li>MinWeight --&gt; {@link DataOutput#writeVLong  Uint64}</li>
+ *   <li>MaxWeight --&gt; {@link DataOutput#writeVLong  Uint64}</li>
+ *   <li>Type --&gt; {@link DataOutput#writeByte  Byte}</li>
  *   <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
  * </ul>
  * <p>Notes:</p>
@@ -80,6 +83,8 @@ import org.apache.lucene.util.fst.FST;
  *   <li>NumSuggestFields is the number of suggest fields indexed</li>
  *   <li>FieldNumber is the fields number from {@link FieldInfos}. (.fnm)</li>
  *   <li>CompletionDictionaryOffset is the file offset of a field's FST in CompletionDictionary (.lkp)</li>
+ *   <li>MinWeight and MaxWeight are the global minimum and maximum weight for the field</li>
+ *   <li>Type indicates if the suggester has context or not</li>
  * </ul>
  *
  * @lucene.experimental

Modified: lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java Thu Jun  4 19:24:05 2015
@@ -18,7 +18,7 @@ package org.apache.lucene.search.suggest
  */
 
 import java.io.IOException;
-import java.util.HashSet;
+import java.util.BitSet;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.Set;
@@ -52,18 +52,18 @@ import static org.apache.lucene.search.s
  * The token stream uses a {@link org.apache.lucene.analysis.tokenattributes.PayloadAttribute} to store
  * a completion's payload (see {@link CompletionTokenStream#setPayload(org.apache.lucene.util.BytesRef)})
  *
+ * @lucene.experimental
  */
-final class CompletionTokenStream extends TokenStream {
+public final class CompletionTokenStream extends TokenStream {
 
   private final PayloadAttribute payloadAttr = addAttribute(PayloadAttribute.class);
   private final PositionIncrementAttribute posAttr = addAttribute(PositionIncrementAttribute.class);
   private final ByteTermAttribute bytesAtt = addAttribute(ByteTermAttribute.class);
 
   private final TokenStream input;
-  private final boolean preserveSep;
-  private final boolean preservePositionIncrements;
-  private final int sepLabel;
-  private final int maxGraphExpansions;
+  final boolean preserveSep;
+  final boolean preservePositionIncrements;
+  final int maxGraphExpansions;
 
   private BytesRef payload;
   private Iterator<IntsRef> finiteStrings;
@@ -77,29 +77,20 @@ final class CompletionTokenStream extend
    * The token stream <code>input</code> is converted to an automaton
    * with the default settings of {@link org.apache.lucene.search.suggest.document.CompletionAnalyzer}
    */
-  public CompletionTokenStream(TokenStream input) {
-    this(input, DEFAULT_PRESERVE_SEP, DEFAULT_PRESERVE_POSITION_INCREMENTS, SEP_LABEL, DEFAULT_MAX_GRAPH_EXPANSIONS);
+  CompletionTokenStream(TokenStream input) {
+    this(input, DEFAULT_PRESERVE_SEP, DEFAULT_PRESERVE_POSITION_INCREMENTS, DEFAULT_MAX_GRAPH_EXPANSIONS);
   }
 
-  CompletionTokenStream(TokenStream input, boolean preserveSep, boolean preservePositionIncrements, int sepLabel, int maxGraphExpansions) {
+  CompletionTokenStream(TokenStream input, boolean preserveSep, boolean preservePositionIncrements, int maxGraphExpansions) {
     // Don't call the super(input) ctor - this is a true delegate and has a new attribute source since we consume
     // the input stream entirely in toFiniteStrings(input)
     this.input = input;
     this.preserveSep = preserveSep;
     this.preservePositionIncrements = preservePositionIncrements;
-    this.sepLabel = sepLabel;
     this.maxGraphExpansions = maxGraphExpansions;
   }
 
   /**
-   * Returns a separator label that is reserved for the payload
-   * in {@link CompletionTokenStream#setPayload(org.apache.lucene.util.BytesRef)}
-   */
-  public int sepLabel() {
-    return sepLabel;
-  }
-
-  /**
    * Sets a payload available throughout successive token stream enumeration
    */
   public void setPayload(BytesRef payload) {
@@ -111,7 +102,7 @@ final class CompletionTokenStream extend
     clearAttributes();
     if (finiteStrings == null) {
       //TODO: make this return a Iterator<IntsRef> instead?
-      Automaton automaton = toAutomaton(input);
+      Automaton automaton = toAutomaton();
       Set<IntsRef> strings = Operations.getFiniteStrings(automaton, maxGraphExpansions);
 
       posInc = strings.size();
@@ -165,9 +156,17 @@ final class CompletionTokenStream extend
   }
 
   /**
-   * Converts <code>tokenStream</code> to an automaton
+   * Converts the token stream to an automaton,
+   * treating the transition labels as utf-8
+   */
+  public Automaton toAutomaton() throws IOException {
+    return toAutomaton(false);
+  }
+
+  /**
+   * Converts the tokenStream to an automaton
    */
-  public Automaton toAutomaton(TokenStream tokenStream) throws IOException {
+  public Automaton toAutomaton(boolean unicodeAware) throws IOException {
     // TODO refactor this
     // maybe we could hook up a modified automaton from TermAutomatonQuery here?
     Automaton automaton = null;
@@ -184,10 +183,11 @@ final class CompletionTokenStream extend
         tsta = new TokenStreamToAutomaton();
       }
       tsta.setPreservePositionIncrements(preservePositionIncrements);
+      tsta.setUnicodeArcs(unicodeAware);
 
-      automaton = tsta.toAutomaton(tokenStream);
+      automaton = tsta.toAutomaton(input);
     } finally {
-      IOUtils.closeWhileHandlingException(tokenStream);
+      IOUtils.closeWhileHandlingException(input);
     }
 
     // TODO: we can optimize this somewhat by determinizing
@@ -281,11 +281,12 @@ final class CompletionTokenStream extend
   }
 
   private static int[] topoSortStates(Automaton a) {
-    int[] states = new int[a.getNumStates()];
-    final Set<Integer> visited = new HashSet<>();
+    int numStates = a.getNumStates();
+    int[] states = new int[numStates];
+    final BitSet visited = new BitSet(numStates);
     final LinkedList<Integer> worklist = new LinkedList<>();
     worklist.add(0);
-    visited.add(0);
+    visited.set(0);
     int upto = 0;
     states[upto] = 0;
     upto++;
@@ -293,10 +294,10 @@ final class CompletionTokenStream extend
     while (worklist.size() > 0) {
       int s = worklist.removeFirst();
       int count = a.initTransition(s, t);
-      for (int i = 0; i < count; i++) {
+      for (int i=0;i<count;i++) {
         a.getNextTransition(t);
-        if (!visited.contains(t.dest)) {
-          visited.add(t.dest);
+        if (!visited.get(t.dest)) {
+          visited.set(t.dest);
           worklist.add(t.dest);
           states[upto++] = t.dest;
         }
@@ -305,21 +306,37 @@ final class CompletionTokenStream extend
     return states;
   }
 
-  public interface ByteTermAttribute extends TermToBytesRefAttribute {
+  /**
+   * Attribute providing access to the term builder and UTF-16 conversion
+   */
+  private interface ByteTermAttribute extends TermToBytesRefAttribute {
     // marker interface
 
     /**
-     * Return the builder from which the term is derived.
+     * Returns the builder from which the term is derived.
      */
-    public BytesRefBuilder builder();
+    BytesRefBuilder builder();
 
-    public CharSequence toUTF16();
+    /**
+     * Returns the term represented as UTF-16
+     */
+    CharSequence toUTF16();
   }
 
+  /**
+   * Custom attribute implementation for completion token stream
+   */
   public static final class ByteTermAttributeImpl extends AttributeImpl implements ByteTermAttribute, TermToBytesRefAttribute {
     private final BytesRefBuilder bytes = new BytesRefBuilder();
     private CharsRefBuilder charsRef;
 
+    /**
+     * Sole constructor
+     * no-op
+     */
+    public ByteTermAttributeImpl() {
+    }
+
     @Override
     public void fillBytesRef() {
       // does nothing - we change in place

Modified: lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggester.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggester.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggester.java Thu Jun  4 19:24:05 2015
@@ -23,19 +23,13 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
 
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.search.CollectionTerminatedException;
-import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.suggest.analyzing.FSTUtil;
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.ByteArrayDataOutput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Accountable;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRefBuilder;
-import org.apache.lucene.util.IntsRef;
-import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.fst.ByteSequenceOutputs;
 import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.PairOutputs;
@@ -48,18 +42,11 @@ import static org.apache.lucene.search.s
 
 /**
  * <p>
- * NRTSuggester returns Top N completions with corresponding documents matching a provided automaton.
- * The completions are returned in descending order of their corresponding weight.
- * Deleted documents are filtered out in near real time using the provided reader.
- * A {@link org.apache.lucene.search.DocIdSet} can be passed in at query time to filter out documents.
- * </p>
+ * NRTSuggester executes Top N search on a weighted FST specified by a {@link CompletionScorer}
  * <p>
- * See {@link #lookup(LeafReader, Automaton, int, DocIdSet, TopSuggestDocsCollector)} for more implementation
+ * See {@link #lookup(CompletionScorer, TopSuggestDocsCollector)} for more implementation
  * details.
  * <p>
- * Builder: {@link NRTSuggesterBuilder}
- * </p>
- * <p>
  * FST Format:
  * <ul>
  *   <li>Input: analyzed forms of input terms</li>
@@ -68,16 +55,17 @@ import static org.apache.lucene.search.s
  * <p>
  * NOTE:
  * <ul>
- *   <li>currently only {@link org.apache.lucene.search.DocIdSet} with random access capabilities are supported.</li>
  *   <li>having too many deletions or using a very restrictive filter can make the search inadmissible due to
- *     over-pruning of potential paths</li>
- *   <li>when a {@link org.apache.lucene.search.DocIdSet} is used, it is assumed that the filter will roughly
- *     filter out half the number of documents that match the provided automaton</li>
+ *     over-pruning of potential paths. See {@link CompletionScorer#accept(int)}</li>
+ *   <li>when matched documents are arbitrarily filtered ({@link CompletionScorer#filtered} set to <code>true</code>,
+ *     it is assumed that the filter will roughly filter out half the number of documents that match
+ *     the provided automaton</li>
  *   <li>lookup performance will degrade as more accepted completions lead to filtered out documents</li>
  * </ul>
  *
+ * @lucene.experimental
  */
-final class NRTSuggester implements Accountable {
+public final class NRTSuggester implements Accountable {
 
   /**
    * FST<Weight,Surface>:
@@ -103,23 +91,16 @@ final class NRTSuggester implements Acco
   private final int payloadSep;
 
   /**
-   * Label used to denote the end of an input in the FST and
-   * the beginning of dedup bytes
-   */
-  private final int endByte;
-
-  /**
    * Maximum queue depth for TopNSearcher
    *
    * NOTE: value should be <= Integer.MAX_VALUE
    */
-  private static final long MAX_TOP_N_QUEUE_SIZE = 1000;
+  private static final long MAX_TOP_N_QUEUE_SIZE = 5000;
 
-  private NRTSuggester(FST<Pair<Long, BytesRef>> fst, int maxAnalyzedPathsPerOutput, int payloadSep, int endByte) {
+  private NRTSuggester(FST<Pair<Long, BytesRef>> fst, int maxAnalyzedPathsPerOutput, int payloadSep) {
     this.fst = fst;
     this.maxAnalyzedPathsPerOutput = maxAnalyzedPathsPerOutput;
     this.payloadSep = payloadSep;
-    this.endByte = endByte;
   }
 
   @Override
@@ -132,102 +113,90 @@ final class NRTSuggester implements Acco
     return Collections.emptyList();
   }
 
-  private static Comparator<Pair<Long, BytesRef>> getComparator() {
-    return new Comparator<Pair<Long, BytesRef>>() {
-      @Override
-      public int compare(Pair<Long, BytesRef> o1, Pair<Long, BytesRef> o2) {
-        return Long.compare(o1.output1, o2.output1);
-      }
-    };
-  }
-
   /**
-   * Collects at most Top <code>num</code> completions, filtered by <code>filter</code> on
-   * corresponding documents, which has a prefix accepted by <code>automaton</code>
-   * <p>
-   * Supports near real time deleted document filtering using <code>reader</code>
+   * Collects at most {@link TopSuggestDocsCollector#getCountToCollect()} completions that
+   * match the provided {@link CompletionScorer}.
    * <p>
-   * {@link TopSuggestDocsCollector#collect(int, CharSequence, long)} is called
-   * for every matched completion
-   * <p>
-   * Completion collection can be early terminated by throwing {@link org.apache.lucene.search.CollectionTerminatedException}
+   * The {@link CompletionScorer#automaton} is intersected with the {@link #fst}.
+   * {@link CompletionScorer#weight} is used to compute boosts and/or extract context
+   * for each matched partial paths. A top N search is executed on {@link #fst} seeded with
+   * the matched partial paths. Upon reaching a completed path, {@link CompletionScorer#accept(int)}
+   * and {@link CompletionScorer#score(float, float)} is used on the document id, index weight
+   * and query boost to filter and score the entry, before being collected via
+   * {@link TopSuggestDocsCollector#collect(int, CharSequence, CharSequence, float)}
    */
-  public void lookup(final LeafReader reader, final Automaton automaton, final int num, final DocIdSet filter, final TopSuggestDocsCollector collector) {
-    final Bits filterDocs;
-    try {
-      if (filter != null) {
-        if (filter.iterator() == null) {
-          return;
+  public void lookup(final CompletionScorer scorer, final TopSuggestDocsCollector collector) throws IOException {
+    final double liveDocsRatio = calculateLiveDocRatio(scorer.reader.numDocs(), scorer.reader.maxDoc());
+    if (liveDocsRatio == -1) {
+      return;
+    }
+    final List<FSTUtil.Path<Pair<Long, BytesRef>>> prefixPaths = FSTUtil.intersectPrefixPaths(scorer.automaton, fst);
+    final int queueSize = getMaxTopNSearcherQueueSize(collector.getCountToCollect() * prefixPaths.size(),
+        scorer.reader.numDocs(), liveDocsRatio, scorer.filtered);
+    Comparator<Pair<Long, BytesRef>> comparator = getComparator();
+    Util.TopNSearcher<Pair<Long, BytesRef>> searcher = new Util.TopNSearcher<Pair<Long, BytesRef>>(fst,
+        collector.getCountToCollect(), queueSize, comparator, new ScoringPathComparator(scorer)) {
+
+      private final CharsRefBuilder spare = new CharsRefBuilder();
+
+      @Override
+      protected boolean acceptResult(Util.FSTPath<Pair<Long, BytesRef>> path) {
+        int payloadSepIndex = parseSurfaceForm(path.cost.output2, payloadSep, spare);
+        int docID = parseDocID(path.cost.output2, payloadSepIndex);
+        if (!scorer.accept(docID)) {
+          return false;
         }
-        if (filter.bits() == null) {
-          throw new IllegalArgumentException("DocIDSet does not provide random access interface");
-        } else {
-          filterDocs = filter.bits();
+        try {
+          float score = scorer.score(decode(path.cost.output1), path.boost);
+          collector.collect(docID, spare.toCharsRef(), path.context, score);
+          return true;
+        } catch (IOException e) {
+          throw new RuntimeException(e);
         }
-      } else {
-        filterDocs = null;
       }
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
+    };
 
-    int queueSize = getMaxTopNSearcherQueueSize(num, reader, filterDocs != null);
-    if (queueSize == -1) {
-      return;
+    for (FSTUtil.Path<Pair<Long, BytesRef>> path : prefixPaths) {
+      scorer.weight.setNextMatch(path.input.get());
+      searcher.addStartPaths(path.fstNode, path.output, false, path.input, scorer.weight.boost(),
+          scorer.weight.context());
     }
+    // hits are also returned by search()
+    // we do not use it, instead collect at acceptResult
+    searcher.search();
+    // search admissibility is not guaranteed
+    // see comment on getMaxTopNSearcherQueueSize
+    // assert  search.isComplete;
+  }
 
-    final Bits liveDocs = reader.getLiveDocs();
-    try {
-      final List<FSTUtil.Path<Pair<Long, BytesRef>>> prefixPaths = FSTUtil.intersectPrefixPaths(automaton, fst);
-      Util.TopNSearcher<Pair<Long, BytesRef>> searcher = new Util.TopNSearcher<Pair<Long, BytesRef>>(fst, num, queueSize, getComparator()) {
-
-        private final CharsRefBuilder spare = new CharsRefBuilder();
-
-        @Override
-        protected boolean acceptResult(IntsRef input, Pair<Long, BytesRef> output) {
-          int payloadSepIndex = parseSurfaceForm(output.output2, payloadSep, spare);
-          int docID = parseDocID(output.output2, payloadSepIndex);
-
-          // filter out deleted docs only if no filter is set
-          if (filterDocs == null && liveDocs != null && !liveDocs.get(docID)) {
-            return false;
-          }
-
-          // filter by filter context
-          if (filterDocs != null && !filterDocs.get(docID)) {
-            return false;
-          }
-
-          try {
-            collector.collect(docID, spare.toCharsRef(), decode(output.output1));
-            return true;
-          } catch (IOException e) {
-            throw new RuntimeException(e);
-          }
-        }
-      };
-
-      // TODO: add fuzzy support
-      for (FSTUtil.Path<Pair<Long, BytesRef>> path : prefixPaths) {
-        searcher.addStartPaths(path.fstNode, path.output, false, path.input);
-      }
+  /**
+   * Compares partial completion paths using {@link CompletionScorer#score(float, float)},
+   * breaks ties comparing path inputs
+   */
+  private static class ScoringPathComparator implements Comparator<Util.FSTPath<Pair<Long, BytesRef>>> {
+    private final CompletionScorer scorer;
 
-      try {
-        // hits are also returned by search()
-        // we do not use it, instead collect at acceptResult
-        Util.TopResults<Pair<Long, BytesRef>> search = searcher.search();
-        // search admissibility is not guaranteed
-        // see comment on getMaxTopNSearcherQueueSize
-        // assert  search.isComplete;
-      } catch (CollectionTerminatedException e) {
-        // terminate
-      }
+    public ScoringPathComparator(CompletionScorer scorer) {
+      this.scorer = scorer;
+    }
 
-    } catch (IOException bogus) {
-      throw new RuntimeException(bogus);
+    @Override
+    public int compare(Util.FSTPath<Pair<Long, BytesRef>> first, Util.FSTPath<Pair<Long, BytesRef>> second) {
+      int cmp = Float.compare(scorer.score(decode(second.cost.output1), second.boost),
+          scorer.score(decode(first.cost.output1), first.boost));
+      return (cmp != 0) ? cmp : first.input.get().compareTo(second.input.get());
     }
   }
 
+  private static Comparator<Pair<Long, BytesRef>> getComparator() {
+    return new Comparator<Pair<Long, BytesRef>>() {
+      @Override
+      public int compare(Pair<Long, BytesRef> o1, Pair<Long, BytesRef> o2) {
+        return Long.compare(o1.output1, o2.output1);
+      }
+    };
+  }
+
   /**
    * Simple heuristics to try to avoid over-pruning potential suggestions by the
    * TopNSearcher. Since suggestion entries can be rejected if they belong
@@ -241,17 +210,13 @@ final class NRTSuggester implements Acco
    * <p>
    * The maximum queue size is {@link #MAX_TOP_N_QUEUE_SIZE}
    */
-  private int getMaxTopNSearcherQueueSize(int num, LeafReader reader, boolean filterEnabled) {
-    double liveDocsRatio = calculateLiveDocRatio(reader.numDocs(), reader.maxDoc());
-    if (liveDocsRatio == -1) {
-      return -1;
-    }
-    long maxQueueSize = num * maxAnalyzedPathsPerOutput;
+  private int getMaxTopNSearcherQueueSize(int topN, int numDocs, double liveDocsRatio, boolean filterEnabled) {
+    long maxQueueSize = topN * maxAnalyzedPathsPerOutput;
     // liveDocRatio can be at most 1.0 (if no docs were deleted)
     assert liveDocsRatio <= 1.0d;
     maxQueueSize = (long) (maxQueueSize / liveDocsRatio);
     if (filterEnabled) {
-      maxQueueSize = maxQueueSize + (reader.numDocs()/2);
+      maxQueueSize = maxQueueSize + (numDocs/2);
     }
     return (int) Math.min(MAX_TOP_N_QUEUE_SIZE, maxQueueSize);
   }
@@ -269,21 +234,27 @@ final class NRTSuggester implements Acco
 
     /* read some meta info */
     int maxAnalyzedPathsPerOutput = input.readVInt();
+    /*
+     * Label used to denote the end of an input in the FST and
+     * the beginning of dedup bytes
+     */
     int endByte = input.readVInt();
     int payloadSep = input.readVInt();
 
-    return new NRTSuggester(fst, maxAnalyzedPathsPerOutput, payloadSep, endByte);
+    return new NRTSuggester(fst, maxAnalyzedPathsPerOutput, payloadSep);
   }
 
   static long encode(long input) {
-    if (input < 0) {
+    if (input < 0 || input > Integer.MAX_VALUE) {
       throw new UnsupportedOperationException("cannot encode value: " + input);
     }
-    return Long.MAX_VALUE - input;
+    return Integer.MAX_VALUE - input;
   }
 
   static long decode(long output) {
-    return (Long.MAX_VALUE - output);
+    assert output >= 0 && output <= Integer.MAX_VALUE :
+        "decoded output: " + output + " is not within 0 and Integer.MAX_VALUE";
+    return Integer.MAX_VALUE - output;
   }
 
   /**
@@ -307,7 +278,8 @@ final class NRTSuggester implements Acco
 
     static int parseDocID(final BytesRef output, int payloadSepIndex) {
       assert payloadSepIndex != -1 : "payload sep index can not be -1";
-      ByteArrayDataInput input = new ByteArrayDataInput(output.bytes, payloadSepIndex + output.offset + 1, output.length - (payloadSepIndex + output.offset));
+      ByteArrayDataInput input = new ByteArrayDataInput(output.bytes, payloadSepIndex + output.offset + 1,
+          output.length - (payloadSepIndex + output.offset));
       return input.readVInt();
     }
 

Modified: lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggesterBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggesterBuilder.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggesterBuilder.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggesterBuilder.java Thu Jun  4 19:24:05 2015
@@ -49,7 +49,7 @@ final class NRTSuggesterBuilder {
    * Marks end of the analyzed input and start of dedup
    * byte.
    */
-  private static final int END_BYTE = 0x0;
+  public static final int END_BYTE = 0x0;
 
   private final PairOutputs<Long, BytesRef> outputs;
   private final Builder<PairOutputs.Pair<Long, BytesRef>> builder;

Modified: lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java Thu Jun  4 19:24:05 2015
@@ -48,20 +48,14 @@ import org.apache.lucene.util.BytesRef;
  * document.add(new SuggestField(name, "suggestion", 4));
  * </pre>
  * To perform document suggestions based on the this field, use
- * {@link SuggestIndexSearcher#suggest(String, CharSequence, int, org.apache.lucene.search.Filter)}
- * <p>
- * Example query usage:
- * <pre class="prettyprint">
- * SuggestIndexSearcher indexSearcher = ..
- * indexSearcher.suggest(name, "su", 2)
- * </pre>
+ * {@link SuggestIndexSearcher#suggest(CompletionQuery, int)}
  *
  * @lucene.experimental
  */
 public class SuggestField extends Field {
 
-  private static final FieldType FIELD_TYPE = new FieldType();
-
+  /** Default field type for suggest field */
+  public static final FieldType FIELD_TYPE = new FieldType();
   static {
     FIELD_TYPE.setTokenized(true);
     FIELD_TYPE.setStored(false);
@@ -71,53 +65,86 @@ public class SuggestField extends Field
     FIELD_TYPE.freeze();
   }
 
+  static final byte TYPE = 0;
+
   private final BytesRef surfaceForm;
-  private final long weight;
+  private final int weight;
 
   /**
    * Creates a {@link SuggestField}
    *
-   * @param name   of the field
-   * @param value  to get suggestions on
-   * @param weight weight of the suggestion
+   * @param name   field name
+   * @param value  field value to get suggestions on
+   * @param weight field weight
+   *
+   * @throws IllegalArgumentException if either the name or value is null,
+   * if value is an empty string, if the weight is negative, if value contains
+   * any reserved characters
    */
-  public SuggestField(String name, String value, long weight) {
+  public SuggestField(String name, String value, int weight) {
     super(name, value, FIELD_TYPE);
-    if (weight < 0l) {
+    if (weight < 0) {
       throw new IllegalArgumentException("weight must be >= 0");
     }
+    if (value.length() == 0) {
+      throw new IllegalArgumentException("value must have a length > 0");
+    }
+    for (int i = 0; i < value.length(); i++) {
+      if (isReserved(value.charAt(i))) {
+        throw new IllegalArgumentException("Illegal input [" + value + "] UTF-16 codepoint [0x"
+            + Integer.toHexString((int) value.charAt(i))+ "] at position " + i + " is a reserved character");
+      }
+    }
     this.surfaceForm = new BytesRef(value);
     this.weight = weight;
   }
 
   @Override
   public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) throws IOException {
-    TokenStream stream = super.tokenStream(analyzer, reuse);
-    CompletionTokenStream completionStream;
+    CompletionTokenStream completionStream = wrapTokenStream(super.tokenStream(analyzer, reuse));
+    completionStream.setPayload(buildSuggestPayload());
+    return completionStream;
+  }
+
+  /**
+   * Wraps a <code>stream</code> with a CompletionTokenStream.
+   *
+   * Subclasses can override this method to change the indexing pipeline.
+   */
+  protected CompletionTokenStream wrapTokenStream(TokenStream stream) {
     if (stream instanceof CompletionTokenStream) {
-      completionStream = (CompletionTokenStream) stream;
+      return (CompletionTokenStream) stream;
     } else {
-      completionStream = new CompletionTokenStream(stream);
+      return new CompletionTokenStream(stream);
     }
-    BytesRef suggestPayload = buildSuggestPayload(surfaceForm, weight, (char) completionStream.sepLabel());
-    completionStream.setPayload(suggestPayload);
-    return completionStream;
   }
 
-  private BytesRef buildSuggestPayload(BytesRef surfaceForm, long weight, char sepLabel) throws IOException {
-    for (int i = 0; i < surfaceForm.length; i++) {
-      if (surfaceForm.bytes[i] == sepLabel) {
-        assert sepLabel == '\u001f';
-        throw new IllegalArgumentException(
-            "surface form cannot contain unit separator character U+001F; this character is reserved");
-      }
-    }
+  /**
+   * Returns a byte to denote the type of the field
+   */
+  protected byte type() {
+    return TYPE;
+  }
+
+  private BytesRef buildSuggestPayload() throws IOException {
     ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
     try (OutputStreamDataOutput output = new OutputStreamDataOutput(byteArrayOutputStream)) {
       output.writeVInt(surfaceForm.length);
       output.writeBytes(surfaceForm.bytes, surfaceForm.offset, surfaceForm.length);
-      output.writeVLong(weight + 1);
+      output.writeVInt(weight + 1);
+      output.writeByte(type());
     }
     return new BytesRef(byteArrayOutputStream.toByteArray());
   }
+
+  private boolean isReserved(char c) {
+    switch (c) {
+      case CompletionAnalyzer.SEP_LABEL:
+      case CompletionAnalyzer.HOLE_CHARACTER:
+      case NRTSuggesterBuilder.END_BYTE:
+        return true;
+      default:
+        return false;
+    }
+  }
 }

Modified: lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestIndexSearcher.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestIndexSearcher.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestIndexSearcher.java Thu Jun  4 19:24:05 2015
@@ -19,132 +19,66 @@ package org.apache.lucene.search.suggest
 
 import java.io.IOException;
 
-import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.search.DocIdSet;
-import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.BulkScorer;
+import org.apache.lucene.search.CollectionTerminatedException;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.util.automaton.Automaton;
-
-import static org.apache.lucene.search.suggest.document.CompletionFieldsProducer.CompletionTerms;
+import org.apache.lucene.search.Weight;
 
 /**
- * Adds document suggest capabilities to IndexSearcher
+ * Adds document suggest capabilities to IndexSearcher.
+ * Any {@link CompletionQuery} can be used to suggest documents.
+ *
+ * Use {@link PrefixCompletionQuery} for analyzed prefix queries,
+ * {@link RegexCompletionQuery} for regular expression prefix queries,
+ * {@link FuzzyCompletionQuery} for analyzed prefix with typo tolerance
+ * and {@link ContextQuery} to boost and/or filter suggestions by contexts
  *
  * @lucene.experimental
  */
 public class SuggestIndexSearcher extends IndexSearcher {
 
-  private final Analyzer queryAnalyzer;
-
   /**
    * Creates a searcher with document suggest capabilities
    * for <code>reader</code>.
-   * <p>
-   * Suggestion <code>key</code> is analyzed with <code>queryAnalyzer</code>
    */
-  public SuggestIndexSearcher(IndexReader reader, Analyzer queryAnalyzer) {
+  public SuggestIndexSearcher(IndexReader reader) {
     super(reader);
-    this.queryAnalyzer = queryAnalyzer;
-  }
-
-  /**
-   * Calls {@link #suggest(String, CharSequence, int, Filter)}
-   * with no document filter
-   */
-  public TopSuggestDocs suggest(String field, CharSequence key, int num) throws IOException {
-    return suggest(field, key, num, (Filter) null);
-  }
-
-  /**
-   * Calls {@link #suggest(String, CharSequence, int, Filter, TopSuggestDocsCollector)}
-   * with no document filter
-   */
-  public void suggest(String field, CharSequence key, int num, TopSuggestDocsCollector collector) throws IOException {
-    suggest(field, key, num, null, collector);
   }
 
   /**
-   * Suggests at most <code>num</code> documents filtered by <code>filter</code>
-   * that completes to <code>key</code> for a suggest <code>field</code>
-   * <p>
-   * Returns at most Top <code>num</code> document ids with corresponding completion and weight pair
-   *
-   * @throws java.lang.IllegalArgumentException if <code>filter</code> does not provide a random access
-   *                                            interface or if <code>field</code> is not a {@link SuggestField}
+   * Returns top <code>n</code> completion hits for
+   * <code>query</code>
    */
-  public TopSuggestDocs suggest(String field, CharSequence key, int num, Filter filter) throws IOException {
-    TopSuggestDocsCollector collector = new TopSuggestDocsCollector(num);
-    suggest(field, key, num, filter, collector);
+  public TopSuggestDocs suggest(CompletionQuery query, int n) throws IOException {
+    TopSuggestDocsCollector collector = new TopSuggestDocsCollector(n);
+    suggest(query, collector);
     return collector.get();
   }
 
   /**
-   * Suggests at most <code>num</code> documents filtered by <code>filter</code>
-   * that completes to <code>key</code> for a suggest <code>field</code>
-   * <p>
-   * Collect completions with {@link TopSuggestDocsCollector}
-   * The completions are collected in order of the suggest <code>field</code> weight.
-   * There can be more than one collection of the same document, if the <code>key</code>
-   * matches multiple <code>field</code> values of the same document
+   * Lower-level suggest API.
+   * Collects completion hits through <code>collector</code> for <code>query</code>.
    *
-   * @throws java.lang.IllegalArgumentException if <code>filter</code> does not provide a random access
-   *                                            interface or if <code>field</code> is not a {@link SuggestField}
+   * <p>{@link TopSuggestDocsCollector#collect(int, CharSequence, CharSequence, float)}
+   * is called for every matching completion hit.
    */
-  public void suggest(String field, CharSequence key, int num, Filter filter, TopSuggestDocsCollector collector) throws IOException {
-    // verify input
-    if (field == null) {
-      throw new IllegalArgumentException("'field' can not be null");
-    }
-    if (num <= 0) {
-      throw new IllegalArgumentException("'num' should be > 0");
-    }
-    if (collector == null) {
-      throw new IllegalArgumentException("'collector' can not be null");
-    }
-
-    // build query automaton
-    CompletionAnalyzer analyzer;
-    if (queryAnalyzer instanceof CompletionAnalyzer) {
-      analyzer = (CompletionAnalyzer) queryAnalyzer;
-    } else {
-      analyzer = new CompletionAnalyzer(queryAnalyzer);
-    }
-    final Automaton automaton = analyzer.toAutomaton(field, key);
-
-    // collect results
+  public void suggest(CompletionQuery query, TopSuggestDocsCollector collector) throws IOException {
+    // TODO use IndexSearcher.rewrite instead
+    // have to implement equals() and hashCode() in CompletionQuerys and co
+    query = (CompletionQuery) query.rewrite(getIndexReader());
+    Weight weight = query.createWeight(this, collector.needsScores());
     for (LeafReaderContext context : getIndexReader().leaves()) {
-      TopSuggestDocsCollector leafCollector = (TopSuggestDocsCollector) collector.getLeafCollector(context);
-      LeafReader reader = context.reader();
-      Terms terms = reader.terms(field);
-      if (terms == null) {
-        continue;
-      }
-      NRTSuggester suggester;
-      if (terms instanceof CompletionTerms) {
-        CompletionTerms completionTerms = (CompletionTerms) terms;
-        suggester = completionTerms.suggester();
-      } else {
-        throw new IllegalArgumentException(field + " is not a SuggestField");
-      }
-      if (suggester == null) {
-        // a segment can have a null suggester
-        // i.e. no FST was built
-        continue;
-      }
-
-      DocIdSet docIdSet = null;
-      if (filter != null) {
-        docIdSet = filter.getDocIdSet(context, reader.getLiveDocs());
-        if (docIdSet == null) {
-          // filter matches no docs in current leave
-          continue;
+      BulkScorer scorer = weight.bulkScorer(context, context.reader().getLiveDocs());
+      if (scorer != null) {
+        try {
+          scorer.score(collector.getLeafCollector(context));
+        } catch (CollectionTerminatedException e) {
+          // collection was terminated prematurely
+          // continue with the following leaf
         }
       }
-      suggester.lookup(reader, automaton, num, docIdSet, leafCollector);
     }
   }
 }

Modified: lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocs.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocs.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocs.java Thu Jun  4 19:24:05 2015
@@ -43,7 +43,12 @@ public class TopSuggestDocs extends TopD
     /**
      * Matched completion key
      */
-    public CharSequence key;
+    public final CharSequence key;
+
+    /**
+     * Context for the completion
+     */
+    public final CharSequence context;
 
     /**
      * Creates a SuggestScoreDoc instance
@@ -52,11 +57,10 @@ public class TopSuggestDocs extends TopD
      * @param key   matched completion
      * @param score weight of the matched completion
      */
-    public SuggestScoreDoc(int doc, CharSequence key, long score) {
-      // loss of precision but not magnitude
-      // implicit conversion from long -> float
+    public SuggestScoreDoc(int doc, CharSequence key, CharSequence context, float score) {
       super(doc, score);
       this.key = key;
+      this.context = context;
     }
 
     @Override

Modified: lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocsCollector.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocsCollector.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocsCollector.java Thu Jun  4 19:24:05 2015
@@ -30,20 +30,23 @@ import static org.apache.lucene.search.s
  * score, along with document id
  * <p>
  * Non scoring collector that collect completions in order of their
- * pre-defined weight.
+ * pre-computed scores.
  * <p>
  * NOTE: One document can be collected multiple times if a document
  * is matched for multiple unique completions for a given query
  * <p>
- * Subclasses should only override {@link TopSuggestDocsCollector#collect(int, CharSequence, long)},
- * {@link #setScorer(org.apache.lucene.search.Scorer)} is not
- * used
+ * Subclasses should only override
+ * {@link TopSuggestDocsCollector#collect(int, CharSequence, CharSequence, float)}.
+ * <p>
+ * NOTE: {@link #setScorer(org.apache.lucene.search.Scorer)} and
+ * {@link #collect(int)} is not used
  *
  * @lucene.experimental
  */
 public class TopSuggestDocsCollector extends SimpleCollector {
 
   private final SuggestScoreDocPriorityQueue priorityQueue;
+  private final int num;
 
   /**
    * Document base offset for the current Leaf
@@ -60,9 +63,17 @@ public class TopSuggestDocsCollector ext
     if (num <= 0) {
       throw new IllegalArgumentException("'num' must be > 0");
     }
+    this.num = num;
     this.priorityQueue = new SuggestScoreDocPriorityQueue(num);
   }
 
+  /**
+   * Returns the number of results to be collected
+   */
+  public int getCountToCollect() {
+    return num;
+  }
+
   @Override
   protected void doSetNextReader(LeafReaderContext context) throws IOException {
     docBase = context.docBase;
@@ -76,8 +87,8 @@ public class TopSuggestDocsCollector ext
    * NOTE: collection at the leaf level is guaranteed to be in
    * descending order of score
    */
-  public void collect(int docID, CharSequence key, long score) throws IOException {
-    SuggestScoreDoc current = new SuggestScoreDoc(docBase + docID, key, score);
+  public void collect(int docID, CharSequence key, CharSequence context, float score) throws IOException {
+    SuggestScoreDoc current = new SuggestScoreDoc(docBase + docID, key, context, score);
     if (current == priorityQueue.insertWithOverflow(current)) {
       // if the current SuggestScoreDoc has overflown from pq,
       // we can assume all of the successive collections from
@@ -104,7 +115,7 @@ public class TopSuggestDocsCollector ext
    */
   @Override
   public void collect(int doc) throws IOException {
-    // {@link #collect(int, CharSequence, long)} is used
+    // {@link #collect(int, CharSequence, CharSequence, long)} is used
     // instead
   }
 
@@ -113,6 +124,6 @@ public class TopSuggestDocsCollector ext
    */
   @Override
   public boolean needsScores() {
-    return false;
+    return true;
   }
 }

Modified: lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java Thu Jun  4 19:24:05 2015
@@ -320,7 +320,7 @@ public abstract class BaseCompoundFormat
     si.getCodec().compoundFormat().write(dir, si, IOContext.DEFAULT);
     Directory cfs = si.getCodec().compoundFormat().getCompoundReader(dir, si, IOContext.DEFAULT);
     try {
-      cfs.makeLock("foobar");
+      cfs.obtainLock("foobar");
       fail("didn't get expected exception");
     } catch (UnsupportedOperationException expected) {
       // expected UOE

Modified: lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/mockfile/MockFileSystemTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/mockfile/MockFileSystemTestCase.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/mockfile/MockFileSystemTestCase.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/mockfile/MockFileSystemTestCase.java Thu Jun  4 19:24:05 2015
@@ -25,6 +25,7 @@ import java.nio.file.DirectoryStream;
 import java.nio.file.Files;
 import java.nio.file.Path;
 
+import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems;
 
@@ -57,6 +58,7 @@ public abstract class MockFileSystemTest
   
   /** Test that URIs are not corrumpted */
   public void testURI() throws IOException {
+    assumeFalse("broken on J9: see https://issues.apache.org/jira/browse/LUCENE-6517", Constants.JAVA_VENDOR.startsWith("IBM"));
     Path dir = wrap(createTempDir());
 
     Path f1 = dir.resolve("file1");

Modified: lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/mockfile/WindowsFS.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/mockfile/WindowsFS.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/mockfile/WindowsFS.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/mockfile/WindowsFS.java Thu Jun  4 19:24:05 2015
@@ -17,15 +17,19 @@ package org.apache.lucene.mockfile;
  * limitations under the License.
  */
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.nio.file.CopyOption;
 import java.nio.file.FileSystem;
 import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
 import java.nio.file.Path;
 import java.nio.file.attribute.BasicFileAttributeView;
 import java.nio.file.attribute.BasicFileAttributes;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 
 /** 
  * FileSystem that (imperfectly) acts like windows. 
@@ -33,8 +37,7 @@ import java.util.Map;
  * Currently this filesystem only prevents deletion of open files.
  */
 public class WindowsFS extends HandleTrackingFS {
-  private final Map<Object,Integer> openFiles = new HashMap<>();
-  
+  final Map<Object,Integer> openFiles = new HashMap<>();
   // TODO: try to make this as realistic as possible... it depends e.g. how you
   // open files, if you map them, etc, if you can delete them (Uwe knows the rules)
   
@@ -60,8 +63,10 @@ public class WindowsFS extends HandleTra
 
   @Override
   protected void onOpen(Path path, Object stream) throws IOException {
-    Object key = getKey(path);
     synchronized (openFiles) {
+      final Object key = getKey(path);
+      // we have to read the key under the lock otherwise me might leak the openFile handle
+      // if we concurrently delete or move this file.
       Integer v = openFiles.get(key);
       if (v != null) {
         v = Integer.valueOf(v.intValue()+1);
@@ -74,9 +79,10 @@ public class WindowsFS extends HandleTra
 
   @Override
   protected void onClose(Path path, Object stream) throws IOException {
-    Object key = getKey(path);
+    Object key = getKey(path); // here we can read this outside of the lock
     synchronized (openFiles) {
       Integer v = openFiles.get(key);
+      assert v != null;
       if (v != null) {
         if (v.intValue() == 1) {
           openFiles.remove(key);
@@ -111,19 +117,25 @@ public class WindowsFS extends HandleTra
 
   @Override
   public void delete(Path path) throws IOException {
-    checkDeleteAccess(path);
-    super.delete(path);
+    synchronized (openFiles) {
+      checkDeleteAccess(path);
+      super.delete(path);
+    }
   }
 
   @Override
   public void move(Path source, Path target, CopyOption... options) throws IOException {
-    checkDeleteAccess(source);
-    super.move(source, target, options);
+    synchronized (openFiles) {
+      checkDeleteAccess(source);
+      super.move(source, target, options);
+    }
   }
 
   @Override
   public boolean deleteIfExists(Path path) throws IOException {
-    checkDeleteAccess(path);
-    return super.deleteIfExists(path);
+    synchronized (openFiles) {
+      checkDeleteAccess(path);
+      return super.deleteIfExists(path);
+    }
   }
 }

Modified: lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanQuery.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanQuery.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanQuery.java Thu Jun  4 19:24:05 2015
@@ -44,7 +44,7 @@ public class AssertingSpanQuery extends
   @Override
   public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores, SpanCollectorFactory factory) throws IOException {
     SpanWeight weight = in.createWeight(searcher, needsScores, factory);
-    return new AssertingSpanWeight(weight);
+    return new AssertingSpanWeight(searcher, weight);
   }
 
   @Override

Modified: lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanWeight.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanWeight.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanWeight.java Thu Jun  4 19:24:05 2015
@@ -20,6 +20,9 @@ package org.apache.lucene.search.spans;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermContext;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Scorer;
 import org.apache.lucene.util.Bits;
 
 import java.io.IOException;
@@ -38,8 +41,8 @@ public class AssertingSpanWeight extends
    * @param in the SpanWeight to wrap
    * @throws IOException on error
    */
-  public AssertingSpanWeight(SpanWeight in) throws IOException {
-    super((SpanQuery) in.getQuery(), in.similarity, in.collectorFactory);
+  public AssertingSpanWeight(IndexSearcher searcher, SpanWeight in) throws IOException {
+    super((SpanQuery) in.getQuery(), searcher, null, in.collectorFactory);
     this.in = in;
   }
 
@@ -60,4 +63,24 @@ public class AssertingSpanWeight extends
   public void extractTerms(Set<Term> terms) {
     in.extractTerms(terms);
   }
+
+  @Override
+  public float getValueForNormalization() throws IOException {
+    return in.getValueForNormalization();
+  }
+
+  @Override
+  public void normalize(float queryNorm, float topLevelBoost) {
+    in.normalize(queryNorm, topLevelBoost);
+  }
+
+  @Override
+  public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+    return in.scorer(context, acceptDocs);
+  }
+
+  @Override
+  public Explanation explain(LeafReaderContext context, int doc) throws IOException {
+    return in.explain(context, doc);
+  }
 }

Modified: lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java Thu Jun  4 19:24:05 2015
@@ -34,6 +34,8 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lucene.index.DirectoryReader;
@@ -74,7 +76,6 @@ public class MockDirectoryWrapper extend
   boolean assertNoDeleteOpenFile = false;
   boolean preventDoubleWrite = true;
   boolean trackDiskUsage = false;
-  boolean wrapLocking = true;
   boolean useSlowOpenClosers = LuceneTestCase.TEST_NIGHTLY;
   boolean enableVirusScanner = true;
   boolean allowRandomFileNotFoundException = true;
@@ -82,7 +83,7 @@ public class MockDirectoryWrapper extend
   private Set<String> unSyncedFiles;
   private Set<String> createdFiles;
   private Set<String> openFilesForWrite = new HashSet<>();
-  Map<String,Exception> openLocks = Collections.synchronizedMap(new HashMap<String,Exception>());
+  ConcurrentMap<String,RuntimeException> openLocks = new ConcurrentHashMap<>();
   volatile boolean crashed;
   private ThrottledIndexOutput throttledOutput;
   private Throttling throttling = LuceneTestCase.TEST_NIGHTLY ? Throttling.SOMETIMES : Throttling.NEVER;
@@ -699,19 +700,6 @@ public class MockDirectoryWrapper extend
   public void setAssertNoUnrefencedFilesOnClose(boolean v) {
     assertNoUnreferencedFilesOnClose = v;
   }
-  
-  /**
-   * Set to false if you want to return the pure {@link LockFactory} and not
-   * wrap all lock with {@code AssertingLock}.
-   * <p>
-   * Be careful if you turn this off: {@code MockDirectoryWrapper} might
-   * no longer be able to detect if you forget to close an {@link IndexWriter},
-   * and spit out horribly scary confusing exceptions instead of
-   * simply telling you that.
-   */
-  public void setAssertLocks(boolean v) {
-    this.wrapLocking = v;
-  }
 
   @Override
   public synchronized void close() throws IOException {
@@ -748,7 +736,7 @@ public class MockDirectoryWrapper extend
       }
       if (openLocks.size() > 0) {
         Exception cause = null;
-        Iterator<Exception> stacktraces = openLocks.values().iterator();
+        Iterator<RuntimeException> stacktraces = openLocks.values().iterator();
         if (stacktraces.hasNext()) {
           cause = stacktraces.next();
         }
@@ -992,47 +980,12 @@ public class MockDirectoryWrapper extend
   }
 
   @Override
-  public synchronized Lock makeLock(String name) {
+  public synchronized Lock obtainLock(String name) throws IOException {
     maybeYield();
-    if (wrapLocking) {
-      return new AssertingLock(super.makeLock(name), name);
-    } else {
-      return super.makeLock(name);
-    }
+    return super.obtainLock(name);
+    // TODO: consider mocking locks, but not all the time, can hide bugs
   }
   
-  private final class AssertingLock extends Lock {
-    private final Lock delegateLock;
-    private final String name;
-    
-    AssertingLock(Lock delegate, String name) {
-      this.delegateLock = delegate;
-      this.name = name;
-    }
-
-    @Override
-    public boolean obtain() throws IOException {
-      if (delegateLock.obtain()) {
-        assert delegateLock == NoLockFactory.SINGLETON_LOCK || !openLocks.containsKey(name);
-        openLocks.put(name, new RuntimeException("lock \"" + name + "\" was not released"));
-        return true;
-      } else {
-        return false;
-      }
-    }
-
-    @Override
-    public void close() throws IOException {
-      delegateLock.close();
-      openLocks.remove(name);
-    }
-
-    @Override
-    public boolean isLocked() throws IOException {
-      return delegateLock.isLocked();
-    }
-  }  
-  
   /** Use this when throwing fake {@code IOException},
    *  e.g. from {@link MockDirectoryWrapper.Failure}. */
   public static class FakeIOException extends IOException {

Modified: lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Thu Jun  4 19:24:05 2015
@@ -489,7 +489,7 @@ public abstract class LuceneTestCase ext
   /**
    * Suite failure marker (any error in the test or suite scope).
    */
-  private static TestRuleMarkFailure suiteFailureMarker;
+  protected static TestRuleMarkFailure suiteFailureMarker;
   
   /**
    * Temporary files cleanup rule.

Modified: lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/util/QuickPatchThreadsFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/util/QuickPatchThreadsFilter.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/util/QuickPatchThreadsFilter.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/util/QuickPatchThreadsFilter.java Thu Jun  4 19:24:05 2015
@@ -26,15 +26,21 @@ public class QuickPatchThreadsFilter imp
   static final boolean isJ9;
   
   static {
-    isJ9 = System.getProperty("java.vm.info", "<?>").contains("IBM J9");
+    isJ9 = Constants.JAVA_VENDOR.startsWith("IBM");
   }
 
   @Override
   public boolean reject(Thread t) {
     if (isJ9) {
+      // LUCENE-6518
+      if ("ClassCache Reaper".equals(t.getName())) {
+        return true;
+      }
+
+      // LUCENE-4736
       StackTraceElement [] stack = t.getStackTrace();
       if (stack.length > 0 && stack[stack.length - 1].getClassName().equals("java.util.Timer$TimerImpl")) {
-        return true; // LUCENE-4736
+        return true;
       }
     }
     return false;

Modified: lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java Thu Jun  4 19:24:05 2015
@@ -271,7 +271,7 @@ public final class TestUtil {
     ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
     // TODO: actually use the dir's locking, unless test uses a special method?
     // some tests e.g. exception tests become much more complicated if they have to close the writer
-    try (CheckIndex checker = new CheckIndex(dir, NoLockFactory.INSTANCE.makeLock(dir, "bogus"))) {
+    try (CheckIndex checker = new CheckIndex(dir, NoLockFactory.INSTANCE.obtainLock(dir, "bogus"))) {
       checker.setCrossCheckTermVectors(crossCheckTermVectors);
       checker.setFailFast(failFast);
       checker.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8), false);

Modified: lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestWindowsFS.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestWindowsFS.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestWindowsFS.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestWindowsFS.java Thu Jun  4 19:24:05 2015
@@ -17,15 +17,25 @@ package org.apache.lucene.mockfile;
  * limitations under the License.
  */
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.lang.Exception;
+import java.lang.InterruptedException;
+import java.lang.NoSuchFieldException;
+import java.lang.RuntimeException;
 import java.net.URI;
 import java.nio.file.FileSystem;
 import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
 import java.nio.file.Path;
 import java.nio.file.StandardCopyOption;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.lucene.mockfile.FilterPath;
+import org.apache.lucene.mockfile.WindowsFS;
 import org.apache.lucene.util.Constants;
 
 /** Basic tests for WindowsFS */
@@ -95,4 +105,57 @@ public class TestWindowsFS extends MockF
     }
     is.close();
   }
+
+  public void testOpenDeleteConcurrently() throws IOException, Exception {
+    final Path dir = wrap(createTempDir());
+    final Path file = dir.resolve("thefile");
+    final CyclicBarrier barrier = new CyclicBarrier(2);
+    final AtomicBoolean stopped = new AtomicBoolean(false);
+    Thread t = new Thread() {
+      @Override
+      public void run() {
+        try {
+          barrier.await();
+        } catch (Exception ex) {
+          throw new RuntimeException(ex);
+        }
+        while (stopped.get() == false) {
+          try {
+            if (random().nextBoolean()) {
+              Files.delete(file);
+            } else if (random().nextBoolean()) {
+              Files.deleteIfExists(file);
+            } else {
+              Path target = file.resolveSibling("other");
+              Files.move(file, target);
+              Files.delete(target);
+            }
+          } catch (IOException ex) {
+            // continue
+          }
+        }
+      }
+    };
+    t.start();
+    barrier.await();
+    try {
+      final int iters = 10 + random().nextInt(100);
+      for (int i = 0; i < iters; i++) {
+        boolean opened = false;
+        try (OutputStream stream = Files.newOutputStream(file)) {
+          opened = true;
+          stream.write(0);
+          // just create
+        } catch (FileNotFoundException | NoSuchFileException ex) {
+          assertEquals("File handle leaked - file is closed but still registered", 0, ((WindowsFS) dir.getFileSystem().provider()).openFiles.size());
+          assertFalse("caught FNF on close", opened);
+        }
+        assertEquals("File handle leaked - file is closed but still registered", 0, ((WindowsFS) dir.getFileSystem().provider()).openFiles.size());
+        Files.deleteIfExists(file);
+      }
+    } finally {
+      stopped.set(true);
+      t.join();
+    }
+  }
 }

Modified: lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/test/org/apache/lucene/store/TestMockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/test/org/apache/lucene/store/TestMockDirectoryWrapper.java?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/test/org/apache/lucene/store/TestMockDirectoryWrapper.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/test-framework/src/test/org/apache/lucene/store/TestMockDirectoryWrapper.java Thu Jun  4 19:24:05 2015
@@ -47,32 +47,6 @@ public class TestMockDirectoryWrapper ex
     super.testThreadSafety();
   }
   
-  public void testFailIfIndexWriterNotClosed() throws IOException {
-    MockDirectoryWrapper dir = newMockDirectory();
-    IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(null));
-    try {
-      dir.close();
-      fail();
-    } catch (Exception expected) {
-      assertTrue(expected.getMessage().contains("there are still open locks"));
-    } finally {
-      IOUtils.closeWhileHandlingException(iw);
-    }
-  }
-  
-  public void testFailIfIndexWriterNotClosedChangeLockFactory() throws IOException {
-    MockDirectoryWrapper dir = newMockDirectory(random(), new SingleInstanceLockFactory());
-    IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(null));
-    try {
-      dir.close();
-      fail();
-    } catch (Exception expected) {
-      assertTrue(expected.getMessage().contains("there are still open locks"));
-    } finally {
-      IOUtils.closeWhileHandlingException(iw);
-    }
-  }
-  
   public void testDiskFull() throws IOException {
     // test writeBytes
     MockDirectoryWrapper dir = newMockDirectory();

Modified: lucene/dev/branches/LUCENE-6481/lucene/tools/junit4/tests.policy
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/tools/junit4/tests.policy?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/tools/junit4/tests.policy (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/tools/junit4/tests.policy Thu Jun  4 19:24:05 2015
@@ -73,6 +73,8 @@ grant {
   permission java.lang.RuntimePermission "getClassLoader";
   // needed to test unmap hack on platforms that support it
   permission java.lang.RuntimePermission "accessClassInPackage.sun.misc";
+  // needed by cyberneko usage by benchmarks on J9
+  permission java.lang.RuntimePermission "accessClassInPackage.org.apache.xerces.util";
   // needed by jacoco to dump coverage
   permission java.lang.RuntimePermission "shutdownHooks";
   

Modified: lucene/dev/branches/LUCENE-6481/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/solr/CHANGES.txt?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/solr/CHANGES.txt (original)
+++ lucene/dev/branches/LUCENE-6481/solr/CHANGES.txt Thu Jun  4 19:24:05 2015
@@ -72,6 +72,16 @@ Upgrading from Solr 5.2
 * SolrJ's CollectionAdminRequest class is now marked as abstract. Use one of its concrete
   sub-classes instead.
 
+* Solr no longer supports forcefully unlocking an index.
+  This is no longer supported by the underlying Lucene locking
+  framework. The setting in solrconfig.xml has no effect anymore.
+  Background: If you use native lock factory, unlocking should
+  not be needed, because the locks are cleared after process
+  shutdown automatically by the operating system. If you are
+  using simple lock factory (not recommended) or hdfs lock
+  factory, you may need to manually unlock by deleting the lock
+  file from filesystem / HDFS.
+
 Detailed Change List
 ----------------------
 
@@ -84,9 +94,26 @@ New Features
 * SOLR-7389: Expose znodeVersion property for each of the collections returned for the clusterstatus
   operation in the collections API (Marius Grama via shalin)
 
+* SOLR-7622: A DocTransformer can now request fields from the SolrIndexSearcher that are not
+  necessarily returned in the file SolrDocument by returning a list of fields from 
+  DocTransformer#getExtraRequestFields  (ryan)
+
 Bug Fixes
 ----------------------
-(no changes)
+
+* SOLR-7361: Slow loading SolrCores should not hold up all other SolrCores that have finished loading from serving
+  requests. (Mark Miller, Timothy Potter, Ramkumar Aiyengar)
+
+* SOLR-7616: Faceting on a numeric field with a unique() subfacet function on another numeric field
+  can result in incorrect results or an exception. (yonik)
+
+* SOLR-7518: New Facet Module should respect shards.tolerant and process all non-failing shards
+  instead of throwing an exception. (yonik)
+
+* SOLR-4506: Clean-up old (unused) index directories in the background after initializing a new index;
+  previously, Solr would leave old index.yyyyMMddHHmmssSSS directories left behind after failed recoveries
+  in the data directory, which unnecessarily consumes disk space. (Mark Miller, Timothy Potter)
+
 
 Optimizations
 ----------------------
@@ -97,6 +124,23 @@ Other Changes
 
 * SOLR-7595: Allow method chaining for all CollectionAdminRequests in Solrj. (shalin)
 
+* SOLR-7146: MiniSolrCloudCluster based tests can fail with ZooKeeperException NoNode for /live_nodes.
+  (Vamsee Yarlagadda via shalin)
+
+* SOLR-7590: Finish and improve MDC context logging support. (Mark Miller)
+
+* SOLR-7599: Remove cruft from SolrCloud tests. (shalin)
+
+* SOLR-7623: Fix regression from SOLR-7484 that made it impossible to override 
+  SolrDispatchFilter#execute() and SolrDispatchFilter#sendError().  You can now override these
+  functions in HttpSolrCall.  (ryan)
+
+* SOLR-7636: CLUSTERSTATUS API is executed at CollectionsHandler (noble)
+
+* LUCENE-6508: Remove ability to forcefully unlock an index.
+  This is no longer supported by the underlying Lucene locking
+  framework.  (Uwe Schindler, Mike McCandless, Robert Muir)
+
 ==================  5.2.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release
@@ -259,7 +303,7 @@ New Features
   which blends RPT indexes for speed with serialized geometry for accuracy.  Includes a Lucene segment based
   in-memory shape cache. (David Smiley)
 
-* SOLR-7465: New file indexing example, under example/files.  (Esther Quansah, Erik Hatcher)
+* SOLR-7465, SOLR-7610: New file indexing example, under example/files.  (Esther Quansah, Erik Hatcher)
 
 * SOLR-7468: Kerberos authenticaion plugin for Solr. This would allow running a Kerberized Solr.
   (Noble Paul, Ishan Chattopadhyaya, Gregory Chanan, Anshum Gupta)
@@ -360,6 +404,21 @@ Bug Fixes
 * SOLR-7585: Fix NoSuchElementException in LFUCache resulting from heavy writes
   making concurrent put() calls. (Maciej Zasada via Shawn Heisey)
 
+* SOLR-7587: Seeding bucket versions from index when the firstSearcher event fires has a race condition
+  that leads to an infinite wait on VersionInfo's ReentrantReadWriteLock because the read-lock acquired
+  during a commit cannot be upgraded to a write-lock needed to block updates; solution is to move the
+  call out of the firstSearcher event path and into the SolrCore constructor. (Timothy Potter)
+
+* SOLR-7625: Ensure that the max value for seeding version buckets is updated after recovery even if
+  the UpdateLog is not replayed. (Timothy Potter)
+
+* SOLR-7610: Fix VelocityResponseWriter's $resource.locale to accurately report locale in use.
+  (ehatcher)
+
+* SOLR-7614: Distributed pivot facet refinement was broken due to a single correlation counter
+  used across multiple requests as if it was private to each request. (yonik)
+
+
 Optimizations
 ----------------------
 
@@ -491,6 +550,9 @@ Other Changes
   enable auto soft-commits for the bin/solr -e cloud example using the Config API.
   (Timothy Potter)
 
+* SOLR-7183: Fix Locale blacklisting for Minikdc based tests. (Ishan Chattopadhyaya, hossman
+  via Anshum Gupta)
+
 ==================  5.1.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release

Modified: lucene/dev/branches/LUCENE-6481/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/solrconfig.xml?rev=1683615&r1=1683614&r2=1683615&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/solrconfig.xml (original)
+++ lucene/dev/branches/LUCENE-6481/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/solrconfig.xml Thu Jun  4 19:24:05 2015
@@ -220,19 +220,6 @@
     -->
     <!-- <lockType>native</lockType> -->
 
-    <!-- Unlock On Startup
-
-         If true, unlock any held write or commit locks on startup.
-         This defeats the locking mechanism that allows multiple
-         processes to safely access a lucene index, and should be used
-         with care. Default is "false".
-
-         This is not needed if lock type is 'none' or 'single'
-     -->
-    <!--
-    <unlockOnStartup>false</unlockOnStartup>
-      -->
-    
     <!-- If true, IndexReaders will be reopened (often more efficient)
          instead of closed and then opened. Default: true
       -->