You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ji...@apache.org on 2019/06/26 10:32:59 UTC

[lucene-solr] branch branch_8x updated: LUCENE-8859: The completion suggester's postings format now have an option to load its internal FST off-heap.

This is an automated email from the ASF dual-hosted git repository.

jimczi pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new d5a1a2b  LUCENE-8859: The completion suggester's postings format now have an option to load its internal FST off-heap.
d5a1a2b is described below

commit d5a1a2b3e13b578b13b089b7de03b88bab676fd6
Author: jimczi <ji...@elastic.co>
AuthorDate: Wed Jun 26 11:16:44 2019 +0200

    LUCENE-8859: The completion suggester's postings format now have an option to load its internal FST off-heap.
---
 lucene/CHANGES.txt                                 |  5 ++-
 .../java/org/apache/lucene/util/fst/FSTStore.java  |  1 +
 .../apache/lucene/util/fst/OffHeapFSTStore.java    |  5 +++
 .../org/apache/lucene/util/fst/OnHeapFSTStore.java | 12 +++++---
 .../document/Completion50PostingsFormat.java       | 15 +++++++--
 .../suggest/document/CompletionFieldsProducer.java |  5 +--
 .../suggest/document/CompletionPostingsFormat.java | 34 +++++++++++++++++++-
 .../suggest/document/CompletionsTermsReader.java   |  8 +++--
 .../search/suggest/document/NRTSuggester.java      | 36 +++++++++++++++++++---
 .../suggest/document/NRTSuggesterBuilder.java      |  5 +--
 .../search/suggest/document/TestSuggestField.java  |  5 ++-
 11 files changed, 109 insertions(+), 22 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index a3100c1..4ba8e91 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -45,6 +45,9 @@ New Features
 * LUCENE-7714: Add a numeric range query in sandbox that takes advantage of index sorting.
   (Julie Tibshirani via Jim Ferenczi)
 
+* LUCENE-8859: The completion suggester's postings format now have an option to
+  load its internal FST off-heap. (Jim Ferenczi)
+
 Bug Fixes
 
 * LUCENE-8831: Fixed LatLonShapeBoundingBoxQuery .hashCode methods. (Ignacio Vera)
@@ -70,7 +73,7 @@ Bug Fixes
   underlying filesystem directory. (Simon Willnauer)
 
 * LUCENE-8853: FileSwitchDirectory now applies best effort to place tmp files in the same
-  direcotry as the target files. (Simon Willnauer)
+  directory as the target files. (Simon Willnauer)
 
 Improvements
 
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/FSTStore.java b/lucene/core/src/java/org/apache/lucene/util/fst/FSTStore.java
index 0cd1340..3e37a0c 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/FSTStore.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/FSTStore.java
@@ -25,6 +25,7 @@ import java.io.IOException;
 /** Abstraction for reading/writing bytes necessary for FST. */
 public interface FSTStore extends Accountable {
     void init(DataInput in, long numBytes) throws IOException;
+    long size();
     FST.BytesReader getReverseBytesReader();
     void writeTo(DataOutput out) throws IOException;
 }
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/OffHeapFSTStore.java b/lucene/core/src/java/org/apache/lucene/util/fst/OffHeapFSTStore.java
index 88c8884..4f8a76c 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/OffHeapFSTStore.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/OffHeapFSTStore.java
@@ -54,6 +54,11 @@ public final class OffHeapFSTStore implements FSTStore {
     }
 
     @Override
