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

[lucene-solr] branch master updated: LUCENE-9049: Remove FST cached root arcs now redundant with labels indexed by bitset.

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

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


The following commit(s) were added to refs/heads/master by this push:
     new e43b5ea  LUCENE-9049: Remove FST cached root arcs now redundant with labels indexed by bitset.
e43b5ea is described below

commit e43b5eaf2481f92718bf9da00ab998dd04826517
Author: Bruno Roustant <br...@salesforce.com>
AuthorDate: Tue Nov 26 15:57:29 2019 +0100

    LUCENE-9049: Remove FST cached root arcs now redundant with labels indexed by bitset.
---
 lucene/CHANGES.txt                                 |   3 +
 .../src/java/org/apache/lucene/util/fst/FST.java   | 117 ---------------------
 2 files changed, 3 insertions(+), 117 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 07a5929..f23952c 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -118,6 +118,9 @@ Optimizations
 
 * LUCENE-9027: Use SIMD instructions to decode postings. (Adrien Grand)
 
+* LUCENE-9049: Remove FST cached root arcs now redundant with labels indexed by bitset.
+  This frees some on-heap FST space. (Jack Conradson via Bruno Roustant)
+
 Bug Fixes
 
 * LUCENE-9001: Fix race condition in SetOnce. (Przemko Robakowski)
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/FST.java b/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
index 60be7b5..46d4b78 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
@@ -127,9 +127,6 @@ public final class FST<T> implements Accountable {
   // non-final node w/ no arcs:
   private static final long NON_FINAL_END_NODE = 0;
 
-  /* Used for memory accounting */
-  private int cachedArcsBytesUsed;
-
   /** If arc has this label then that arc is final/accepted */
   public static final int END_LABEL = -1;
 
@@ -150,8 +147,6 @@ public final class FST<T> implements Accountable {
 
   public final Outputs<T> outputs;
 
-  private Arc<T>[] cachedRootArcs;
-
   /** Represents a single arc. */
   public static final class Arc<T> {
 
@@ -462,26 +457,6 @@ public final class FST<T> implements Accountable {
 
     long numBytes = in.readVLong();
     this.fstStore.init(in, numBytes);
-    cacheRootArcs();
-  }
-
-  private long ramBytesUsed(Arc<T>[] arcs) {
-    long size = 0;
-    if (arcs != null) {
-      size += RamUsageEstimator.shallowSizeOf(arcs);
-      for (Arc<T> arc : arcs) {
-        if (arc != null) {
-          size += ARC_SHALLOW_RAM_BYTES_USED;
-          if (arc.output() != null && arc.output() != outputs.getNoOutput()) {
-            size += outputs.ramBytesUsed(arc.output());
-          }
-          if (arc.nextFinalOutput() != null && arc.nextFinalOutput() != outputs.getNoOutput()) {
-            size += outputs.ramBytesUsed(arc.nextFinalOutput());
-          }
-        }
-      }
-    }
-    return size;
   }
 
   @Override
@@ -493,7 +468,6 @@ public final class FST<T> implements Accountable {
       size += bytes.ramBytesUsed();
     }
 
-    size += cachedArcsBytesUsed;
     return size;
   }
 
@@ -512,44 +486,6 @@ public final class FST<T> implements Accountable {
     }
     startNode = newStartNode;
     bytes.finish();
-    cacheRootArcs();
-  }
-  
-  // Optionally caches first 128 labels
-  @SuppressWarnings({"rawtypes","unchecked"})
-  private void cacheRootArcs() throws IOException {
-    // We should only be called once per FST:
-    assert cachedArcsBytesUsed == 0;
-
-    final Arc<T> arc = new Arc<>();
-    getFirstArc(arc);
-    if (targetHasArcs(arc)) {
-      final BytesReader in = getBytesReader();
-      Arc<T>[] arcs = (Arc<T>[]) new Arc[0x80];
-      readFirstRealTargetArc(arc.target(), arc, in);
-      int count = 0;
-      while(true) {
-        assert arc.label() != END_LABEL;
-        if (arc.label() < arcs.length) {
-          arcs[arc.label()] = new Arc<T>().copyFrom(arc);
-        } else {
-          break;
-        }
-        if (arc.isLast()) {
-          break;
-        }
-        readNextRealArc(arc, in);
-        count++;
-      }
-
-      int cacheRAM = (int) ramBytesUsed(arcs);
-
-      // Don't cache if there are only a few arcs or if the cache would use > 20% RAM of the FST itself:
-      if (count >= FIXED_LENGTH_ARC_SHALLOW_NUM_ARCS && cacheRAM < ramBytesUsed()/5) {
-        cachedRootArcs = arcs;
-        cachedArcsBytesUsed = cacheRAM;
-      }
-    }
   }
   
   public T getEmptyOutput() {
@@ -1375,49 +1311,12 @@ public final class FST<T> implements Accountable {
     }
   }
 
-  // LUCENE-5152: called only from asserts, to validate that the
-  // non-cached arc lookup would produce the same result, to
-  // catch callers that illegally modify shared structures with
-  // the result (we shallow-clone the Arc itself, but e.g. a BytesRef
-  // output is still shared):
-  private boolean assertRootCachedArc(int label, Arc<T> cachedArc) throws IOException {
-    Arc<T> arc = new Arc<>();
-    getFirstArc(arc);
-    BytesReader in = getBytesReader();
-    Arc<T> result = findTargetArc(label, arc, arc, in, false);
-    if (result == null) {
-      assert cachedArc == null;
-    } else {
-      assert cachedArc != null;
-      assert cachedArc.arcIdx() == result.arcIdx();
-      assert cachedArc.bytesPerArc() == result.bytesPerArc();
-      assert cachedArc.flags() == result.flags();
-      assert cachedArc.label() == result.label();
-      assert cachedArc.bytesPerArc() != 0 || cachedArc.nextArc() == result.nextArc();
-      assert cachedArc.nextFinalOutput().equals(result.nextFinalOutput());
-      assert cachedArc.numArcs() == result.numArcs();
-      assert cachedArc.output().equals(result.output());
-      assert cachedArc.posArcsStart() == result.posArcsStart();
-      assert cachedArc.target() == result.target();
-      assert cachedArc.nodeFlags() == result.nodeFlags();
-      assert cachedArc.nodeFlags() != ARCS_FOR_DIRECT_ADDRESSING || cachedArc.firstLabel() == result.firstLabel();
-    }
-
-    return true;
-  }
-
   // TODO: could we somehow [partially] tableize arc lookups
   // like automaton?
 
   /** Finds an arc leaving the incoming arc, replacing the arc in place.
    *  This returns null if the arc was not found, else the incoming arc. */
   public Arc<T> findTargetArc(int labelToMatch, Arc<T> follow, Arc<T> arc, BytesReader in) throws IOException {
-    return findTargetArc(labelToMatch, follow, arc, in, true);
-  }
-
-  /** Finds an arc leaving the incoming arc, replacing the arc in place.
-   *  This returns null if the arc was not found, else the incoming arc. */
-  private Arc<T> findTargetArc(int labelToMatch, Arc<T> follow, Arc<T> arc, BytesReader in, boolean useRootArcCache) throws IOException {
 
     if (labelToMatch == END_LABEL) {
       if (follow.isFinal()) {
@@ -1437,22 +1336,6 @@ public final class FST<T> implements Accountable {
       }
     }
 
-    // Short-circuit if this arc is in the root arc cache:
-    if (useRootArcCache && cachedRootArcs != null && follow.target() == startNode && labelToMatch < cachedRootArcs.length) {
-      final Arc<T> result = cachedRootArcs[labelToMatch];
-
-      // LUCENE-5152: detect tricky cases where caller
-      // modified previously returned cached root-arcs:
-      assert assertRootCachedArc(labelToMatch, result);
-
-      if (result == null) {
-        return null;
-      } else {
-        arc.copyFrom(result);
-        return arc;
-      }
-    }
-
     if (!targetHasArcs(follow)) {
       return null;
     }