+    public long size() {
+        return numBytes;
+    }
+
+    @Override
     public FST.BytesReader getReverseBytesReader() {
         try {
             return new ReverseRandomAccessReader(in.randomAccessSlice(offset, numBytes));
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/OnHeapFSTStore.java b/lucene/core/src/java/org/apache/lucene/util/fst/OnHeapFSTStore.java
index 7dc09b9..0cd81b7 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/OnHeapFSTStore.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/OnHeapFSTStore.java
@@ -62,15 +62,17 @@ public final class OnHeapFSTStore implements FSTStore {
     }
 
     @Override
-    public long ramBytesUsed() {
-        long size = BASE_RAM_BYTES_USED;
+    public long size() {
         if (bytesArray != null) {
-            size += bytesArray.length;
+            return bytesArray.length;
         } else {
-            size += bytes.ramBytesUsed();
+            return bytes.ramBytesUsed();
         }
+    }
 
-        return size;
+    @Override
+    public long ramBytesUsed() {
+        return BASE_RAM_BYTES_USED + size();
     }
 
     @Override
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion50PostingsFormat.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion50PostingsFormat.java
index f859262..fcc0584 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion50PostingsFormat.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion50PostingsFormat.java
@@ -25,12 +25,21 @@ import org.apache.lucene.codecs.PostingsFormat;
  * @lucene.experimental
  */
 public class Completion50PostingsFormat extends CompletionPostingsFormat {
-
   /**
-   * Sole Constructor
+   * Creates a {@link Completion50PostingsFormat} that will load
+   * the completion FST on-heap.
    */
   public Completion50PostingsFormat() {
-    super();
+    this(FSTLoadMode.ON_HEAP);
+  }
+
+  /**
+   * Creates a {@link Completion50PostingsFormat} that will
+   * use the provided <code>fstLoadMode</code> to determine
+   * if the completion FST should be loaded on or off heap.
+   */
+  public Completion50PostingsFormat(FSTLoadMode fstLoadMode) {
+    super(fstLoadMode);
   }
 
   @Override
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java
index b998f8e..b21b0b5 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java
@@ -32,6 +32,7 @@ import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.Terms;
+import org.apache.lucene.search.suggest.document.CompletionPostingsFormat.FSTLoadMode;
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Accountable;
@@ -71,7 +72,7 @@ final class CompletionFieldsProducer extends FieldsProducer {
     this.readers = readers;
   }
 
-  CompletionFieldsProducer(SegmentReadState state) throws IOException {
+  CompletionFieldsProducer(SegmentReadState state, FSTLoadMode fstLoadMode) throws IOException {
     String indexFile = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, INDEX_EXTENSION);
     delegateFieldsProducer = null;
     boolean success = false;
@@ -101,7 +102,7 @@ final class CompletionFieldsProducer extends FieldsProducer {
         byte type = index.readByte();
         FieldInfo fieldInfo = state.fieldInfos.fieldInfo(fieldNumber);
         // we don't load the FST yet
-        readers.put(fieldInfo.name, new CompletionsTermsReader(dictIn, offset, minWeight, maxWeight, type));
+        readers.put(fieldInfo.name, new CompletionsTermsReader(dictIn, offset, minWeight, maxWeight, type, fstLoadMode));
       }
       CodecUtil.checkFooter(index);
       success = true;
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java
index e28106f..2598ecf 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java
@@ -97,10 +97,42 @@ public abstract class CompletionPostingsFormat extends PostingsFormat {
   static final String DICT_EXTENSION = "lkp";
 
   /**
+   * An enum that allows to control if suggester FSTs are loaded into memory or read off-heap
+   */
+  public enum FSTLoadMode {
+    /**
+     * Always read FSTs from disk.
+     * NOTE: If this option is used the FST will be read off-heap even if buffered directory implementations
+     * are used.
+     */
+    OFF_HEAP,
+    /**
+     * Never read FSTs from disk ie. all suggest fields FSTs are loaded into memory
+     */
+    ON_HEAP,
+    /**
+     * Automatically make the decision if FSTs are read from disk depending if the segment read from an MMAPDirectory
+     */
+    AUTO
+  }
+
+  private final FSTLoadMode fstLoadMode;
+
+  /**
    * Used only by core Lucene at read-time via Service Provider instantiation
    */
   public CompletionPostingsFormat() {
+    this(FSTLoadMode.ON_HEAP);
+  }
+
+  /**
+   * Creates a {@link CompletionPostingsFormat} that will
+   * use the provided <code>fstLoadMode</code> to determine
+   * if the completion FST should be loaded on or off heap.
+   */
+  public CompletionPostingsFormat(FSTLoadMode fstLoadMode) {
     super(CODEC_NAME);
+    this.fstLoadMode = fstLoadMode;
   }
 
   /**
@@ -120,6 +152,6 @@ public abstract class CompletionPostingsFormat extends PostingsFormat {
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    return new CompletionFieldsProducer(state);
+    return new CompletionFieldsProducer(state, fstLoadMode);
   }
 }
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionsTermsReader.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionsTermsReader.java
index 5eeeb69..1776ce6 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionsTermsReader.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionsTermsReader.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
 
+import org.apache.lucene.search.suggest.document.CompletionPostingsFormat.FSTLoadMode;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Accountable;
 
@@ -39,13 +40,15 @@ public final class CompletionsTermsReader implements Accountable {
   private final IndexInput dictIn;
   private final long offset;
 
+  private final FSTLoadMode fstLoadMode;
+
   private NRTSuggester suggester;
 
   /**
    * Creates a CompletionTermsReader to load a field-specific suggester
    * from the index <code>dictIn</code> with <code>offset</code>
    */
-  CompletionsTermsReader(IndexInput dictIn, long offset, long minWeight, long maxWeight, byte type) throws IOException {
+  CompletionsTermsReader(IndexInput dictIn, long offset, long minWeight, long maxWeight, byte type, FSTLoadMode fstLoadMode) {
     assert minWeight <= maxWeight;
     assert offset >= 0l && offset < dictIn.length();
     this.dictIn = dictIn;
@@ -53,6 +56,7 @@ public final class CompletionsTermsReader implements Accountable {
     this.minWeight = minWeight;
     this.maxWeight = maxWeight;
     this.type = type;
+    this.fstLoadMode = fstLoadMode;
   }
 
   /**
@@ -63,7 +67,7 @@ public final class CompletionsTermsReader implements Accountable {
     if (suggester == null) {
       try (IndexInput dictClone = dictIn.clone()) { // let multiple fields load concurrently
         dictClone.seek(offset);
-        suggester = NRTSuggester.load(dictClone);
+        suggester = NRTSuggester.load(dictClone, fstLoadMode);
       }
     }
     return suggester;
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggester.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggester.java
index 7b8981a..6a9e494 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggester.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggester.java
@@ -23,8 +23,10 @@ import java.util.Comparator;
 import java.util.List;
 
 import org.apache.lucene.search.suggest.analyzing.FSTUtil;
+import org.apache.lucene.search.suggest.document.CompletionPostingsFormat.FSTLoadMode;
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.ByteArrayDataOutput;
+import org.apache.lucene.store.ByteBufferIndexInput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Bits;
@@ -32,6 +34,7 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.fst.ByteSequenceOutputs;
 import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.OffHeapFSTStore;
 import org.apache.lucene.util.fst.PairOutputs.Pair;
 import org.apache.lucene.util.fst.PairOutputs;
 import org.apache.lucene.util.fst.PositiveIntOutputs;
@@ -298,12 +301,36 @@ public final class NRTSuggester implements Accountable {
     return (numDocs > 0) ? ((double) numDocs / maxDocs) : -1;
   }
 
+  private static boolean shouldLoadFSTOffHeap(IndexInput input, FSTLoadMode fstLoadMode) {
+    switch (fstLoadMode) {
+      case ON_HEAP:
+        return false;
+      case OFF_HEAP:
+        return true;
+      case AUTO:
+        return input instanceof ByteBufferIndexInput;
+      default:
+        throw new IllegalStateException("unknown enum constant: " + fstLoadMode);
+    }
+  }
+
   /**
-   * Loads a {@link NRTSuggester} from {@link org.apache.lucene.store.IndexInput}
+   * Loads a {@link NRTSuggester} from {@link org.apache.lucene.store.IndexInput} on or off-heap
+   * depending on the provided <code>fstLoadMode</code>
    */
-  public static NRTSuggester load(IndexInput input) throws IOException {
-    final FST<Pair<Long, BytesRef>> fst = new FST<>(input, new PairOutputs<>(
-        PositiveIntOutputs.getSingleton(), ByteSequenceOutputs.getSingleton()));
+  public static NRTSuggester load(IndexInput input, FSTLoadMode fstLoadMode) throws IOException {
+    final FST<Pair<Long, BytesRef>> fst;
+    if (shouldLoadFSTOffHeap(input, fstLoadMode)) {
+      OffHeapFSTStore store = new OffHeapFSTStore();
+      IndexInput clone = input.clone();
+      clone.seek(input.getFilePointer());
+      fst = new FST<>(clone, new PairOutputs<>(
+          PositiveIntOutputs.getSingleton(), ByteSequenceOutputs.getSingleton()), store);
+      input.seek(clone.getFilePointer() + store.size());
+    } else {
+      fst = new FST<>(input, new PairOutputs<>(
+          PositiveIntOutputs.getSingleton(), ByteSequenceOutputs.getSingleton()));
+    }
 
     /* read some meta info */
     int maxAnalyzedPathsPerOutput = input.readVInt();
@@ -313,7 +340,6 @@ public final class NRTSuggester implements Accountable {
      */
     int endByte = input.readVInt();
     int payloadSep = input.readVInt();
-
     return new NRTSuggester(fst, maxAnalyzedPathsPerOutput, payloadSep);
   }
 
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggesterBuilder.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggesterBuilder.java
index 5ca4993..5133855 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggesterBuilder.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggesterBuilder.java
@@ -21,6 +21,7 @@ import java.util.PriorityQueue;
 
 import org.apache.lucene.analysis.miscellaneous.ConcatenateGraphFilter;
 import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IntsRefBuilder;
@@ -115,7 +116,7 @@ final class NRTSuggesterBuilder {
 
   /**
    * Builds and stores a FST that can be loaded with
-   * {@link NRTSuggester#load(org.apache.lucene.store.IndexInput)}
+   * {@link NRTSuggester#load(IndexInput, CompletionPostingsFormat.FSTLoadMode)})}
    */
   public boolean store(DataOutput output) throws IOException {
     final FST<PairOutputs.Pair<Long, BytesRef>> build = builder.finish();
@@ -124,7 +125,7 @@ final class NRTSuggesterBuilder {
     }
     build.save(output);
 
-    /* write some more meta-info */
+    /* write some more  meta-info */
     assert maxAnalyzedPathsPerOutput > 0;
     output.writeVInt(maxAnalyzedPathsPerOutput);
     output.writeVInt(END_BYTE);
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
index d5503a5..68dc014 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
@@ -29,6 +29,7 @@ import java.util.Set;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.CyclicBarrier;
 
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.TokenFilter;
@@ -886,7 +887,9 @@ public class TestSuggestField extends LuceneTestCase {
     IndexWriterConfig iwc = newIndexWriterConfig(random(), analyzer);
     iwc.setMergePolicy(newLogMergePolicy());
     Codec filterCodec = new Lucene80Codec() {
-      PostingsFormat postingsFormat = new Completion50PostingsFormat();
+      CompletionPostingsFormat.FSTLoadMode fstLoadMode =
+          RandomPicks.randomFrom(random(), CompletionPostingsFormat.FSTLoadMode.values());
+      PostingsFormat postingsFormat = new Completion50PostingsFormat(fstLoadMode);
 
       @Override
       public PostingsFormat getPostingsFormatForField(String field) {