You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by "dungba88 (via GitHub)" <gi...@apache.org> on 2023/10/04 22:11:58 UTC

[PR] Allow FST builder to use different writer (#12543) [lucene]

dungba88 opened a new pull request, #12624:
URL: https://github.com/apache/lucene/pull/12624

   ### Description
   
   Refactor the method in `BytesStore` needed for FST construction to an abstract class and allow it to be passed from `FSTCompiler.Builder`. The Builder will still maintain `bytesPageBits(int)` for backward-compatibility, in which it will pass the BytesStore.
   
   Issue: https://github.com/apache/lucene/issues/12543


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1391172089


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -120,31 +122,54 @@ public class FSTCompiler<T> {
   final float directAddressingMaxOversizingFactor;
   long directAddressingExpansionCredit;
 
-  final BytesStore bytes;
+  // writer for frozen nodes
+  final FSTDataOutputWriter fstWriter;
+  // reader for the frozen nodes
+  final FSTReader fstReader;

Review Comment:
   ~~That's true, I'll remove it.~~
   
   Actually, the FSTReader was needed to create the FST itself. This reader was mainly used for:
   - Read from the FST with `getBytesReader()`
   - Save the FST with `save()`
   
   So even if we track the size ourselves, we still need to have the FSTReader here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1391973476


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -26,7 +26,8 @@
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements FSTReader {
+// TODO: Separate the scratch writer and reader functionality
+class BytesStore extends DataOutput implements FSTReader, Freezeable {

Review Comment:
   > Do we really need to publish a Freezable interface
   
   Now thinking again, I think even the ByteBuffersDataOutput would need freezing as well, otherwise its last block will always be a full block, and might be a waste of RAM. Freezable provides a chance to optimize those datastructure as they are not to be modified again.
   
   Anyhow I removed the interface and the method for now. It's easy to add back.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1392197630


##########
lucene/core/src/java/org/apache/lucene/util/fst/ByteBuffersFSTReader.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/** An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader} */
+final class ByteBuffersFSTReader extends DataOutput implements FSTReader {

Review Comment:
   I'm nervous having the ByteBuffersDataOutput to implement FSTReader directly, hence the adapter class



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1391146752


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -26,7 +26,8 @@
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements FSTReader {
+// TODO: Separate the scratch writer and reader functionality
+class BytesStore extends DataOutput implements FSTReader, Freezeable {

Review Comment:
   Hm, I'm wondering why should we let the FSTCompiler to know about the RandomAccessInput? In the most case, we would let the FSTCompiler to just write to a DataOutput, then users are to read it with the public FST constructor right?
   
   The case where we want to read from the FST immediately after writing is the one with BytesStore.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1391172089


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -120,31 +122,54 @@ public class FSTCompiler<T> {
   final float directAddressingMaxOversizingFactor;
   long directAddressingExpansionCredit;
 
-  final BytesStore bytes;
+  // writer for frozen nodes
+  final FSTDataOutputWriter fstWriter;
+  // reader for the frozen nodes
+  final FSTReader fstReader;

Review Comment:
   ~~That's true, I'll remove it.~~
   
   Actually, the FSTReader was needed to create the FST itself. This reader was mainly used for:
   - Read from the FST with `getBytesReader()`
   - Save the FST with `save()`
   
   So even if we track the size ourselves, we still need to have the FSTReader when creating the FST object. But I have removed it from the class attributes.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1392201110


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,12 +21,13 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
-// TODO: merge with PagedBytes, except PagedBytes doesn't
-// let you read while writing which FST needs
+// TODO: merge with or use PagedBytes/ByteBuffersDataOutput, as we no longer need to read while
+// writing
 
-class BytesStore extends DataOutput implements FSTReader {
+class BytesStore extends DataOutput implements Accountable {

Review Comment:
   This class originally has 3 purposes:
   - Act as a scratch area writer with some complex backward writing operations
   - Acts as a DataOutput to write the FST frozen nodes
   - Acts as a FSTReader to read from the FST once constructed
   
   The 2nd and 3rd use case has been moved to use ByteBuffersDataOutput, so it is only used as the scratch area writer now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1400663661


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -827,22 +910,24 @@ void setEmptyOutput(T v) {
   }
 
   void finish(long newStartNode) {
-    assert newStartNode <= bytes.size();
+    assert newStartNode <= numBytesWritten;
     if (fst.metadata.startNode != -1) {
       throw new IllegalStateException("already finished");
     }
     if (newStartNode == FINAL_END_NODE && fst.metadata.emptyOutput != null) {
       newStartNode = 0;
     }
     fst.metadata.startNode = newStartNode;
-    fst.metadata.numBytes = bytes.getPosition();
+    fst.metadata.numBytes = numBytesWritten;
   }
 
   private boolean validOutput(T output) {
     return output == NO_OUTPUT || !output.equals(NO_OUTPUT);
   }
 
   /** Returns final FST. NOTE: this will return null if nothing is accepted by the FST. */
+  // TODO: make this method to only return the FSTMetadata and user needs to construct the FST
+  // themselves
   public FST<T> compile() throws IOException {

Review Comment:
   Currently if we try to read from it, the reader would return null (and be changed to throw an IllegalStateException). If we change this method to just return the FSTMetadata the user will be mindful that the FST might not be readable and we will never return an unusable FST. But yeah it will touch a lot of codes.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1402022525


##########
lucene/core/src/java/org/apache/lucene/util/fst/FST.java:
##########
@@ -435,6 +433,13 @@ public FST(FSTMetadata<T> metadata, DataInput in, Outputs<T> outputs, FSTStore f
     this.fstReader = fstReader;
   }
 
+  /**
+   * @return true if and only if this FST is readable (e.g has a reverse BytesReader)
+   */
+  public boolean isReadable() {

Review Comment:
   We can remove this in the followon PR (the one that changes FSTCompiler to return FSTMetadata not FST)?



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsWriter.java:
##########
@@ -16,6 +16,8 @@
  */
 package org.apache.lucene.codecs.lucene90.blocktree;
 
+import static org.apache.lucene.util.fst.FSTCompiler.getOnHeapDataOutput;

Review Comment:
   Can we rename to `getOnHeapReaderWriter`?



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -248,15 +305,17 @@ public Builder<T> directAddressingMaxOversizingFactor(float factor) {
 
     /** Creates a new {@link FSTCompiler}. */
     public FSTCompiler<T> build() {
-      FSTCompiler<T> fstCompiler =
-          new FSTCompiler<>(
-              inputType,
-              suffixRAMLimitMB,
-              outputs,
-              allowFixedLengthArcs,
-              bytesPageBits,
-              directAddressingMaxOversizingFactor);
-      return fstCompiler;
+      try {
+        return new FSTCompiler<>(
+            inputType,
+            suffixRAMLimitMB,
+            outputs,
+            allowFixedLengthArcs,
+            dataOutput,
+            directAddressingMaxOversizingFactor);
+      } catch (IOException e) {
+        throw new RuntimeException(e);

Review Comment:
   OK I merged #12830.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "uschindler (via GitHub)" <gi...@apache.org>.
uschindler commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1418764876


##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader, Freezable {
+
+  private final ByteBuffersDataOutput dataOutput;
+  // the DataInput to read from in case the DataOutput has multiple blocks
+  private ByteBuffersDataInput dataInput;
+  // the ByteBuffers to read from in case the DataOutput has a single block
+  private ByteBuffer byteBuffer;
+
+  public ReadWriteDataOutput(ByteBuffersDataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    dataOutput.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) {
+    dataOutput.writeBytes(b, offset, length);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return dataOutput.ramBytesUsed();
+  }
+
+  @Override
+  public void freeze() {
+    // these operations are costly, so we want to compute it once and cache
+    List<ByteBuffer> byteBuffers = dataOutput.toWriteableBufferList();
+    if (byteBuffers.size() == 1) {

Review Comment:
   P.S.: I have not looked into this PR, this is just my general suggestion regarding ByteBuffer and escape analysis, see also #12841.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1390363849


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -26,7 +27,8 @@
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements FSTReader {
+// TODO: Separate the scratch writer and reader functionality
+class BytesStore extends DataOutput implements FSTReader, Closeable {

Review Comment:
   I was thinking of changing this to something like Freezable or Finishable



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1415876855


##########
lucene/core/src/test/org/apache/lucene/util/fst/Test2BFSTOffHeap.java:
##########
@@ -0,0 +1,341 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.MMapDirectory;
+import org.apache.lucene.tests.util.LuceneTestCase;
+import org.apache.lucene.tests.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.lucene.tests.util.TimeUnits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
+
+// Similar to Test2BFST but will build and read the FST off-heap and can be run with small heap
+
+// Run something like this:
+//    ./gradlew test --tests Test2BFSTOffHeap -Dtests.verbose=true --max-workers=1
+
+// @Ignore("Requires tons of heap to run (30 GB hits OOME but 35 GB passes after ~4.5 hours)")

Review Comment:
   Ah I should have turned on this @Ignore on the PR!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1777157731

   @mikemccand 
   
   I rebased and created some implementation of DataOutput-based FSTWriter. I think I need to write tests, but let me know what you think.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1400539919


##########
lucene/core/src/java/org/apache/lucene/util/fst/ByteBuffersFSTReader.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/** An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader} */
+final class ByteBuffersFSTReader extends DataOutput implements FSTReader {

Review Comment:
   It's kinda of jarring that this class is named `XFSTReader`, implements `FSTReader`, yet extends `DataOutput`?  Is it a reader or a writer?



##########
lucene/core/src/java/org/apache/lucene/util/fst/FST.java:
##########
@@ -123,10 +124,7 @@ public enum INPUT_TYPE {
   /** If arc has this label then that arc is final/accepted */
   public static final int END_LABEL = -1;
 
-  /**
-   * A {@link BytesStore}, used during building, or during reading when the FST is very large (more
-   * than 1 GB). If the FST is less than 1 GB then bytesArray is set instead.
-   */
+  /** The reader of the FST */

Review Comment:
   Maybe `Used to read bytes from the underlying FST storage` or so?



##########
lucene/core/src/java/org/apache/lucene/util/fst/FST.java:
##########
@@ -435,6 +433,13 @@ public FST(FSTMetadata<T> metadata, DataInput in, Outputs<T> outputs, FSTStore f
     this.fstReader = fstReader;
   }
 
+  /**
+   * @return true if and only if this FST is readable (i.e. has a reverse BytesReader)
+   */
+  public boolean hasReverseBytesReader() {

Review Comment:
   Is it only an FST being actively written by `FSTCompiler` that would not be readable?



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -120,22 +120,45 @@ public class FSTCompiler<T> {
   final float directAddressingMaxOversizingFactor;
   long directAddressingExpansionCredit;
 
-  final BytesStore bytes;
+  // the DataOutput to write the FST to

Review Comment:
   Maybe `to stream the FST bytes to`?  Emphasize the append-only-ness of it.



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -441,23 +501,25 @@ long addNode(FSTCompiler.UnCompiledNode<T> nodeIn) throws IOException {
       boolean continuousLabel = labelRange == nodeIn.numArcs;
       if (continuousLabel) {
         writeNodeForDirectAddressingOrContinuous(
-            nodeIn, startAddress, maxBytesPerArcWithoutLabel, labelRange, true);
+            nodeIn, maxBytesPerArcWithoutLabel, labelRange, true);
         continuousNodeCount++;
       } else if (shouldExpandNodeWithDirectAddressing(
           nodeIn, maxBytesPerArc, maxBytesPerArcWithoutLabel, labelRange)) {
         writeNodeForDirectAddressingOrContinuous(
-            nodeIn, startAddress, maxBytesPerArcWithoutLabel, labelRange, false);
+            nodeIn, maxBytesPerArcWithoutLabel, labelRange, false);
         directAddressingNodeCount++;
       } else {
-        writeNodeForBinarySearch(nodeIn, startAddress, maxBytesPerArc);
+        writeNodeForBinarySearch(nodeIn, maxBytesPerArc);
         binarySearchNodeCount++;
       }
     }
 
-    final long thisNodeAddress = bytes.getPosition() - 1;
-    bytes.reverse(startAddress, thisNodeAddress);
+    reverseScratchBytes();

Review Comment:
   Yay, thank you!  So much cleaner that caller (us, here) do this reversal than the underlying storage impl.



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -568,18 +629,44 @@ private void writeNodeForBinarySearch(
                   + arcLen
                   + " nodeIn.numArcs="
                   + nodeIn.numArcs;
-          bytes.copyBytes(srcPos, destPos, arcLen);
+          // copy the bytes from srcPos to destPos, essentially expanding the arc from variable
+          // length to fixed length
+          writeScratchBytes(destPos, scratchBytes.getBytes(), srcPos, arcLen);
         }
       }
     }
 
-    // Write the header.
-    bytes.writeBytes(startAddress, fixedLengthArcsBuffer.getBytes(), 0, headerLen);
+    // Finally write the header
+    writeScratchBytes(0, fixedLengthArcsBuffer.getBytes(), 0, headerLen);
+  }
+
+  /** Reverse the scratch bytes */
+  private void reverseScratchBytes() {
+    int pos = scratchBytes.getPosition();
+    byte[] bytes = scratchBytes.getBytes();
+    int limit = pos / 2;
+    for (int i = 0; i < limit; i++) {
+      byte b = bytes[i];
+      bytes[i] = bytes[pos - 1 - i];
+      bytes[pos - 1 - i] = b;
+    }
+  }
+
+  /**
+   * Write bytes from a source byte[] to the scratch bytes

Review Comment:
   Maybe add that the written bytes must fit within what's already been written to `scratchBytes`?  I.e. you are only "rewriting" what was already written, never extending the `scratchBytes`?  And maybe also state that `scratchBytes.getPosition()` is unaffected by this.



##########
lucene/core/src/java/org/apache/lucene/util/fst/GrowableByteArrayDataOutput.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
+
+// Storing a byte[] for the current node of the FST we are writing. The byte[] will only grow, never
+// shrink.
+final class GrowableByteArrayDataOutput extends DataOutput implements Accountable {
+
+  private static final long BASE_RAM_BYTES_USED =
+      RamUsageEstimator.shallowSizeOfInstance(GrowableByteArrayDataOutput.class);
+
+  private static final int INITIAL_SIZE = 1 << 8;
+
+  // holds an initial size of 256 bytes. this byte array will only grow, but not shrink
+  private byte[] bytes = new byte[INITIAL_SIZE];
+
+  private int nextWrite;
+
+  @Override
+  public void writeByte(byte b) {
+    ensureCapacity(1);
+    bytes[nextWrite++] = b;
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int len) {
+    ensureCapacity(len);
+    System.arraycopy(b, offset, bytes, nextWrite, len);
+    nextWrite += len;
+  }
+
+  public int getPosition() {
+    return nextWrite;
+  }
+
+  public byte[] getBytes() {
+    return bytes;
+  }
+
+  /** Set the position of the byte[], increasing the capacity if needed */
+  public void setPosition(int newLen) {
+    assert newLen >= 0;
+    if (newLen > nextWrite) {
+      ensureCapacity(newLen - nextWrite);
+    }
+    nextWrite = newLen;

Review Comment:
   Hmm should we ever downsize the `byte[]`?  When building a big FST it will grow to hold biggest frozen node it's seen so far?  I think that will tend to be towards the very end of FST building?  Also, the max frozen FST node size is ... not too big?  I suppose if you are using full Unicode alphabet it might be massive, hmm.  I don't think we need to shrink for now?



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -218,13 +269,19 @@ public Builder<T> allowFixedLengthArcs(boolean allowFixedLengthArcs) {
     }
 
     /**
-     * How many bits wide to make each byte[] block in the BytesStore; if you know the FST will be
-     * large then make this larger. For example 15 bits = 32768 byte pages.
+     * Set the {@link DataOutput} which is used for low-level writing of FST. If you want the FST to
+     * be readable, you need to use a DataOutput that also implements {@link FSTReader}, such as

Review Comment:
   Maybe `to be immediately readable`?



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -120,22 +120,45 @@ public class FSTCompiler<T> {
   final float directAddressingMaxOversizingFactor;
   long directAddressingExpansionCredit;
 
-  final BytesStore bytes;
+  // the DataOutput to write the FST to
+  final DataOutput dataOutput;
+
+  // buffer to store bytes for the one node we are currently writing
+  final GrowableByteArrayDataOutput scratchBytes = new GrowableByteArrayDataOutput();
+
+  private long numBytesWritten;
+
+  /**
+   * Get an on-heap DataOutput that allows the FST to be read immediately after writing.
+   *
+   * @param blockBits how many bits wide to make each block of the DataOutput
+   * @return the DataOutput
+   */
+  public static DataOutput getOnHeapDataOutput(int blockBits) {

Review Comment:
   Nice.



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -153,6 +176,34 @@ private FSTCompiler(
     }
   }
 
+  // Get the respective FSTReader of the DataOutput. If the DataOutput is also a FSTReader then we
+  // will use it. Otherwise, we will use NullFSTReader, which does not allow reading.
+  private FSTReader toFSTReader(DataOutput dataOutput) {
+    if (dataOutput instanceof FSTReader) {
+      return (FSTReader) dataOutput;
+    }
+    return new NullFSTReader();
+  }
+
+  private static final class NullFSTReader implements FSTReader {
+
+    @Override
+    public FST.BytesReader getReverseBytesReader() {
+      return null;
+    }
+
+    @Override
+    public void writeTo(DataOutput out) {
+      throw new UnsupportedOperationException("writeTo(DataOutput) is not supported");
+    }
+
+    @Override
+    public long ramBytesUsed() {
+      return 0;
+    }
+  }
+  ;

Review Comment:
   Hmm is this one of those deadly lurking semicolons?



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -568,18 +629,44 @@ private void writeNodeForBinarySearch(
                   + arcLen
                   + " nodeIn.numArcs="
                   + nodeIn.numArcs;
-          bytes.copyBytes(srcPos, destPos, arcLen);
+          // copy the bytes from srcPos to destPos, essentially expanding the arc from variable
+          // length to fixed length
+          writeScratchBytes(destPos, scratchBytes.getBytes(), srcPos, arcLen);
         }
       }
     }
 
-    // Write the header.
-    bytes.writeBytes(startAddress, fixedLengthArcsBuffer.getBytes(), 0, headerLen);
+    // Finally write the header
+    writeScratchBytes(0, fixedLengthArcsBuffer.getBytes(), 0, headerLen);
+  }
+
+  /** Reverse the scratch bytes */

Review Comment:
   Maybe add `does not affect scratchBytes.getPosition()`.  And say "in place", e.g. `Reverse the scratch bytes in place`?



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -153,6 +176,34 @@ private FSTCompiler(
     }
   }
 
+  // Get the respective FSTReader of the DataOutput. If the DataOutput is also a FSTReader then we
+  // will use it. Otherwise, we will use NullFSTReader, which does not allow reading.
+  private FSTReader toFSTReader(DataOutput dataOutput) {
+    if (dataOutput instanceof FSTReader) {
+      return (FSTReader) dataOutput;
+    }
+    return new NullFSTReader();

Review Comment:
   Can we throw an exception instead?  I'd rather not mask (from the caller) that what they are trying to do won't work.  Like if they try to read anything from the `NullFSTReader` it will fail?



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -277,9 +336,9 @@ public long getMappedStateCount() {
     return dedupHash == null ? 0 : nodeCount;
   }
 
-  private CompiledNode compileNode(UnCompiledNode<T> nodeIn, int tailLength) throws IOException {
+  private CompiledNode compileNode(UnCompiledNode<T> nodeIn) throws IOException {

Review Comment:
   Ahh this `int tailLength` was unused now?



##########
lucene/core/src/java/org/apache/lucene/util/fst/GrowableByteArrayDataOutput.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
+
+// Storing a byte[] for the current node of the FST we are writing. The byte[] will only grow, never

Review Comment:
   Maybe add `into a single contiguous byte[]`?  I.e. no blocks.



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -827,22 +910,24 @@ void setEmptyOutput(T v) {
   }
 
   void finish(long newStartNode) {
-    assert newStartNode <= bytes.size();
+    assert newStartNode <= numBytesWritten;
     if (fst.metadata.startNode != -1) {
       throw new IllegalStateException("already finished");
     }
     if (newStartNode == FINAL_END_NODE && fst.metadata.emptyOutput != null) {
       newStartNode = 0;
     }
     fst.metadata.startNode = newStartNode;
-    fst.metadata.numBytes = bytes.getPosition();
+    fst.metadata.numBytes = numBytesWritten;
   }
 
   private boolean validOutput(T output) {
     return output == NO_OUTPUT || !output.equals(NO_OUTPUT);
   }
 
   /** Returns final FST. NOTE: this will return null if nothing is accepted by the FST. */
+  // TODO: make this method to only return the FSTMetadata and user needs to construct the FST
+  // themselves
   public FST<T> compile() throws IOException {

Review Comment:
   Ahh good point -- what can this returned `FST<T>` actually do?  It's invalid to try to use it if you backed your `DataOutput` by e.g. an `IndexOutput` writing to disk, unable to give a `DataInput` immediately?  What happens if you try?  This seems like important TODO but will touch so much code ... we can defer.



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -248,15 +305,17 @@ public Builder<T> directAddressingMaxOversizingFactor(float factor) {
 
     /** Creates a new {@link FSTCompiler}. */
     public FSTCompiler<T> build() {
-      FSTCompiler<T> fstCompiler =
-          new FSTCompiler<>(
-              inputType,
-              suffixRAMLimitMB,
-              outputs,
-              allowFixedLengthArcs,
-              bytesPageBits,
-              directAddressingMaxOversizingFactor);
-      return fstCompiler;
+      try {
+        return new FSTCompiler<>(
+            inputType,
+            suffixRAMLimitMB,
+            outputs,
+            allowFixedLengthArcs,
+            dataOutput,
+            directAddressingMaxOversizingFactor);
+      } catch (IOException e) {
+        throw new RuntimeException(e);

Review Comment:
   Can we add `throws IOException` to the method signature?  It's important caller knows when writing to a storage device which has some problem that they may see a (legit) `IOException` now?



##########
lucene/core/src/java/org/apache/lucene/util/fst/GrowableByteArrayDataOutput.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
+
+// Storing a byte[] for the current node of the FST we are writing. The byte[] will only grow, never
+// shrink.
+final class GrowableByteArrayDataOutput extends DataOutput implements Accountable {
+
+  private static final long BASE_RAM_BYTES_USED =
+      RamUsageEstimator.shallowSizeOfInstance(GrowableByteArrayDataOutput.class);
+
+  private static final int INITIAL_SIZE = 1 << 8;
+
+  // holds an initial size of 256 bytes. this byte array will only grow, but not shrink
+  private byte[] bytes = new byte[INITIAL_SIZE];
+
+  private int nextWrite;
+
+  @Override
+  public void writeByte(byte b) {
+    ensureCapacity(1);
+    bytes[nextWrite++] = b;
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int len) {
+    ensureCapacity(len);
+    System.arraycopy(b, offset, bytes, nextWrite, len);
+    nextWrite += len;
+  }
+
+  public int getPosition() {
+    return nextWrite;
+  }
+
+  public byte[] getBytes() {
+    return bytes;
+  }
+
+  /** Set the position of the byte[], increasing the capacity if needed */
+  public void setPosition(int newLen) {
+    assert newLen >= 0;
+    if (newLen > nextWrite) {
+      ensureCapacity(newLen - nextWrite);
+    }
+    nextWrite = newLen;
+  }
+
+  /**
+   * Ensure we can write additional capacityToWrite bytes.
+   *
+   * @param capacityToWrite the additional bytes to write
+   */
+  private void ensureCapacity(int capacityToWrite) {
+    bytes = ArrayUtil.grow(bytes, nextWrite + capacityToWrite);

Review Comment:
   Maybe `assert capacityToWrite > 0`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1805190081

   @mikemccand I put out another revision. Basically the idea is to write everything to a DataOutput (BytesStore is also a DataOutput). To support write-then-read-immediately use case that we are able to do today, if the DataOutput implements the FSTReader then we will use it to read. Otherwise we are following a general use case, which is to use the FST public constructor to read from previously compiled one. Let me know what you think.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dweiss (via GitHub)" <gi...@apache.org>.
dweiss commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1416950035


##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader, Freezable {
+
+  private final ByteBuffersDataOutput dataOutput;
+  // the DataInput to read from in case the DataOutput has multiple blocks
+  private ByteBuffersDataInput dataInput;
+  // the ByteBuffers to read from in case the DataOutput has a single block
+  private ByteBuffer byteBuffer;
+
+  public ReadWriteDataOutput(ByteBuffersDataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    dataOutput.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) {
+    dataOutput.writeBytes(b, offset, length);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return dataOutput.ramBytesUsed();
+  }
+
+  @Override
+  public void freeze() {
+    // these operations are costly, so we want to compute it once and cache
+    List<ByteBuffer> byteBuffers = dataOutput.toWriteableBufferList();
+    if (byteBuffers.size() == 1) {

Review Comment:
   Thanks! My viewpoint is that it's a tradeoff between the amount of additional code to maintain, exposing implementation details (the bit mask exposed in 12879) and raw speed. If it's an order of magnitude then sure - something is not right. But if it's a few seconds on millions of lookups then perhaps in real use cases it's not even visible and the cost of a lot of additional code may not be worth the gain?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1418892433


##########
lucene/core/src/java/org/apache/lucene/util/fst/FST.java:
##########
@@ -419,7 +418,8 @@ public FST(FSTMetadata<T> metadata, DataInput in, FSTStore fstStore) throws IOEx
 
   /** Create the FST with a metadata object and a FSTReader. */
   FST(FSTMetadata<T> metadata, FSTReader fstReader) {
-    this.metadata = metadata;
+    assert fstReader != null;

Review Comment:
   Hmm also this one?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "uschindler (via GitHub)" <gi...@apache.org>.
uschindler commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1418761384


##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader, Freezable {
+
+  private final ByteBuffersDataOutput dataOutput;
+  // the DataInput to read from in case the DataOutput has multiple blocks
+  private ByteBuffersDataInput dataInput;
+  // the ByteBuffers to read from in case the DataOutput has a single block
+  private ByteBuffer byteBuffer;
+
+  public ReadWriteDataOutput(ByteBuffersDataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    dataOutput.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) {
+    dataOutput.writeBytes(b, offset, length);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return dataOutput.ramBytesUsed();
+  }
+
+  @Override
+  public void freeze() {
+    // these operations are costly, so we want to compute it once and cache
+    List<ByteBuffer> byteBuffers = dataOutput.toWriteableBufferList();
+    if (byteBuffers.size() == 1) {

Review Comment:
   Yes, please stick with ByteBuffer. I have a similar suggestion with the stupid groupVInts. Don't splatter our code with highly specialized implementations everywhere. Create a default ByteBuffer impl. There's no overhead as the JVM has optimized ByteBuffer access to make those invisible.
   
   What you see in benchmarks like this is the overhead of C2 jumping in. To make a "correct" benchmark use JMH. In addition: Don't trust profilers, they are useless in those areas as they measure bullshit.
   
   So:
   - create a JMH benchmark
   - use ByteBuffer only and use ByteBuffer.wrap() to temporarily create wrappers around other implementations like byte[] or MemorySegment



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1391146752


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -26,7 +26,8 @@
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements FSTReader {
+// TODO: Separate the scratch writer and reader functionality
+class BytesStore extends DataOutput implements FSTReader, Freezeable {

Review Comment:
   Hm, I'm wondering why should we let the FSTCompiler to know about the RandomAccessInput? In the most case, we would let the FSTCompiler to just write to a DataOutput, then users are to read it with the public FST constructor right?
   
   The case where we want to read from the FST immediately after writing is the one with BytesStore.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1391973476


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -26,7 +26,8 @@
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements FSTReader {
+// TODO: Separate the scratch writer and reader functionality
+class BytesStore extends DataOutput implements FSTReader, Freezeable {

Review Comment:
   Now thinking again, I think even the ByteBuffersDataOutput would need freezing as well, otherwise its last block will always be a full block, and might be a waste of RAM.
   
   Anyhow I removed the interface and the method for now. It's easy to add back.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1373993909


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -287,9 +315,9 @@ public long getMappedStateCount() {
     return dedupHash == null ? 0 : nodeCount;
   }
 
-  private CompiledNode compileNode(UnCompiledNode<T> nodeIn, int tailLength) throws IOException {

Review Comment:
   This `tailLength` is not being used anywhere, hence removed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1415627354


##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader, Freezable {
+
+  private final ByteBuffersDataOutput dataOutput;
+  // the DataInput to read from in case the DataOutput has multiple blocks
+  private ByteBuffersDataInput dataInput;
+  // the ByteBuffers to read from in case the DataOutput has a single block
+  private ByteBuffer byteBuffer;
+
+  public ReadWriteDataOutput(ByteBuffersDataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    dataOutput.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) {
+    dataOutput.writeBytes(b, offset, length);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return dataOutput.ramBytesUsed();
+  }
+
+  @Override
+  public void freeze() {
+    // these operations are costly, so we want to compute it once and cache
+    List<ByteBuffer> byteBuffers = dataOutput.toWriteableBufferList();
+    if (byteBuffers.size() == 1) {

Review Comment:
   I think it didn't handle the single buffer use case, and ByteBuffersDataInput would fall into the same performance regression problem as BytesStore with multiple blocks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1778455090

   There is one thing that baffled me is that we are writing the metadata, including the numBytes & start node in the beginning of the DataOutput. That means once the FST is completed, we can't write the metadata to the same DataOutput as the main body, we have to write it to a different DataOutput (that's why I added a method to just write the metadata).
   
   Wondering if it makes sense to write the metadata on the end of the DataOutput (and also read from the end) instead.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1402017454


##########
lucene/core/src/java/org/apache/lucene/util/fst/ByteBuffersFSTReader.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/** An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader} */
+final class ByteBuffersFSTReader extends DataOutput implements FSTReader {

Review Comment:
   Maybe just `ReadWriteDataOutput` or so?



##########
lucene/core/src/java/org/apache/lucene/util/fst/ByteBuffersFSTReader.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/** An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader} */
+final class ByteBuffersFSTReader extends DataOutput implements FSTReader {

Review Comment:
   Naming is the hardest part!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1400663661


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -827,22 +910,24 @@ void setEmptyOutput(T v) {
   }
 
   void finish(long newStartNode) {
-    assert newStartNode <= bytes.size();
+    assert newStartNode <= numBytesWritten;
     if (fst.metadata.startNode != -1) {
       throw new IllegalStateException("already finished");
     }
     if (newStartNode == FINAL_END_NODE && fst.metadata.emptyOutput != null) {
       newStartNode = 0;
     }
     fst.metadata.startNode = newStartNode;
-    fst.metadata.numBytes = bytes.getPosition();
+    fst.metadata.numBytes = numBytesWritten;
   }
 
   private boolean validOutput(T output) {
     return output == NO_OUTPUT || !output.equals(NO_OUTPUT);
   }
 
   /** Returns final FST. NOTE: this will return null if nothing is accepted by the FST. */
+  // TODO: make this method to only return the FSTMetadata and user needs to construct the FST
+  // themselves
   public FST<T> compile() throws IOException {

Review Comment:
   Currently if we try to read from it, the reader would return null (and be changed to throw an IllegalStateException). If we change this method to just return the FSTMetadata the user will be mindful that the FST might not be readable and we will never return an unusable FST. But yeah it will touch a lot of codes.
   
   I also put together this PR for the migration later: https://github.com/dungba88/lucene/pull/19. It will ensure that we will never return an invalid/unusable FST.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1832050665

   I re-ran the Test2BFST with the new change, it looks much better
   
   ```
     1> TEST: now verify [fst size=4621076364; nodeCount=2252341486; arcCount=2264078585]
     1> 0...: took 0 seconds
     1> 1000000...: took 27 seconds
     1> 2000000...: took 54 seconds
     1> 3000000...: took 82 seconds
     1> 4000000...: took 109 seconds
     1> 5000000...: took 137 seconds
     1> 6000000...: took 165 seconds
     1> 7000000...: took 192 seconds
     1> 8000000...: took 219 seconds
     1> 9000000...: took 247 seconds
     1> 10000000...: took 275 seconds
     1> 11000000...: took 302 seconds
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1813856323

   Seems like this PR is getting long, so I spawned 2 PR out of it:
   - https://github.com/apache/lucene/pull/12814: Simplify `BytesStore` operations (which was changed to GrowableByteArrayDataOutput in this PR)
   - https://github.com/apache/lucene/pull/12802: Remove the size() method in FSTReader


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1418919257


##########
lucene/core/src/java/org/apache/lucene/util/fst/FST.java:
##########
@@ -419,7 +418,8 @@ public FST(FSTMetadata<T> metadata, DataInput in, FSTStore fstStore) throws IOEx
 
   /** Create the FST with a metadata object and a FSTReader. */
   FST(FSTMetadata<T> metadata, FSTReader fstReader) {
-    this.metadata = metadata;
+    assert fstReader != null;

Review Comment:
   fstReader is not directly passed by users. If the users call the constructor with FSTStore, that cannot be null and it will throw an NPE due to the call `fstStore.init()`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1840646081

   > > I tested just how much slower the ByteBuffer based store is than the FST's BytesStore:
   > 
   > I assume this is before the last iteration that does the freeze, is that right? What do you think about the last results?
   
   Sorry yes -- I'll try to come back to this soon, and re-test.  Thanks @dungba88 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1415871453


##########
lucene/test-framework/src/java/org/apache/lucene/tests/util/fst/FSTTester.java:
##########
@@ -316,6 +313,15 @@ public FST<T> doTest() throws IOException {
     return fst;
   }
 
+  protected FST<T> compile(FSTCompiler<T> fstCompiler) throws IOException {
+    return fstCompiler.compile();
+  }
+
+  protected FSTCompiler.Builder<T> getFSTBuilder() {
+    return new FSTCompiler.Builder<>(

Review Comment:
   I added this test along with the 2B nodes off-heap



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1415625009


##########
lucene/core/src/test/org/apache/lucene/util/fst/TestFSTDataOutputWriter.java:
##########
@@ -0,0 +1,230 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import static org.apache.lucene.tests.util.fst.FSTTester.toIntsRef;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.tests.store.MockDirectoryWrapper;
+import org.apache.lucene.tests.util.LuceneTestCase;
+import org.apache.lucene.tests.util.TestUtil;
+import org.apache.lucene.tests.util.fst.FSTTester;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
+
+public class TestFSTDataOutputWriter extends LuceneTestCase {
+
+  private MockDirectoryWrapper dir;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    dir = newMockDirectory();
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    // can be null if we force simpletext (funky, some kind of bug in test runner maybe)
+    if (dir != null) {
+      dir.close();
+    }
+    super.tearDown();
+  }
+
+  public void testRandom() throws Exception {
+
+    final int iters = atLeast(10);
+    final int maxBytes = TEST_NIGHTLY ? 200000 : 20000;
+    for (int iter = 0; iter < iters; iter++) {
+      final int numBytes = TestUtil.nextInt(random(), 1, maxBytes);
+      final byte[] expected = new byte[numBytes];
+      final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      final DataOutput dataOutput = new OutputStreamDataOutput(baos);

Review Comment:
   Ah I think it was left over code from previous implementation. Will remove it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1391172089


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -120,31 +122,54 @@ public class FSTCompiler<T> {
   final float directAddressingMaxOversizingFactor;
   long directAddressingExpansionCredit;
 
-  final BytesStore bytes;
+  // writer for frozen nodes
+  final FSTDataOutputWriter fstWriter;
+  // reader for the frozen nodes
+  final FSTReader fstReader;

Review Comment:
   That's true, I'll remove it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1392645763


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -337,11 +349,23 @@ public long size() {
     return getPosition();
   }
 
+  /** Similar to {@link #truncate(long)} with newLen=0 but keep the first block to reduce GC. */
+  public void reset() {

Review Comment:
   Also, do you think this change would be simpler if we go and do #12355 first?  I suppose that won't make FST building any simpler since we'd likely write the whole thing in reverse (like we do now), but then make 2nd pass to re-reverse the bytes so everything becomes forwards.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1392710843


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,12 +21,13 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
-// TODO: merge with PagedBytes, except PagedBytes doesn't
-// let you read while writing which FST needs
+// TODO: merge with or use PagedBytes/ByteBuffersDataOutput, as we no longer need to read while
+// writing
 
-class BytesStore extends DataOutput implements FSTReader {
+class BytesStore extends DataOutput implements Accountable {

Review Comment:
   Do you mean essentially use byte[] instead of byte block? I think that would simpler the writing logic (I was hesitated as it means we would need double RAM when resizing, but not too bad as it's only one node, especially if we keep reusing the same byte[])
   
   But we still need to have the same functionality as the current BytesStore right? I feel like if we just remove the BytesStore down right, it's just moving the complexity from BytesStore to FSTCompiler, and I think having low-level writing in another class is better.
   
   How's about we changing the BytesStore to use byte[] instead of byte block?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1393461923


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -247,16 +306,14 @@ public Builder<T> directAddressingMaxOversizingFactor(float factor) {
     }
 
     /** Creates a new {@link FSTCompiler}. */
-    public FSTCompiler<T> build() {
-      FSTCompiler<T> fstCompiler =
-          new FSTCompiler<>(
-              inputType,
-              suffixRAMLimitMB,
-              outputs,
-              allowFixedLengthArcs,
-              bytesPageBits,
-              directAddressingMaxOversizingFactor);
-      return fstCompiler;
+    public FSTCompiler<T> build() throws IOException {

Review Comment:
   This new `throws IOException` causes diff in some other classes. If preferred I could delay this, and do try-catch here instead to avoid the diffs.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1393547261


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,12 +21,13 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
-// TODO: merge with PagedBytes, except PagedBytes doesn't
-// let you read while writing which FST needs
+// TODO: merge with or use PagedBytes/ByteBuffersDataOutput, as we no longer need to read while
+// writing
 
-class BytesStore extends DataOutput implements FSTReader {
+class BytesStore extends DataOutput implements Accountable {

Review Comment:
   > Hmm where do we do this
   
   TBH I haven't gone through all of them, but one instance is when we write the arc in backward, another is when we first skip the header and after all is written we go back and write the header, presence bits and label.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1407526003


##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader, Freezable {
+
+  private final ByteBuffersDataOutput dataOutput;
+  private ByteBuffersDataInput dataInput;
+  private List<ByteBuffer> byteBuffers;
+
+  public ReadWriteDataOutput(ByteBuffersDataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    dataOutput.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) {
+    dataOutput.writeBytes(b, offset, length);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return dataOutput.ramBytesUsed();
+  }
+
+  @Override
+  public void freeze() {
+    // these operations are costly, so we want to compute it once and cache
+    byteBuffers = dataOutput.toWriteableBufferList();

Review Comment:
   I tried several ways to optimize the `getReverseBytesReader()` method, such as bypassing all `asReadOnlyBuffer()` and accessing the byte array directly, but they wouldn't come close compared to BytesStore:
   - The `ByteBuffersDataOutput.blocks` is a Deque and thus not random accessible. We need to copy to a list, and this takes time as the number of blocks grow.
   - It took ~15 seconds to verify (`Util.get()`) 100K times, compared to only ~2 seconds with BytesStore.
   - I also tried to use PagedBytes, but PagedBytes requires a call to freeze() before getting the DataInput anyway.
   
   Hence I had to re-add `Freezable()` here, so that the costly operations will only be called once. The alternative way is to cache, but `getReverseBytesReader()` can be called concurrently hence we need some kind of ConcurrentHashMap with only 2 items, which is a weird way to cache. Another way to cache, but avoid the added interface is to simply use a volatile variable and have a simple null check (the volatile is there to enforce a write barrier).
   
   If we really don't want to add a new (package-private) interface, then a bit ugly way is to check if the DataOutput is instanceof ReadWriteDataOutput, as this class is the only one implementing the interface anyway.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1829633268

   > More than two orders-of-magnitude (base 10) slower!
   
   I wonder: are there other places in Lucene that might fall prey to this performance trap (calling `toDataInput` frequently while continuing to append bytes to the `DataOutput`)?  `FreqProxTermsWriter` seems to use this for each term to re-read the postings (maybe for the static sort use case).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1415869383


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -120,22 +125,44 @@ public class FSTCompiler<T> {
   final float directAddressingMaxOversizingFactor;
   long directAddressingExpansionCredit;
 
-  final BytesStore bytes;
+  // the DataOutput to stream the FST bytes to
+  final DataOutput dataOutput;
+
+  // buffer to store bytes for the one node we are currently writing
+  final GrowableByteArrayDataOutput scratchBytes = new GrowableByteArrayDataOutput();
+
+  private long numBytesWritten;
+
+  /**
+   * Get an on-heap DataOutput that allows the FST to be read immediately after writing.

Review Comment:
   I added a new Test2BFSTOffHeap and running it:
   
   ```
   100000: 424 RAM bytes used; 39257811 FST bytes; 19189176 nodes; took 23 seconds
   200000: 424 RAM bytes used; 78522623 FST bytes; 38378071 nodes; took 49 seconds
   300000: 424 RAM bytes used; 117788163 FST bytes; 57567190 nodes; took 80 seconds
   400000: 424 RAM bytes used; 157053095 FST bytes; 76756389 nodes; took 107 seconds
   500000: 424 RAM bytes used; 196318494 FST bytes; 95945639 nodes; took 133 seconds
   600000: 424 RAM bytes used; 235583412 FST bytes; 115134691 nodes; took 170 seconds
   700000: 480 RAM bytes used; 274866378 FST bytes; 134324199 nodes; took 198 seconds
   800000: 480 RAM bytes used; 314246540 FST bytes; 153513668 nodes; took 222 seconds
   900000: 480 RAM bytes used; 353626848 FST bytes; 172703151 nodes; took 245 seconds
   1000000: 480 RAM bytes used; 393006717 FST bytes; 191892620 nodes; took 277 seconds
   1100000: 480 RAM bytes used; 432387052 FST bytes; 211082115 nodes; took 311 seconds
   1200000: 480 RAM bytes used; 471766692 FST bytes; 230271461 nodes; took 334 seconds
   1300000: 480 RAM bytes used; 511147081 FST bytes; 249461034 nodes; took 357 seconds
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1400638946


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -153,6 +176,34 @@ private FSTCompiler(
     }
   }
 
+  // Get the respective FSTReader of the DataOutput. If the DataOutput is also a FSTReader then we
+  // will use it. Otherwise, we will use NullFSTReader, which does not allow reading.
+  private FSTReader toFSTReader(DataOutput dataOutput) {
+    if (dataOutput instanceof FSTReader) {
+      return (FSTReader) dataOutput;
+    }
+    return new NullFSTReader();

Review Comment:
   We still need to pass a `FSTReader` to the FST constructor. This is for when using a non-FSTReader DataOutput, like IndexOutput. There are 2 ways:
   - Using a `NullFSTReader` like this.
   - Pass actual null and do null-check/throw on FST itself.
   
   I think it might be preferable to do the first, as we can still throw exception? I do throw on `writeTo()`. I could throw in `getReverseBytesReader()` as well.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1400638946


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -153,6 +176,34 @@ private FSTCompiler(
     }
   }
 
+  // Get the respective FSTReader of the DataOutput. If the DataOutput is also a FSTReader then we
+  // will use it. Otherwise, we will use NullFSTReader, which does not allow reading.
+  private FSTReader toFSTReader(DataOutput dataOutput) {
+    if (dataOutput instanceof FSTReader) {
+      return (FSTReader) dataOutput;
+    }
+    return new NullFSTReader();

Review Comment:
   We still need to pass a `FSTReader` to the FST constructor. This is for when using a non-FSTReader DataOutput, like IndexOutput. There are 2 ways:
   - Using a `NullFSTReader` like this.
   - Pass actual null and do null-check/throw on FST itself.
   
   Ok let me try the second approach



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1400687529


##########
lucene/core/src/java/org/apache/lucene/util/fst/ByteBuffersFSTReader.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/** An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader} */
+final class ByteBuffersFSTReader extends DataOutput implements FSTReader {

Review Comment:
   I could rename it as `ByteBuffersDataOutputFSTReaderAdapter`. A bit long but it was only a package-private that is not visible outside, so it might be okay.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1395333947


##########
lucene/core/src/java/org/apache/lucene/util/fst/GrowableByteArrayDataOutput.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
+
+// Storing a byte[] for the current node of the FST we are writing. The byte[] will only grow, never
+// shrink.
+final class GrowableByteArrayDataOutput extends DataOutput implements Accountable {

Review Comment:
   This seems to be much cleaner than the previous BytesStore, and does not have any FST-specific operations. @mikemccand do you think this can be opened up, maybe moving it to `org.apache.lucene.store`?
   
   I'm fine with leaving it as package-private as well.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1400633005


##########
lucene/core/src/java/org/apache/lucene/util/fst/GrowableByteArrayDataOutput.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
+
+// Storing a byte[] for the current node of the FST we are writing. The byte[] will only grow, never
+// shrink.
+final class GrowableByteArrayDataOutput extends DataOutput implements Accountable {
+
+  private static final long BASE_RAM_BYTES_USED =
+      RamUsageEstimator.shallowSizeOfInstance(GrowableByteArrayDataOutput.class);
+
+  private static final int INITIAL_SIZE = 1 << 8;
+
+  // holds an initial size of 256 bytes. this byte array will only grow, but not shrink
+  private byte[] bytes = new byte[INITIAL_SIZE];
+
+  private int nextWrite;
+
+  @Override
+  public void writeByte(byte b) {
+    ensureCapacity(1);
+    bytes[nextWrite++] = b;
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int len) {
+    ensureCapacity(len);
+    System.arraycopy(b, offset, bytes, nextWrite, len);
+    nextWrite += len;
+  }
+
+  public int getPosition() {
+    return nextWrite;
+  }
+
+  public byte[] getBytes() {
+    return bytes;
+  }
+
+  /** Set the position of the byte[], increasing the capacity if needed */
+  public void setPosition(int newLen) {
+    assert newLen >= 0;
+    if (newLen > nextWrite) {
+      ensureCapacity(newLen - nextWrite);
+    }
+    nextWrite = newLen;

Review Comment:
   Yeah I think don't need to shrink for now. It would also reduce the GC load and the times we need to grow the array (comparing to when we shrink and expand the `byte[]` as it goes). I ran with a custom dictionary of size ~1MB and the maximum node is <256 bytes. For text-based dictionary I think it would depend on the length of the word (which is usually short) and the output (which depends). Large dictionary which has larger node addresses would also require more RAM, which is also ideal for us.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1391247632


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -337,11 +349,23 @@ public long size() {
     return getPosition();
   }
 
+  /** Similar to {@link #truncate(long)} with newLen=0 but keep the first block to reduce GC. */
+  public void reset() {

Review Comment:
   The public API of `ByteBuffersDataOutput` class seems to only allow appending only, but for scratch area we still need to some backward writing right?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1391272833


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -359,7 +383,9 @@ public void truncate(long newLen) {
     assert newLen == getPosition();
   }
 
-  public void finish() {
+  @Override
+  public void freeze() {
+    this.frozen = true;

Review Comment:
   I removed the Freezeable as well as the freeze functionality. It was previously used in writeTo(DataOutput) but even then it's not really needed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1392710843


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,12 +21,13 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
-// TODO: merge with PagedBytes, except PagedBytes doesn't
-// let you read while writing which FST needs
+// TODO: merge with or use PagedBytes/ByteBuffersDataOutput, as we no longer need to read while
+// writing
 
-class BytesStore extends DataOutput implements FSTReader {
+class BytesStore extends DataOutput implements Accountable {

Review Comment:
   Do you mean essentially use byte[] instead of byte block? I think that would simpler the writing logic (I was hesitated as it means we would need double RAM when resizing, but not too bad as it's only one node)
   
   But we still need to have the same functionality as the current BytesStore right? I feel like if we just remove the BytesStore down right, it's just moving the complexity from BytesStore to FSTCompiler, and I think having low-level writing in another class is better.
   
   How's about we changing the BytesStore to use byte[] instead of byte block?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1402058230


##########
lucene/core/src/java/org/apache/lucene/util/fst/FST.java:
##########
@@ -435,6 +433,13 @@ public FST(FSTMetadata<T> metadata, DataInput in, Outputs<T> outputs, FSTStore f
     this.fstReader = fstReader;
   }
 
+  /**
+   * @return true if and only if this FST is readable (e.g has a reverse BytesReader)
+   */
+  public boolean isReadable() {

Review Comment:
   Yeah, I was thinking as we will remove it anyway, maybe we don't need this, along with the null check (as they seems to be throw-away code). If users try to use a non-readable FST now, NPE will thrown instead (of IllegalStateExeption). And then when we change FSTCompiler to return FSTMetadata, they can't even create a non-readable FST. What do you think?
   
   Alternatively, we can double back to `NullFSTReader`, but let it throw exception like this:
   
   ```
   private static final class NullFSTReader implements FSTReader {
   
       @Override
       public long ramBytesUsed() {
         return 0;
       }
   
       @Override
       public FST.BytesReader getReverseBytesReader() {
         throw new UnsupportedOperationException("NullFSTReader does not support getReverseBytesReader()");
       }
   
       @Override
       public void writeTo(DataOutput out) {
         throw new UnsupportedOperationException("NullFSTReader does not support writeTo(DataOutput)");
       }
     }
   ```
   
   The benefits are:
   - We can enforce `FSTReader` to be non-null.
   - Remove all null-check.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1373993909


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -287,9 +315,9 @@ public long getMappedStateCount() {
     return dedupHash == null ? 0 : nodeCount;
   }
 
-  private CompiledNode compileNode(UnCompiledNode<T> nodeIn, int tailLength) throws IOException {

Review Comment:
   This `tailLength` is not being used anywhere, hence removed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1350425577


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,19 +21,18 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements Accountable {
+class BytesStore extends FSTWriter {

Review Comment:
   Besides `reverse` most of the methods here also needs to write/update arbitrarily previously written bytes (within the current frontier input), such as writeBytes/writeByte/copyBytes/truncate.
   
   Moreover, there is also methods which requires reading of previously written bytes that can't simply fit in the DataOutput (getReverseBytesReader e.g).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1353826324


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,19 +21,18 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements Accountable {
+class BytesStore extends FSTWriter {

Review Comment:
   > Once FST.addNode completes, those written bytes are never altered?
   
   More precisely, those bytes are never altered after `FSTCompiler.add` completes. It seems we need to write all nodes of the input in reverse as a whole. But yeah the BytesStore use the head of the buffer as scratch area.
   
   > Hmm getReverseBytesReader is indeed a problem. I wonder how the [Tantivy FST impl](https://blog.burntsushi.net/transducers/) deals with this?
   
   It seems Tantivy segregate the building and the traverse of FST as 2 different entity. The FST Builder will just write the FST to a DataOutput and not allow it to be read directly. I was thinking of this too, as currently we are mixing up the writing and reading:
   - Load a previously saved FST from a DataInput. This is read-only and is fine, and it's how Tantivy FST is created as well.
   - Construct a FST on-the-fly and use it right away. This is both read & write and it uses BytesStore.
   
   I'm kind of favoring the way Tantivy is doing, it's cleaner and more "facade pattern". Maybe we could first refactor so that the FST created on the fly will be written directly to a DataOutput, and then instead of using it directly, we construct a FST from that DataOutput?
   
   If we are doing that, then we can get rid of the `getReverseBytesReader`. However one issue remains: we still need `getReverseBytesReaderForSuffixSharing` for NodeHash. Or at least some operation for random-access. I think Tantivy is using LRU cache for this in write-through mode: write the node into both the DataOutput and the LRU at the same time. This means we don't even need to read from the DataOutput, but it won't be perfectly minimal (as there will be false-negative cache-miss). I understand that there is the trade-off, but we might also need to support the current minimalist mode.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1392603399


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,12 +21,13 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
-// TODO: merge with PagedBytes, except PagedBytes doesn't
-// let you read while writing which FST needs
+// TODO: merge with or use PagedBytes/ByteBuffersDataOutput, as we no longer need to read while
+// writing
 
-class BytesStore extends DataOutput implements FSTReader {
+class BytesStore extends DataOutput implements Accountable {

Review Comment:
   > The 2nd and 3rd use case has been moved to use ByteBuffersDataOutput, so it is only used as the scratch area writer now.
   
   Yay, progress :)
   
   > * Act as a scratch area writer with some complex backward writing operations
   
   Hmm where do we do this?  I guess it is where we do in-place fixing of arc encoding after we've decided whether it should be sparse/dense/contiguous?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1400663661


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -827,22 +910,24 @@ void setEmptyOutput(T v) {
   }
 
   void finish(long newStartNode) {
-    assert newStartNode <= bytes.size();
+    assert newStartNode <= numBytesWritten;
     if (fst.metadata.startNode != -1) {
       throw new IllegalStateException("already finished");
     }
     if (newStartNode == FINAL_END_NODE && fst.metadata.emptyOutput != null) {
       newStartNode = 0;
     }
     fst.metadata.startNode = newStartNode;
-    fst.metadata.numBytes = bytes.getPosition();
+    fst.metadata.numBytes = numBytesWritten;
   }
 
   private boolean validOutput(T output) {
     return output == NO_OUTPUT || !output.equals(NO_OUTPUT);
   }
 
   /** Returns final FST. NOTE: this will return null if nothing is accepted by the FST. */
+  // TODO: make this method to only return the FSTMetadata and user needs to construct the FST
+  // themselves
   public FST<T> compile() throws IOException {

Review Comment:
   Currently if we try to read from it, the reader would return null (and be changed to throw an IllegalStateException). If we change this method to just return the FSTMetadata the user will be mindful that the FST might not be readable. But yeah it will touch a lot of codes.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1400638946


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -153,6 +176,34 @@ private FSTCompiler(
     }
   }
 
+  // Get the respective FSTReader of the DataOutput. If the DataOutput is also a FSTReader then we
+  // will use it. Otherwise, we will use NullFSTReader, which does not allow reading.
+  private FSTReader toFSTReader(DataOutput dataOutput) {
+    if (dataOutput instanceof FSTReader) {
+      return (FSTReader) dataOutput;
+    }
+    return new NullFSTReader();

Review Comment:
   We still need to pass a `FSTReader` to the FST constructor. The `NullFSTReader` is for when using a non-FSTReader DataOutput, like IndexOutput. There are 2 ways:
   - Using a `NullFSTReader` like this.
   - Pass actual null and do null-check/throw on FST itself.
   
   Ok let me try the second approach



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1415627354


##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader, Freezable {
+
+  private final ByteBuffersDataOutput dataOutput;
+  // the DataInput to read from in case the DataOutput has multiple blocks
+  private ByteBuffersDataInput dataInput;
+  // the ByteBuffers to read from in case the DataOutput has a single block
+  private ByteBuffer byteBuffer;
+
+  public ReadWriteDataOutput(ByteBuffersDataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    dataOutput.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) {
+    dataOutput.writeBytes(b, offset, length);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return dataOutput.ramBytesUsed();
+  }
+
+  @Override
+  public void freeze() {
+    // these operations are costly, so we want to compute it once and cache
+    List<ByteBuffer> byteBuffers = dataOutput.toWriteableBufferList();
+    if (byteBuffers.size() == 1) {

Review Comment:
   I think it didn't handle the single buffer use case, and ByteBuffersDataInput would fall into the same performance regression problem as BytesStore with multiple blocks (which due to the fact that the method is size is larger and won't be inlined by JVM).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1415666711


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -218,13 +279,19 @@ public Builder<T> allowFixedLengthArcs(boolean allowFixedLengthArcs) {
     }
 
     /**
-     * How many bits wide to make each byte[] block in the BytesStore; if you know the FST will be
-     * large then make this larger. For example 15 bits = 32768 byte pages.
+     * Set the {@link DataOutput} which is used for low-level writing of FST. If you want the FST to
+     * be immediately readable, you need to use a DataOutput that also implements {@link FSTReader},

Review Comment:
   I think when we change the compile() to only return metadata, users need to create the FST with the on-heap FSTReader, and thus it needs to be public.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1845106275

   Since we are struggling to best measure FST performance impact of these changes, I opened a spinoff [issue to create a dedicated FST microbenchmark](https://github.com/apache/lucene/pull/12624/files).  Having such a compass that's quick to check would really help us make more informed decisions on complex "code complexity vs performance" types of FST changes.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1418746164


##########
lucene/core/src/java/org/apache/lucene/util/fst/FST.java:
##########
@@ -419,6 +417,8 @@ public FST(FSTMetadata<T> metadata, DataInput in, FSTStore fstStore) throws IOEx
 
   /** Create the FST with a metadata object and a FSTReader. */
   FST(FSTMetadata<T> metadata, FSTReader fstReader) {
+    assert metadata != null;

Review Comment:
   Can we make these real `if`?  User might hit this if they pass `null` right?  We try to use `assert` when only a weird bug in our code might cause something.  When it's a real thing that could happen due to user actions, we try to use real `if`.  You could just do `this.metadata = Objects.requireNonNull(metadata);`?



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -248,12 +327,16 @@ public Builder<T> directAddressingMaxOversizingFactor(float factor) {
 
     /** Creates a new {@link FSTCompiler}. */
     public FSTCompiler<T> build() throws IOException {
+      // create a default DataOutput if not specified
+      if (dataOutput == null) {

Review Comment:
   Thanks :)



##########
lucene/core/src/test/org/apache/lucene/util/fst/Test2BFSTOffHeap.java:
##########
@@ -0,0 +1,342 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.MMapDirectory;
+import org.apache.lucene.tests.util.LuceneTestCase;
+import org.apache.lucene.tests.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.lucene.tests.util.TimeUnits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
+import org.junit.Ignore;
+
+// Similar to Test2BFST but will build and read the FST off-heap and can be run with small heap
+
+// Run something like this:
+//    ./gradlew test --tests Test2BFSTOffHeap -Dtests.verbose=true --max-workers=1

Review Comment:
   Yay, no massive heap needed!!  So sweet.



##########
lucene/test-framework/src/java/org/apache/lucene/tests/util/fst/FSTTester.java:
##########
@@ -255,9 +255,16 @@ private T randomAcceptedWord(FST<T> fst, IntsRefBuilder in) throws IOException {
 
   public FST<T> doTest() throws IOException {
 
+    IndexOutput indexOutput = null;
+    boolean useOffHeap = random.nextBoolean();
+    if (useOffHeap) {
+      indexOutput = dir.createOutput("fstOffHeap.bin", IOContext.DEFAULT);
+    }
+
     final FSTCompiler<T> fstCompiler =
         new FSTCompiler.Builder<>(
                 inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, outputs)
+            .dataOutput(indexOutput)

Review Comment:
   Hmm -- I don't think this Builder method (`.dataOutput`) should allow `null`?  And then we should conditionalize this call to `if (useOffHeap) ...`?



##########
lucene/test-framework/src/java/org/apache/lucene/tests/util/fst/FSTTester.java:
##########
@@ -274,8 +281,16 @@ public FST<T> doTest() throws IOException {
       }
     }
     FST<T> fst = fstCompiler.compile();
+    ;
 
-    if (random.nextBoolean() && fst != null) {
+    if (useOffHeap) {
+      indexOutput.close();
+      try (IndexInput in = dir.openInput("fstOffHeap.bin", IOContext.DEFAULT)) {
+        fst = new FST<>(fst.getMetadata(), in);
+      } finally {
+        dir.deleteFile("fstOffHeap.bin");

Review Comment:
   Whoa, nice usage of "delete on final close" semantics!



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -153,6 +192,41 @@ private FSTCompiler(
     }
   }
 
+  // Get the respective FSTReader of the DataOutput. If the DataOutput is also a FSTReader then we
+  // will use it, otherwise we will return a NullFSTReader. Attempting to read from a FST with
+  // NullFSTReader
+  // will throw UnsupportedOperationException

Review Comment:
   Merge the above two lines?  Annoying that spotless does not actually tidy up on itself, rather than brutally chopping up text to fit within the styling requirements!



##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader {
+
+  private final ByteBuffersDataOutput dataOutput;
+  // the DataInput to read from once we finish writing
+  private ByteBuffersDataInput dataInput;
+  // whether this DataOutput is already frozen
+  private boolean frozen;
+
+  public ReadWriteDataOutput(ByteBuffersDataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    if (frozen) {
+      throw new IllegalStateException("Already frozen");
+    }
+    dataOutput.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) {
+    if (frozen) {

Review Comment:
   I think these could be `assert` instead?  User should not be able to do anything to trigger this exception?  Only a bug in our code could...?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1407526003


##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader, Freezable {
+
+  private final ByteBuffersDataOutput dataOutput;
+  private ByteBuffersDataInput dataInput;
+  private List<ByteBuffer> byteBuffers;
+
+  public ReadWriteDataOutput(ByteBuffersDataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    dataOutput.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) {
+    dataOutput.writeBytes(b, offset, length);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return dataOutput.ramBytesUsed();
+  }
+
+  @Override
+  public void freeze() {
+    // these operations are costly, so we want to compute it once and cache
+    byteBuffers = dataOutput.toWriteableBufferList();

Review Comment:
   I tried several ways to optimize the `getReverseBytesReader()` method, such as bypassing all `asReadOnlyBuffer()` and accessing the byte array directly, but they wouldn't come close compared to BytesStore:
   - The `ByteBuffersDataOutput.blocks` is a Deque and thus not random accessible. We need to copy to a list, and this takes time as the number of blocks grow.
   - It took ~15 seconds to verify (`Util.get()`) 100K times, compared to only ~2 seconds with BytesStore.
   
   Hence I had to re-add `Freezable()` here, so that the costly operations will only be called once. The alternative way is to cache, but `getReverseBytesReader()` can be called concurrently hence we need some kind of ConcurrentHashMap with only 2 items, which is a weird way to cache.
   
   Another way that is not so efficient as Freezable, but avoid the added interface is to simply use a volatile variable and have a simple null check (the volatile is there to enforce a write barrier).
   
   ```
   private volatile ByteBuffersDataInput dataInput;
   private volatile List<ByteBuffer> byteBuffers;
   
   public FST.BytesReader getReverseBytesReader() {
       if (byteBuffers == null) {
           byteBuffers = dataOutput.toWriteableBufferList();
           dataInput = new ByteBuffersDataInput(byteBuffers);
       }
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1360701669


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,19 +21,18 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements Accountable {
+class BytesStore extends FSTWriter {

Review Comment:
   > I'm kind of favoring the way Tantivy is doing, it's cleaner and more "facade pattern". Maybe we could first refactor so that the FST created on the fly will be written directly to a DataOutput, and then instead of using it directly, we construct a FST from that DataOutput? From the Builder point-of-view it will still create the FST eventually, but could pave the way for segregation of building & reading later (i.e Builder will only write to the DataOutput and it's up to the users to create the corresponding DataInput and construct a FST from that).
   
   +1 -- I like this approach.  Let's not special case the "build FST in memory and immediately use it from memory" case.  Let the OS cache the bytes we write and then keep them hot (in RAM) when we immediately open it for reading.  That's a nice standalone simplification.
   
   > UPDATE: The above seems to be a breaking change and might not be a small, incremental one, as NodeHash needs the FST to be initialized first so that it can search over the previously written nodes.
   
   It indeed reads from the still-writing FST, but only to confirm whether a newly created node is already in the `NodeHash`.  `NodeHash` is really a `Set<Node>`.  Normally a Set would hold pointers to java objects as its values, but in our case, instead of holding a true object pointer, we hold a pointer back into the FST since the FST has the node already.  We could consider changing that so that `NodeHash` holds a copy of the full node itself (a `byte[]` or a slice into a shared `byte[]`).  This is a bit wasteful in RAM, but it'd fully decouple `NodeHash` from the growing / appending FST's `byte[]`?
   
   > If we are doing that, then we can get rid of the getReverseBytesReader. However one issue remains: we still need getReverseBytesReaderForSuffixSharing for NodeHash. Or at least some operation for random-access. I think Tantivy is using LRU cache for this in write-through mode: write the node into both the DataOutput and the LRU at the same time. This means we don't even need to read from the DataOutput, but it won't be perfectly minimal (as there will be false-negative cache-miss). I understand that there is the trade-off, but we might also need to support the current minimalist mode.
   
   Aha!  It sounds like Tantivy indeed does store-by-value in its LRU suffix cache, not store-by-reference like Lucene does today?
   
   Yet another option would be to consider allowing a still open `IndexOutput` to be read by an `IndexInput`?  This would be a change in the semantics of our IO abstractions (`Directory` and its babies), and likely problematic for some filesystems (HDFS?), but for most common filesystems would be easily supported.  Then we wouldn't have to duplicate the nodes in the `NodeHash` and in the FST.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1360715823


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,19 +21,18 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements Accountable {
+class BytesStore extends FSTWriter {

Review Comment:
   Actually, while it may seem RAM-wasteful to fully copy the `byte[]` node that are stored in `NodeHash`, if it then (eventually?) enables append-only writing of the FST, net/net that will likely be a RAM savings overall, once we finish with the [LRU NodeHash PR](https://github.com/apache/lucene/pull/12633).  So I think that PR is a precursor to this one?  This already long pole is growing longer!  Like Pinocchio's nose:
   
   ![image](https://github.com/apache/lucene/assets/796508/f473ca7f-da8d-4270-aa85-8957ee2efba4)
   
   I'll try to make time to wrap that PR up soon.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1393547261


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,12 +21,13 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
-// TODO: merge with PagedBytes, except PagedBytes doesn't
-// let you read while writing which FST needs
+// TODO: merge with or use PagedBytes/ByteBuffersDataOutput, as we no longer need to read while
+// writing
 
-class BytesStore extends DataOutput implements FSTReader {
+class BytesStore extends DataOutput implements Accountable {

Review Comment:
   > Hmm where do we do this
   
   TBH I haven't gone through all of them, but one instance is when we write the arc in backward, another is when we first skip the header and after all is written we go back and write the header, presence bits and label.
   
   Anyhow I changed to a byte[] to simplify its implementation.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1392585398


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -120,31 +122,54 @@ public class FSTCompiler<T> {
   final float directAddressingMaxOversizingFactor;
   long directAddressingExpansionCredit;
 
-  final BytesStore bytes;
+  // writer for frozen nodes
+  final FSTDataOutputWriter fstWriter;
+  // reader for the frozen nodes
+  final FSTReader fstReader;
+
+  // buffer to store the scratch bytes before writing to the fstWriter
+  final BytesStore bytes = new BytesStore(DEFAULT_SCRATCH_PAGE_BITS);
 
   /**
    * Instantiates an FST/FSA builder with default settings and pruning options turned off. For more
    * tuning and tweaking, see {@link Builder}.
    */
   // TODO: remove this?  Builder API should be the only entry point?
   public FSTCompiler(FST.INPUT_TYPE inputType, Outputs<T> outputs) {
-    this(inputType, 32.0, outputs, true, 15, 1f);
+    this(
+        inputType,
+        32.0,
+        outputs,
+        true,
+        new FSTDataOutputWriter(getLegacyDataOutput(DEFAULT_BLOCK_BITS)),
+        1f);
+  }
+
+  static DataOutput getLegacyDataOutput(int blockBits) {
+    return new BytesStore(blockBits);
   }
 
   private FSTCompiler(
       FST.INPUT_TYPE inputType,
       double suffixRAMLimitMB,
       Outputs<T> outputs,
       boolean allowFixedLengthArcs,
-      int bytesPageBits,
+      FSTDataOutputWriter fstWriter,
       float directAddressingMaxOversizingFactor) {
     this.allowFixedLengthArcs = allowFixedLengthArcs;
     this.directAddressingMaxOversizingFactor = directAddressingMaxOversizingFactor;
-    bytes = new BytesStore(bytesPageBits);
     // pad: ensure no node gets address 0 which is reserved to mean
     // the stop state w/ no arcs
-    bytes.writeByte((byte) 0);
-    fst = new FST<>(new FST.FSTMetadata<>(inputType, null, -1, VERSION_CURRENT, 0), outputs, bytes);
+    try {
+      fstWriter.writeByte((byte) 0);
+    } catch (IOException e) {

Review Comment:
   OK I just merged #12715.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1391146752


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -26,7 +26,8 @@
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements FSTReader {
+// TODO: Separate the scratch writer and reader functionality
+class BytesStore extends DataOutput implements FSTReader, Freezeable {

Review Comment:
   Hm, I'm wondering why should we let the FSTCompiler to know about the RandomAccessInput? In the most case, we would let the FSTCompiler to just write to a DataOutput, then users are to read it with the public FST constructor right?
   
   The case where we want to read from the FST immediately after writing is the one with BytesStore.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1392201110


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,12 +21,13 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
-// TODO: merge with PagedBytes, except PagedBytes doesn't
-// let you read while writing which FST needs
+// TODO: merge with or use PagedBytes/ByteBuffersDataOutput, as we no longer need to read while
+// writing
 
-class BytesStore extends DataOutput implements FSTReader {
+class BytesStore extends DataOutput implements Accountable {

Review Comment:
   This class originally has 3 purposes:
   - Act as a scratch area writer with complex backward writing operations
   - Acts as a DataOutput to write the FST frozen nodes
   - Acts as a FSTReader to read from the FST once constructed
   
   The 2nd and 3rd use case has been moved to use ByteBuffersDataOutput, so it is only used as the scratch area writer now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1384518777


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTWriter.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+
+/** Abstract class which provides low-level functionality to write to a FST */
+public interface FSTWriter {
+
+  /**

Review Comment:
   These 2 methods are same as DataOutput, but DataOutput is a class which we want to avoid extending from.



##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -26,7 +27,8 @@
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements FSTReader {
+// TODO: Separate the scratch writer and reader functionality
+class BytesStore extends DataOutput implements FSTReader, Closeable {

Review Comment:
   I was thinking of changing this to something like Freezable or Finishable



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1392598634


##########
lucene/core/src/java/org/apache/lucene/util/fst/ByteBuffersFSTReader.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/** An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader} */
+final class ByteBuffersFSTReader extends DataOutput implements FSTReader {

Review Comment:
   Yeah +1 we should not have `ByteBuffersDataOutput` implement `FSTReader`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1392595332


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -337,11 +349,23 @@ public long size() {
     return getPosition();
   }
 
+  /** Similar to {@link #truncate(long)} with newLen=0 but keep the first block to reduce GC. */
+  public void reset() {

Review Comment:
   Can't we turn it into a `byte[]` (`.toArrayCopy()`) and do our own reversal?  These `byte[]` are small, just a single node right?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1395069004


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -337,11 +349,23 @@ public long size() {
     return getPosition();
   }
 
+  /** Similar to {@link #truncate(long)} with newLen=0 but keep the first block to reduce GC. */
+  public void reset() {

Review Comment:
   Yeah it seems the main complexity is due to the fact that we need to override the previously written bytes, sometimes when we want to remove the labels, sometimes when we need to update the header after writing everything. So I guess making it forward would not eliminate those backward writing operation.
   
   I simplified the writing operation a bit in the direct addressing mode. Seems like we can convert it to append-only mode. But the binary search one seems to be difficult, as when we fix the arc we need to do in backward (I assume to avoid overriding?). Anyhow, the remaining backward operations are:
   - writeBytes: As mentioned, this is used in the binary search mode. If we first copy the whole buffer to a new one, we can start appending back to back. But this isn't as efficient?
   - reverse: This is only done at the end before writing to the DataOutput and the NodeHash.ByteBlockPool



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dweiss (via GitHub)" <gi...@apache.org>.
dweiss commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1820305577

   Package-private is fine, I think. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1400641544


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -248,15 +305,17 @@ public Builder<T> directAddressingMaxOversizingFactor(float factor) {
 
     /** Creates a new {@link FSTCompiler}. */
     public FSTCompiler<T> build() {
-      FSTCompiler<T> fstCompiler =
-          new FSTCompiler<>(
-              inputType,
-              suffixRAMLimitMB,
-              outputs,
-              allowFixedLengthArcs,
-              bytesPageBits,
-              directAddressingMaxOversizingFactor);
-      return fstCompiler;
+      try {
+        return new FSTCompiler<>(
+            inputType,
+            suffixRAMLimitMB,
+            outputs,
+            allowFixedLengthArcs,
+            dataOutput,
+            directAddressingMaxOversizingFactor);
+      } catch (IOException e) {
+        throw new RuntimeException(e);

Review Comment:
   Can we defer this? I could add a TODO and do a follow-on PR. The reason is that it will cause even more diffs and not directly related to this PR main purpose.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1350191783


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,19 +21,18 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements Accountable {
+class BytesStore extends FSTWriter {

Review Comment:
   I'm a little nervous about this approach -- I was hoping to reduce the storage-like abstractions here, by sticking with `DataOutput` and showing how it could be backed by an `IndexOutput` that streams straight to disk.  But this PR is adding yet another storage abstraction :)
   
   Maybe a simpler first baby step would be to eliminate APIs in `BytesStore` that an `IndexOutput` would not easily support, e.g. this sneaky `reverse` method?  It is only called in one place, to reverse just written bytes, so that place could/should just as easily buffer up its own `byte[]` in a scratch/reused array, reverse that in place before writing, then write it?  Then we can remove this `reverse` method and `BytesStore` is closer to just a `DataOutput`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1828548480

   Hmm I'm running `Test2BFSTs` on this patch and noticed it seems to take very much longer during the `TEST: now verify` step where it confirms the built FST accepts all the inputs it just compiled into it.
   
   I `jstack`'d it and found this:
   
   ```
      java.lang.Thread.State: RUNNABLE
           at java.nio.ByteBuffer.position(java.base@17.0.9/ByteBuffer.java:1516)
           at java.nio.ByteBuffer.position(java.base@17.0.9/ByteBuffer.java:267)
           at java.nio.Buffer.<init>(java.base@17.0.9/Buffer.java:246)
           at java.nio.ByteBuffer.<init>(java.base@17.0.9/ByteBuffer.java:288)
           at java.nio.HeapByteBuffer.<init>(java.base@17.0.9/HeapByteBuffer.java:95)
           at java.nio.HeapByteBufferR.<init>(java.base@17.0.9/HeapByteBufferR.java:102)
           at java.nio.HeapByteBufferR.duplicate(java.base@17.0.9/HeapByteBufferR.java:135)
           at java.nio.HeapByteBufferR.asReadOnlyBuffer(java.base@17.0.9/HeapByteBufferR.java:148)
           at org.apache.lucene.store.ByteBuffersDataInput.<init>(ByteBuffersDataInput.java:60)
           at org.apache.lucene.store.ByteBuffersDataOutput.toDataInput(ByteBuffersDataOutput.java:279)
           at org.apache.lucene.util.fst.ReadWriteDataOutput.getReverseBytesReader(ReadWriteDataOutput.java:52)
           at org.apache.lucene.util.fst.FST.getBytesReader(FST.java:1181)
           at org.apache.lucene.util.fst.Util.get(Util.java:50)
           at org.apache.lucene.util.fst.Test2BFST.test(Test2BFST.java:113)
   ```
   
   It looks like getting the reversed reader has maybe become quite a bit more expensive than `BytesStore` was?
   
   Note that `Test2BFSTs` is quite silly -- it uses `Util.get` for every lookup, instead of pulling and reusing a `ReverseBytesReader` like most "real" usages of FST (e.g. terms dict) will do.  So perhaps this performance change doesn't really matter in practice?  Though I wonder if all consumers of FST are re-using their reversed readers?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1407526003


##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader, Freezable {
+
+  private final ByteBuffersDataOutput dataOutput;
+  private ByteBuffersDataInput dataInput;
+  private List<ByteBuffer> byteBuffers;
+
+  public ReadWriteDataOutput(ByteBuffersDataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    dataOutput.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) {
+    dataOutput.writeBytes(b, offset, length);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return dataOutput.ramBytesUsed();
+  }
+
+  @Override
+  public void freeze() {
+    // these operations are costly, so we want to compute it once and cache
+    byteBuffers = dataOutput.toWriteableBufferList();

Review Comment:
   I tried several ways to optimize the `getReverseBytesReader()` method, such as bypassing all `asReadOnlyBuffer()` and accessing the byte array directly, but they wouldn't come close compared to BytesStore:
   - The `ByteBuffersDataOutput.blocks` is a Deque and thus not random accessible. We need to copy to a list, and this takes time as the number of blocks grow.
   - It took ~15 seconds to verify (`Util.get()`) 100K times, compared to only ~2 seconds with BytesStore.
   - I also tried to use PagedBytes, but PagedBytes requires a call to freeze() before getting the DataInput anyway.
   
   Hence I had to re-add `Freezable()` here, so that the costly operations will only be called once. The alternative way is to cache, but `getReverseBytesReader()` can be called concurrently hence we need some kind of ConcurrentHashMap with only 2 items, which is a weird way to cache.
   
   Another way that is not so efficient as Freezable, but avoid the added interface is to simply use a volatile variable and have a simple null check (the volatile is there to enforce a write barrier).
   
   ```
   private volatile ByteBuffersDataInput dataInput;
   private volatile List<ByteBuffer> byteBuffers;
   
   public FST.BytesReader getReverseBytesReader() {
       if (byteBuffers == null) {
           byteBuffers = dataOutput.toWriteableBufferList();
       }
       if (dataInput == null) {
           dataInput = new ByteBuffersDataInput(byteBuffers);
       }
       // use the variables
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1418734995


##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader, Freezable {
+
+  private final ByteBuffersDataOutput dataOutput;
+  // the DataInput to read from in case the DataOutput has multiple blocks
+  private ByteBuffersDataInput dataInput;
+  // the ByteBuffers to read from in case the DataOutput has a single block
+  private ByteBuffer byteBuffer;
+
+  public ReadWriteDataOutput(ByteBuffersDataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    dataOutput.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) {
+    dataOutput.writeBytes(b, offset, length);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return dataOutput.ramBytesUsed();
+  }
+
+  @Override
+  public void freeze() {
+    // these operations are costly, so we want to compute it once and cache
+    List<ByteBuffer> byteBuffers = dataOutput.toWriteableBufferList();
+    if (byteBuffers.size() == 1) {

Review Comment:
   +1 to sticking to the "clean but slower" impl for this PR, and seeing if we can / it's worth clawing back the performance cost/overhead of `ByteBuffer`s  in a separate follow-on PR.  Maybe @uschindler has some ideas on recovering performance here too ...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dweiss (via GitHub)" <gi...@apache.org>.
dweiss commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1416181846


##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader, Freezable {
+
+  private final ByteBuffersDataOutput dataOutput;
+  // the DataInput to read from in case the DataOutput has multiple blocks
+  private ByteBuffersDataInput dataInput;
+  // the ByteBuffers to read from in case the DataOutput has a single block
+  private ByteBuffer byteBuffer;
+
+  public ReadWriteDataOutput(ByteBuffersDataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    dataOutput.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) {
+    dataOutput.writeBytes(b, offset, length);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return dataOutput.ramBytesUsed();
+  }
+
+  @Override
+  public void freeze() {
+    // these operations are costly, so we want to compute it once and cache
+    List<ByteBuffer> byteBuffers = dataOutput.toWriteableBufferList();
+    if (byteBuffers.size() == 1) {

Review Comment:
   I don't think it has anything to do with method sizes. I think it's got more to do with ByteBuffersDataInput wrapping input buffers with asReadOnlyBuffer:
   ```
     public ByteBuffersDataInput(List<ByteBuffer> buffers) {
       ensureAssumptions(buffers);
   
       this.blocks = buffers.toArray(ByteBuffer[]::new);
       for (int i = 0; i < blocks.length; ++i) {
         blocks[i] = blocks[i].asReadOnlyBuffer().order(ByteOrder.LITTLE_ENDIAN);
       }
   ```
   It also eagerly preallocates arrays for float and long buffers - something that was added later (I wasn't aware), which may also contribute to performance if you call this method millions of times. But this is only my gut feeling, it'd have to be verified with a profile run. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1416403950


##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader, Freezable {
+
+  private final ByteBuffersDataOutput dataOutput;
+  // the DataInput to read from in case the DataOutput has multiple blocks
+  private ByteBuffersDataInput dataInput;
+  // the ByteBuffers to read from in case the DataOutput has a single block
+  private ByteBuffer byteBuffer;
+
+  public ReadWriteDataOutput(ByteBuffersDataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    dataOutput.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) {
+    dataOutput.writeBytes(b, offset, length);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return dataOutput.ramBytesUsed();
+  }
+
+  @Override
+  public void freeze() {
+    // these operations are costly, so we want to compute it once and cache
+    List<ByteBuffer> byteBuffers = dataOutput.toWriteableBufferList();
+    if (byteBuffers.size() == 1) {

Review Comment:
   Even for ByteBuffersDataInput in multi-block cases, there's a slight regression over the BytesStore implementation (note that we already called `freeze()`). The diff is https://github.com/dungba88/lucene/pull/13/files.
   
   ### ReverseRandomBytesReader with ByteBuffersDataInput
   
   ```
     1> TEST: now verify [fst size=4621076364; nodeCount=2252341486; arcCount=2264078585]
     1> 0...: took 0 seconds
     1> 1000000...: took 27 seconds
     1> 2000000...: took 54 seconds
     1> 3000000...: took 82 seconds
     1> 4000000...: took 109 seconds
     1> 5000000...: took 137 seconds
     1> 6000000...: took 165 seconds
     1> 7000000...: took 192 seconds
     1> 8000000...: took 219 seconds
     1> 9000000...: took 247 seconds
     1> 10000000...: took 275 seconds
     1> 11000000...: took 300 seconds
   ```
   
   ### BytesStore-like BytesReader
   
   ```
     1> TEST: now verify [fst size=4621076364; nodeCount=2252341486; arcCount=2264078585]
     1> 0...: took 0 seconds
     1> 1000000...: took 22 seconds
     1> 2000000...: took 44 seconds
     1> 3000000...: took 66 seconds
     1> 4000000...: took 89 seconds
     1> 5000000...: took 111 seconds
     1> 6000000...: took 133 seconds
     1> 7000000...: took 155 seconds
     1> 8000000...: took 178 seconds
     1> 9000000...: took 200 seconds
     1> 10000000...: took 222 seconds
     1> 11000000...: took 245 seconds
   ```
   
   As this PR is already rather long, I can make the first implementation simple by solely using ByteBuffersDataInput, then open an issue to benchmark different alternatives thoroughly. WDYT?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1395069004


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -337,11 +349,23 @@ public long size() {
     return getPosition();
   }
 
+  /** Similar to {@link #truncate(long)} with newLen=0 but keep the first block to reduce GC. */
+  public void reset() {

Review Comment:
   Yeah it seems the main complexity is due to the fact that we need to override the previously written bytes, sometimes when we want to remove the labels, sometimes when we need to update the header after writing everything. So I guess making it forward would not eliminate those backward writing operation.
   
   But as the implementation is much simpler now (with the growable byte array), would we still need to pursuit this?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1393462969


##########
lucene/core/src/java/org/apache/lucene/util/fst/OnHeapFSTStore.java:
##########
@@ -64,22 +66,13 @@ public FSTStore init(DataInput in, long numBytes) throws IOException {
     return this;
   }
 
-  @Override

Review Comment:
   This change is in a different PR: https://github.com/apache/lucene/pull/12802. Ideally we would want to merge that one first.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1391165022


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -26,7 +26,8 @@
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements FSTReader {
+// TODO: Separate the scratch writer and reader functionality
+class BytesStore extends DataOutput implements FSTReader, Freezeable {

Review Comment:
   > Could we remove this class entirely? And callers that want to write FST and immediately use it in RAM should just use ByteBuffersDataOutput for their scratch area?
   
   This BytesStore class currently serves as 3 purposes:
   - Acts as a scratch area writer. Their operations (copying random bytes from one place to other, truncating, skipping the bytes) seems to be complicated to model with other DataOutput
   - Acts as a FST writer. Any other DataOutput can be used.
   - Acts as a FST reader. This is tricky as DataOutput does not support read operation, and a separate DataInput needs to be provided.
   
   To replace the third one, I think we should keep both the second and third as a single class, because they have to match (a ByteBufferDataOutput needs to go with the ByteBufferRandomAccessInput). However the third purpose is only useful for write-then-read-immediately, which I assume is not the most common use case. Here we can either use the existing BytesStore, or create something based on ByteBufferDataOutput as you suggested. Both are easily replaceable in the future.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1393462969


##########
lucene/core/src/java/org/apache/lucene/util/fst/OnHeapFSTStore.java:
##########
@@ -64,22 +66,13 @@ public FSTStore init(DataInput in, long numBytes) throws IOException {
     return this;
   }
 
-  @Override

Review Comment:
   This change is in a different PR: https://github.com/apache/lucene/pull/12802. Ideally we would want to merge that one first.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1391183424


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -120,31 +122,54 @@ public class FSTCompiler<T> {
   final float directAddressingMaxOversizingFactor;
   long directAddressingExpansionCredit;
 
-  final BytesStore bytes;
+  // writer for frozen nodes
+  final FSTDataOutputWriter fstWriter;
+  // reader for the frozen nodes
+  final FSTReader fstReader;
+
+  // buffer to store the scratch bytes before writing to the fstWriter
+  final BytesStore bytes = new BytesStore(DEFAULT_SCRATCH_PAGE_BITS);
 
   /**
    * Instantiates an FST/FSA builder with default settings and pruning options turned off. For more
    * tuning and tweaking, see {@link Builder}.
    */
   // TODO: remove this?  Builder API should be the only entry point?
   public FSTCompiler(FST.INPUT_TYPE inputType, Outputs<T> outputs) {
-    this(inputType, 32.0, outputs, true, 15, 1f);
+    this(
+        inputType,
+        32.0,
+        outputs,
+        true,
+        new FSTDataOutputWriter(getLegacyDataOutput(DEFAULT_BLOCK_BITS)),
+        1f);
+  }
+
+  static DataOutput getLegacyDataOutput(int blockBits) {
+    return new BytesStore(blockBits);
   }
 
   private FSTCompiler(
       FST.INPUT_TYPE inputType,
       double suffixRAMLimitMB,
       Outputs<T> outputs,
       boolean allowFixedLengthArcs,
-      int bytesPageBits,
+      FSTDataOutputWriter fstWriter,
       float directAddressingMaxOversizingFactor) {
     this.allowFixedLengthArcs = allowFixedLengthArcs;
     this.directAddressingMaxOversizingFactor = directAddressingMaxOversizingFactor;
-    bytes = new BytesStore(bytesPageBits);
     // pad: ensure no node gets address 0 which is reserved to mean
     // the stop state w/ no arcs
-    bytes.writeByte((byte) 0);
-    fst = new FST<>(new FST.FSTMetadata<>(inputType, null, -1, VERSION_CURRENT, 0), outputs, bytes);
+    try {
+      fstWriter.writeByte((byte) 0);
+    } catch (IOException e) {

Review Comment:
   Can we do this in a follow-up PR? Having this constructor throwing exception means the public one also needs to throw exception, and that was used in a lot of places :)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1391146752


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -26,7 +26,8 @@
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements FSTReader {
+// TODO: Separate the scratch writer and reader functionality
+class BytesStore extends DataOutput implements FSTReader, Freezeable {

Review Comment:
   Hm, I'm wondering why should we let the FSTCompiler to know about the RandomAccessInput? In the most case, we would let the FSTCompiler to just write to a DataOutput, then users are to read it with the public FST constructor right?
   
   The case where we want to read from the FST immediately after writing is the one with BytesStore.
   
   > Do we really need to publish a Freezable interface? Can't it just be a private boolean frozen in this class? Is anything needing freezing besides this BytesStore?
   
   I could remove the Freezeable interface. It was only used for the writeTo(DataOutput), but that would also be unnecessary.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1400641544


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -248,15 +305,17 @@ public Builder<T> directAddressingMaxOversizingFactor(float factor) {
 
     /** Creates a new {@link FSTCompiler}. */
     public FSTCompiler<T> build() {
-      FSTCompiler<T> fstCompiler =
-          new FSTCompiler<>(
-              inputType,
-              suffixRAMLimitMB,
-              outputs,
-              allowFixedLengthArcs,
-              bytesPageBits,
-              directAddressingMaxOversizingFactor);
-      return fstCompiler;
+      try {
+        return new FSTCompiler<>(
+            inputType,
+            suffixRAMLimitMB,
+            outputs,
+            allowFixedLengthArcs,
+            dataOutput,
+            directAddressingMaxOversizingFactor);
+      } catch (IOException e) {
+        throw new RuntimeException(e);

Review Comment:
   Can we defer this? I could add a TODO and do a follow-on PR. The reason is that it will cause even more diffs and not directly related to this PR main purpose.
   
   Anyhow I added another PR to propagate this IOException: https://github.com/apache/lucene/pull/12830. It can be merged in any order (merging #12830 first means we don't have to add the TODO, but merging this is more natural)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1395333947


##########
lucene/core/src/java/org/apache/lucene/util/fst/GrowableByteArrayDataOutput.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
+
+// Storing a byte[] for the current node of the FST we are writing. The byte[] will only grow, never
+// shrink.
+final class GrowableByteArrayDataOutput extends DataOutput implements Accountable {

Review Comment:
   This seems to be much cleaner than the previous BytesStore, and does not have any FST-specific operations. @mikemccand do you think this can be opened up, maybe moving it to `org.apache.lucene.store`?
   
   I'm fine with leaving it as package-private as well.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1353826324


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,19 +21,18 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements Accountable {
+class BytesStore extends FSTWriter {

Review Comment:
   > Once FST.addNode completes, those written bytes are never altered?
   
   More precisely, those bytes are never altered after `FSTCompiler.add` completes. It seems we need to write all nodes of the input in reverse as a whole. But yeah the BytesStore use the head of the buffer as scratch area.
   
   > Hmm getReverseBytesReader is indeed a problem. I wonder how the [Tantivy FST impl](https://blog.burntsushi.net/transducers/) deals with this?
   
   It seems Tantivy segregate the building and the traverse of FST as 2 different entity. The FST Builder will just write the FST to a DataOutput and not allow it to be read directly. I was thinking of this too, as currently we are mixing up the writing and reading:
   - Load a previously saved FST from a DataInput. This is read-only and is fine, and it's how Tantivy FST is created as well.
   - Construct a FST on-the-fly and use it right away. This is both read & write and it uses BytesStore.
   
   I'm kind of favoring the way Tantivy is doing, it's cleaner and more "facade pattern". Maybe we could first refactor so that the FST created on the fly will be written directly to a DataOutput, and then instead of using it directly, we construct a FST from that DataOutput? From the Builder point-of-view it will still create the FST eventually, but could pave the way for segregation of building & reading later (i.e Builder will only write to the DataOutput and it's up to the users to create the corresponding DataInput and construct a FST from that). 
   
   If we are doing that, then we can get rid of the `getReverseBytesReader`. However one issue remains: we still need `getReverseBytesReaderForSuffixSharing` for NodeHash. Or at least some operation for random-access. I think Tantivy is using LRU cache for this in write-through mode: write the node into both the DataOutput and the LRU at the same time. This means we don't even need to read from the DataOutput, but it won't be perfectly minimal (as there will be false-negative cache-miss). I understand that there is the trade-off, but we might also need to support the current minimalist mode.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1828590265

   Hmm, also the `FSTCompiler.ramBytesUsed()` seems to no longer return the growing FST size:
   
   ```
     1> 3100000: 560 bytes; 594876500 nodes
     1> 3200000: 560 bytes; 614066389 nodes
   ```
   
   vs 9.x output:
   
   ```
     1> 3100000: 1220879080 bytes; 594876036 nodes
     1> 3200000: 1260262272 bytes; 614066120 nodes
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1828839806

   Ah I think since we removed the finish(), getting the reverse bytes reader is expectedly slower. We have to copy the bytes to a readonly buffer every time. If this is a problem maybe let try to add the finish() again? WDYT
   
   The ram bytes used is an easy fix. I'll change it in next rev.
   
   The difference in number of nodes is interesting. Lwt me look deeper. But did the 9.x you are testing has incorporated the value-based node hash?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1354640682


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,19 +21,18 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements Accountable {
+class BytesStore extends FSTWriter {

Review Comment:
   As an incremental refactoring, I split the scratch bytes operations out of FSTWriter. We still need the FSTWriter for the other methods (getReverseBytesReader, writeTo, finish, etc.) but it's be much simpler. The scratch bytes operations will still be handled by BytesStore.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1384518777


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTWriter.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+
+/** Abstract class which provides low-level functionality to write to a FST */
+public interface FSTWriter {
+
+  /**

Review Comment:
   These 2 methods are same as DataOutput, but DataOutput is a class which we want to avoid extending from.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1393461923


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -247,16 +306,14 @@ public Builder<T> directAddressingMaxOversizingFactor(float factor) {
     }
 
     /** Creates a new {@link FSTCompiler}. */
-    public FSTCompiler<T> build() {
-      FSTCompiler<T> fstCompiler =
-          new FSTCompiler<>(
-              inputType,
-              suffixRAMLimitMB,
-              outputs,
-              allowFixedLengthArcs,
-              bytesPageBits,
-              directAddressingMaxOversizingFactor);
-      return fstCompiler;
+    public FSTCompiler<T> build() throws IOException {

Review Comment:
   This new `throws IOException` causes diff in some other classes. If preferred I could delay this, and do try-catch here instead to avoid the diffs.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1418728841


##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader, Freezable {
+
+  private final ByteBuffersDataOutput dataOutput;
+  // the DataInput to read from in case the DataOutput has multiple blocks
+  private ByteBuffersDataInput dataInput;
+  // the ByteBuffers to read from in case the DataOutput has a single block
+  private ByteBuffer byteBuffer;
+
+  public ReadWriteDataOutput(ByteBuffersDataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    dataOutput.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) {
+    dataOutput.writeBytes(b, offset, length);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return dataOutput.ramBytesUsed();
+  }
+
+  @Override
+  public void freeze() {
+    // these operations are costly, so we want to compute it once and cache
+    List<ByteBuffer> byteBuffers = dataOutput.toWriteableBufferList();
+    if (byteBuffers.size() == 1) {

Review Comment:
   > I did see improvement as shown in the log above.
   
   I compared 9.7.x and main + this PR (an earlier rev than the current one, the rev where we re-added `.freeze()`), running `Test2BFSTs`.  This PR is a bit faster at building (



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1845435406

   Thanks for persisting @dungba88 -- this was a crazy long and tricky exercise.  I'm so excited Lucene can finally build arbitrarily large FSTs with bounded heap usage.
   
   I'll merge this soon and backport, then let's watch builds for any exciting FST related failures...


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1402058230


##########
lucene/core/src/java/org/apache/lucene/util/fst/FST.java:
##########
@@ -435,6 +433,13 @@ public FST(FSTMetadata<T> metadata, DataInput in, Outputs<T> outputs, FSTStore f
     this.fstReader = fstReader;
   }
 
+  /**
+   * @return true if and only if this FST is readable (e.g has a reverse BytesReader)
+   */
+  public boolean isReadable() {

Review Comment:
   Yeah, I was thinking as we will remove it anyway, maybe we don't need this, along with the null check (as they seems to be throw-away code). If users try to use a non-readable FST now, NPE will thrown instead (of IllegalStateExeption). And then when we change FSTCompiler to return FSTMetadata, they can't even create a non-readable FST. What do you think?
   
   Alternatively, we can double back to `NullFSTReader`, but let it throw exception like this:
   
   ```
   private static final class NullFSTReader implements FSTReader {
   
       @Override
       public long ramBytesUsed() {
         return 0;
       }
   
       @Override
       public FST.BytesReader getReverseBytesReader() {
         throw new UnsupportedOperationException("NullFSTReader does not support getReverseBytesReader()");
       }
   
       @Override
       public void writeTo(DataOutput out) {
         throw new UnsupportedOperationException("NullFSTReader does not support writeTo(DataOutput)");
       }
     }
   ```
   
   The benefits are:
   - We can enforce `FSTReader` to be non-null.
   - Remove all null-check.
   
   In the next PR, `NullFSTReader` can be used for the temporary FST created during building.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1402058230


##########
lucene/core/src/java/org/apache/lucene/util/fst/FST.java:
##########
@@ -435,6 +433,13 @@ public FST(FSTMetadata<T> metadata, DataInput in, Outputs<T> outputs, FSTStore f
     this.fstReader = fstReader;
   }
 
+  /**
+   * @return true if and only if this FST is readable (e.g has a reverse BytesReader)
+   */
+  public boolean isReadable() {

Review Comment:
   Yeah, I was thinking as we will remove it anyway, maybe we don't need this, along with the null check (as they seems to be throw-away code). If users try to use a non-readable FST now, NPE will thrown instead (of IllegalStateExeption). And then when we change FSTCompiler to return FSTMetadata, they can't even create a non-readable FST. What do you think?
   
   Alternatively, we can double back to `NullFSTReader`, but let it throw exception like this:
   
   ```
   private static final class NullFSTReader implements FSTReader {
   
       @Override
       public long ramBytesUsed() {
         return 0;
       }
   
       @Override
       public FST.BytesReader getReverseBytesReader() {
         throw new UnsupportedOperationException("NullFSTReader does not support getReverseBytesReader()");
       }
   
       @Override
       public void writeTo(DataOutput out) {
         throw new UnsupportedOperationException("NullFSTReader does not support writeTo(DataOutput)");
       }
     }
   ```
   
   The benefits are:
   - We can enforce `FSTReader` to be non-null and remove all null-check, while still throwing meaningful exception when users try to use a nkn-readable FST.
   - In the next PR, `NullFSTReader` can be used for the temporary FST created during building so it won't be throw-way code.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1778336652

   (A small note: Tantivy use a value-based LRU cache with 2-item bucket, items will be evicted per bucket: https://github.com/BurntSushi/fst/blob/a0936e9b25a888a0d5b9f94b91997216253e7088/src/raw/registry.rs#L76-L90)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1415641634


##########
lucene/core/src/java/org/apache/lucene/util/fst/FST.java:
##########
@@ -500,6 +502,12 @@ public FSTMetadata<T> getMetadata() {
     return metadata;
   }
 
+  /**
+   * Save the FST to DataOutput.
+   *
+   * @param metaOut the DataOutput to write the metadata to
+   * @param out the DataOutput to write the FST bytes to
+   */
   public void save(DataOutput metaOut, DataOutput out) throws IOException {

Review Comment:
   Yeah. It's also not supported (throwing exception). I'll add a comment.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1418742557


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -218,13 +279,19 @@ public Builder<T> allowFixedLengthArcs(boolean allowFixedLengthArcs) {
     }
 
     /**
-     * How many bits wide to make each byte[] block in the BytesStore; if you know the FST will be
-     * large then make this larger. For example 15 bits = 32768 byte pages.
+     * Set the {@link DataOutput} which is used for low-level writing of FST. If you want the FST to
+     * be immediately readable, you need to use a DataOutput that also implements {@link FSTReader},

Review Comment:
   OK.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1419045318


##########
lucene/core/src/java/org/apache/lucene/util/fst/FST.java:
##########
@@ -419,7 +418,8 @@ public FST(FSTMetadata<T> metadata, DataInput in, FSTStore fstStore) throws IOEx
 
   /** Create the FST with a metadata object and a FSTReader. */
   FST(FSTMetadata<T> metadata, FSTReader fstReader) {
-    this.metadata = metadata;
+    assert fstReader != null;

Review Comment:
   Aha, super!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1391272833


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -359,7 +383,9 @@ public void truncate(long newLen) {
     assert newLen == getPosition();
   }
 
-  public void finish() {
+  @Override
+  public void freeze() {
+    this.frozen = true;

Review Comment:
   I removed the Freezeable as well as the freeze functionality. It was previously used in writeTo(DataOutput) but even then it's not really needed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1395333947


##########
lucene/core/src/java/org/apache/lucene/util/fst/GrowableByteArrayDataOutput.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
+
+// Storing a byte[] for the current node of the FST we are writing. The byte[] will only grow, never
+// shrink.
+final class GrowableByteArrayDataOutput extends DataOutput implements Accountable {

Review Comment:
   This seems to be much cleaner than the previous BytesStore, and does not have any FST-specific operations. @mikemccand do you think this can be opened up, maybe moving it to `org.apache.lucene.store`?
   
   I'm fine with leaving it as package-private as well.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1391064549


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -359,7 +383,9 @@ public void truncate(long newLen) {
     assert newLen == getPosition();
   }
 
-  public void finish() {
+  @Override
+  public void freeze() {
+    this.frozen = true;

Review Comment:
   Should we `assert frozen == false` before setting it to `true`?  Is caller allowed to `freeze()` more than once?
   
   And maybe remove `this.` prefix?



##########
lucene/core/src/java/org/apache/lucene/util/fst/Freezeable.java:
##########
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+/** Represent a datastructure that can be frozen and become immutable. */
+public interface Freezeable {

Review Comment:
   Can this be package private?
   
   Also `Freezeable -> Freezable`, tricky.



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -120,31 +122,54 @@ public class FSTCompiler<T> {
   final float directAddressingMaxOversizingFactor;
   long directAddressingExpansionCredit;
 
-  final BytesStore bytes;
+  // writer for frozen nodes
+  final FSTDataOutputWriter fstWriter;
+  // reader for the frozen nodes
+  final FSTReader fstReader;
+
+  // buffer to store the scratch bytes before writing to the fstWriter
+  final BytesStore bytes = new BytesStore(DEFAULT_SCRATCH_PAGE_BITS);

Review Comment:
   Rename to `scratchBytes`, and maybe say in the comment `// buffer to store bytes for the one node we are currently writing`?
   
   Also, instead of `BytesStore`, could we use Lucene's existing `oal.store` `ByteBuffersDataOutput.newResettableInstance()`?  Let's try to get away from FST implementing so many of its own storage classes/interfaces...



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTDataOutputWriter.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.RamUsageEstimator;
+
+/**
+ * Allow the FST to be written to a {@link DataOutput}. Generally it only supports writing to the
+ * {@link DataOutput} and not reading from it. To read, you must either: 1. construct a
+ * corresponding {@link org.apache.lucene.store.DataInput} and use the {@link FSTStore} to read pr
+ * 2. use a DataOutput which also implements {@link FSTReader}
+ */
+final class FSTDataOutputWriter implements Freezeable {
+
+  private static final long BASE_RAM_BYTES_USED =
+      RamUsageEstimator.shallowSizeOfInstance(FSTDataOutputWriter.class);
+
+  /** the main DataOutput to store the FST bytes */
+  private final DataOutput dataOutput;
+
+  private long size = 0L;

Review Comment:
   You don't need the `= 0L` -- it's java's default already.



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -120,31 +122,54 @@ public class FSTCompiler<T> {
   final float directAddressingMaxOversizingFactor;
   long directAddressingExpansionCredit;
 
-  final BytesStore bytes;
+  // writer for frozen nodes
+  final FSTDataOutputWriter fstWriter;
+  // reader for the frozen nodes
+  final FSTReader fstReader;
+
+  // buffer to store the scratch bytes before writing to the fstWriter
+  final BytesStore bytes = new BytesStore(DEFAULT_SCRATCH_PAGE_BITS);
 
   /**
    * Instantiates an FST/FSA builder with default settings and pruning options turned off. For more
    * tuning and tweaking, see {@link Builder}.
    */
   // TODO: remove this?  Builder API should be the only entry point?
   public FSTCompiler(FST.INPUT_TYPE inputType, Outputs<T> outputs) {
-    this(inputType, 32.0, outputs, true, 15, 1f);
+    this(
+        inputType,
+        32.0,
+        outputs,
+        true,
+        new FSTDataOutputWriter(getLegacyDataOutput(DEFAULT_BLOCK_BITS)),
+        1f);
+  }
+
+  static DataOutput getLegacyDataOutput(int blockBits) {
+    return new BytesStore(blockBits);
   }
 
   private FSTCompiler(
       FST.INPUT_TYPE inputType,
       double suffixRAMLimitMB,
       Outputs<T> outputs,
       boolean allowFixedLengthArcs,
-      int bytesPageBits,
+      FSTDataOutputWriter fstWriter,
       float directAddressingMaxOversizingFactor) {
     this.allowFixedLengthArcs = allowFixedLengthArcs;
     this.directAddressingMaxOversizingFactor = directAddressingMaxOversizingFactor;
-    bytes = new BytesStore(bytesPageBits);
     // pad: ensure no node gets address 0 which is reserved to mean
     // the stop state w/ no arcs
-    bytes.writeByte((byte) 0);
-    fst = new FST<>(new FST.FSTMetadata<>(inputType, null, -1, VERSION_CURRENT, 0), outputs, bytes);
+    try {
+      fstWriter.writeByte((byte) 0);
+    } catch (IOException e) {

Review Comment:
   Maybe we add `throws IOException` to this ctor?



##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -26,7 +26,8 @@
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements FSTReader {
+// TODO: Separate the scratch writer and reader functionality
+class BytesStore extends DataOutput implements FSTReader, Freezeable {

Review Comment:
   Could we remove this class entirely?  And callers that want to write FST and immediately use it in RAM should just use `ByteBuffersDataOutput` for their scratch area?
   
   Do we really need to publish a `Freezable` interface?  Can't it just be a `private boolean frozen` in this class?  Is anything needing freezing besides this `BytesStore`?
   
   And then `FSTCompiler` should only get a `DataOutput`?  Hmm this may require implementing a `ByteBuffersRandomAccessInput` or so, so the reverse (positional) byte reads work.



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -286,9 +331,9 @@ public long getMappedStateCount() {
     return dedupHash == null ? 0 : nodeCount;
   }
 
-  private CompiledNode compileNode(UnCompiledNode<T> nodeIn, int tailLength) throws IOException {
+  private CompiledNode compileNode(UnCompiledNode<T> nodeIn) throws IOException {
     final long node;
-    long bytesPosStart = bytes.getPosition();
+    long bytesPosStart = fstReader.size();

Review Comment:
   Can't we just record our current output position and use that as the address?



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -120,31 +122,54 @@ public class FSTCompiler<T> {
   final float directAddressingMaxOversizingFactor;
   long directAddressingExpansionCredit;
 
-  final BytesStore bytes;
+  // writer for frozen nodes
+  final FSTDataOutputWriter fstWriter;
+  // reader for the frozen nodes
+  final FSTReader fstReader;

Review Comment:
   I'm confused -- we don't ever need to read frozen nodes anymore, now that `NodeHash` takes a complete copy of the nodes it wants to read, and it decodes the node itself?  It seems like we (mostly?) just use this `fstReader`'s `.size()` method to know the address of the next written node/arc?  Can't we just track that ourselves (`numBytesWritten` or so)?



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -38,9 +38,6 @@
 import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.fst.FST.INPUT_TYPE; // javadoc
 
-// TODO: could we somehow stream an FST to disk while we

Review Comment:
   Yay :)



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -85,6 +84,9 @@ public class FSTCompiler<T> {
    */
   private static final float DIRECT_ADDRESSING_MAX_OVERSIZE_WITH_CREDIT_FACTOR = 1.66f;
 
+  // The number of bits for scratch area when adding a node
+  private static final int DEFAULT_SCRATCH_PAGE_BITS = 8;

Review Comment:
   Remove `DEFAULT_` prefix?  This is not controllable by the user right?  And move the constant to right before we create the scratch bytes?



##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -337,11 +349,23 @@ public long size() {
     return getPosition();
   }
 
+  /** Similar to {@link #truncate(long)} with newLen=0 but keep the first block to reduce GC. */
+  public void reset() {

Review Comment:
   If we can switch to Lucene's existing `ByteBuffersDataOutput` for our scratch single-node bytes then we can remove this?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1395333947


##########
lucene/core/src/java/org/apache/lucene/util/fst/GrowableByteArrayDataOutput.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
+
+// Storing a byte[] for the current node of the FST we are writing. The byte[] will only grow, never
+// shrink.
+final class GrowableByteArrayDataOutput extends DataOutput implements Accountable {

Review Comment:
   This seems to be much cleaner than the previous BytesStore, and does not have any FST-specific operations. @mikemccand do you think this can be opened up, maybe moving it to `org.apache.lucene.store`? (The reverse function might stand out a bit)
   
   I'm fine with leaving it as package-private as well.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1820094902

   > I've taken a look - I think this class should be kept package-private (or even class-private). It is a DataOutput but it serves very specific purposes (with the unusual methods to move the write pointer at random, expand the underlying buffer, etc.).
   
   I think that makes sense. Maybe let keep it package-private so that it can be unit-tested?
   
   > I also changed the implementation to use OutputStreamDataOutput [1] but it's not any more elegant than written explicitly, so please disregard that suggestion.
   
   Thank you for giving it a try!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1400642484


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -277,9 +336,9 @@ public long getMappedStateCount() {
     return dedupHash == null ? 0 : nodeCount;
   }
 
-  private CompiledNode compileNode(UnCompiledNode<T> nodeIn, int tailLength) throws IOException {
+  private CompiledNode compileNode(UnCompiledNode<T> nodeIn) throws IOException {

Review Comment:
   It was unused even before this PR, I just remove it for convenience.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1402058230


##########
lucene/core/src/java/org/apache/lucene/util/fst/FST.java:
##########
@@ -435,6 +433,13 @@ public FST(FSTMetadata<T> metadata, DataInput in, Outputs<T> outputs, FSTStore f
     this.fstReader = fstReader;
   }
 
+  /**
+   * @return true if and only if this FST is readable (e.g has a reverse BytesReader)
+   */
+  public boolean isReadable() {

Review Comment:
   Yeah, I was thinking as we will remove it anyway, maybe we don't need this, along with the null check (as they seems to be throw-away code). If users try to use a non-readable FST now, NPE will thrown instead (of IllegalStateExeption). And then when we change FSTCompiler to return FSTMetadata, they can't even create a non-readable FST. What do you think?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1353826324


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,19 +21,18 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements Accountable {
+class BytesStore extends FSTWriter {

Review Comment:
   > Once FST.addNode completes, those written bytes are never altered?
   
   More precisely, those bytes are never altered after `FSTCompiler.add` completes. It seems we need to write all nodes of the input in reverse as a whole. But yeah the BytesStore use the head of the buffer as scratch area.
   
   > Hmm getReverseBytesReader is indeed a problem. I wonder how the [Tantivy FST impl](https://blog.burntsushi.net/transducers/) deals with this?
   
   It seems Tantivy segregate the building and the traverse of FST as 2 different entity. The FST Builder will just write the FST to a DataOutput and not allow it to be read directly. I was thinking of this too, as currently we are mixing up the writing and reading:
   - Load a previously saved FST from a DataInput. This is read-only and is fine, and it's how Tantivy FST is created as well.
   - Construct a FST on-the-fly and use it right away. This is both read & write and it uses BytesStore.
   
   I'm kind of favoring the way Tantivy is doing, it's cleaner and more "facade pattern". Maybe we could first refactor so that the FST created on the fly will be written directly to a DataOutput, and then instead of using it directly, we construct a FST from that DataOutput? From the Builder point-of-view it will still create the FST eventually, but could pave the way for segregation of building & reading later (i.e Builder will only write to the DataOutput and it's up to the users to create the corresponding DataInput and construct a FST from that).
   
   UPDATE: The above seems to be a breaking change and might not be a small, incremental one, as NodeHash needs the FST to be initialized first so that it can search over the previously written nodes.
   
   If we are doing that, then we can get rid of the `getReverseBytesReader`. However one issue remains: we still need `getReverseBytesReaderForSuffixSharing` for NodeHash. Or at least some operation for random-access. I think Tantivy is using LRU cache for this in write-through mode: write the node into both the DataOutput and the LRU at the same time. This means we don't even need to read from the DataOutput, but it won't be perfectly minimal (as there will be false-negative cache-miss). I understand that there is the trade-off, but we might also need to support the current minimalist mode.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1351949356


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,19 +21,18 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements Accountable {
+class BytesStore extends FSTWriter {

Review Comment:
   > Besides `reverse` most of the methods here also needs to write/update arbitrarily previously written bytes (within the current frontier input), such as writeBytes/writeByte/copyBytes/truncate.
   
   I think these "modify prior bytes" are only called from `fst.addNode`, where that (complex!!) method is basically using the tail of the `BytesStore` as a scratch area?  I.e. it writes some bytes first, and might go back and shuffle the bytes around, depending on whether it's "direct addressing" or "binary search"?  Once `FST.addNode` completes, those written bytes are never altered?
   
   > Moreover, there is also methods which requires reading of previously written bytes that can't simply fit in the DataOutput (getReverseBytesReader e.g).
   
   Hmm `getReverseBytesReader` is indeed a problem.  I wonder how the [Tantivy FST impl](https://blog.burntsushi.net/transducers/) deals with this?
   
   If we take the `FSTWriter` approach, are you thinking that we could make an impl of this class based e.g. on `FileChannel` directly (supports absolute positional reads, reading from a still-appending file), bypassing Lucene's `Directory` abstraction entirely?  That is not a great solution (we try to have all IO go through `Directory`), but, perhaps as an intermediate state, for users directly creating massive FSTs, it's acceptable.  But that'd mean we could not fix Lucene by default to do all of its FST compilation off-heap...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1400633005


##########
lucene/core/src/java/org/apache/lucene/util/fst/GrowableByteArrayDataOutput.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
+
+// Storing a byte[] for the current node of the FST we are writing. The byte[] will only grow, never
+// shrink.
+final class GrowableByteArrayDataOutput extends DataOutput implements Accountable {
+
+  private static final long BASE_RAM_BYTES_USED =
+      RamUsageEstimator.shallowSizeOfInstance(GrowableByteArrayDataOutput.class);
+
+  private static final int INITIAL_SIZE = 1 << 8;
+
+  // holds an initial size of 256 bytes. this byte array will only grow, but not shrink
+  private byte[] bytes = new byte[INITIAL_SIZE];
+
+  private int nextWrite;
+
+  @Override
+  public void writeByte(byte b) {
+    ensureCapacity(1);
+    bytes[nextWrite++] = b;
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int len) {
+    ensureCapacity(len);
+    System.arraycopy(b, offset, bytes, nextWrite, len);
+    nextWrite += len;
+  }
+
+  public int getPosition() {
+    return nextWrite;
+  }
+
+  public byte[] getBytes() {
+    return bytes;
+  }
+
+  /** Set the position of the byte[], increasing the capacity if needed */
+  public void setPosition(int newLen) {
+    assert newLen >= 0;
+    if (newLen > nextWrite) {
+      ensureCapacity(newLen - nextWrite);
+    }
+    nextWrite = newLen;

Review Comment:
   Yeah I think don't need to shrink for now. It would also reduce the GC load and the times we need to grow the array (comparing to when we shrink and expand the `byte[]` as it goes). I ran with a custom dictionary of size ~1MB and the maximum node is <256 bytes. For text-based dictionary I think it would depend on the length of the word (which is usually short) and the output (which depends).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1400663661


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -827,22 +910,24 @@ void setEmptyOutput(T v) {
   }
 
   void finish(long newStartNode) {
-    assert newStartNode <= bytes.size();
+    assert newStartNode <= numBytesWritten;
     if (fst.metadata.startNode != -1) {
       throw new IllegalStateException("already finished");
     }
     if (newStartNode == FINAL_END_NODE && fst.metadata.emptyOutput != null) {
       newStartNode = 0;
     }
     fst.metadata.startNode = newStartNode;
-    fst.metadata.numBytes = bytes.getPosition();
+    fst.metadata.numBytes = numBytesWritten;
   }
 
   private boolean validOutput(T output) {
     return output == NO_OUTPUT || !output.equals(NO_OUTPUT);
   }
 
   /** Returns final FST. NOTE: this will return null if nothing is accepted by the FST. */
+  // TODO: make this method to only return the FSTMetadata and user needs to construct the FST
+  // themselves
   public FST<T> compile() throws IOException {

Review Comment:
   Currently if we try to read from it, and if the DataOutput is not a FSTReader then the reader would return null (and be changed to throw an IllegalStateException). If we change this method to just return the FSTMetadata the user will be mindful that the FST might not be readable and we will never return an unusable FST. But yeah it will touch a lot of codes.
   
   I also put together this PR for the migration later: https://github.com/dungba88/lucene/pull/19. It will ensure that we will never return an invalid/unusable FST.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1400633005


##########
lucene/core/src/java/org/apache/lucene/util/fst/GrowableByteArrayDataOutput.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
+
+// Storing a byte[] for the current node of the FST we are writing. The byte[] will only grow, never
+// shrink.
+final class GrowableByteArrayDataOutput extends DataOutput implements Accountable {
+
+  private static final long BASE_RAM_BYTES_USED =
+      RamUsageEstimator.shallowSizeOfInstance(GrowableByteArrayDataOutput.class);
+
+  private static final int INITIAL_SIZE = 1 << 8;
+
+  // holds an initial size of 256 bytes. this byte array will only grow, but not shrink
+  private byte[] bytes = new byte[INITIAL_SIZE];
+
+  private int nextWrite;
+
+  @Override
+  public void writeByte(byte b) {
+    ensureCapacity(1);
+    bytes[nextWrite++] = b;
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int len) {
+    ensureCapacity(len);
+    System.arraycopy(b, offset, bytes, nextWrite, len);
+    nextWrite += len;
+  }
+
+  public int getPosition() {
+    return nextWrite;
+  }
+
+  public byte[] getBytes() {
+    return bytes;
+  }
+
+  /** Set the position of the byte[], increasing the capacity if needed */
+  public void setPosition(int newLen) {
+    assert newLen >= 0;
+    if (newLen > nextWrite) {
+      ensureCapacity(newLen - nextWrite);
+    }
+    nextWrite = newLen;

Review Comment:
   Yeah I think don't need to shrink for now. It would also reduce the GC load and the times we need to grow the array (comparing to when we shrink and expand the `byte[]` as it goes)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1400634116


##########
lucene/core/src/java/org/apache/lucene/util/fst/FST.java:
##########
@@ -435,6 +433,13 @@ public FST(FSTMetadata<T> metadata, DataInput in, Outputs<T> outputs, FSTStore f
     this.fstReader = fstReader;
   }
 
+  /**
+   * @return true if and only if this FST is readable (i.e. has a reverse BytesReader)
+   */
+  public boolean hasReverseBytesReader() {

Review Comment:
   Yes, if the `FSTCompiler` is written to a non-FSTReader DataOutput, such as IndexOutput then it won't be readable. But this method has no use for now, I think I can remove it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1391285510


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -120,31 +122,54 @@ public class FSTCompiler<T> {
   final float directAddressingMaxOversizingFactor;
   long directAddressingExpansionCredit;
 
-  final BytesStore bytes;
+  // writer for frozen nodes
+  final FSTDataOutputWriter fstWriter;
+  // reader for the frozen nodes
+  final FSTReader fstReader;
+
+  // buffer to store the scratch bytes before writing to the fstWriter
+  final BytesStore bytes = new BytesStore(DEFAULT_SCRATCH_PAGE_BITS);
 
   /**
    * Instantiates an FST/FSA builder with default settings and pruning options turned off. For more
    * tuning and tweaking, see {@link Builder}.
    */
   // TODO: remove this?  Builder API should be the only entry point?
   public FSTCompiler(FST.INPUT_TYPE inputType, Outputs<T> outputs) {
-    this(inputType, 32.0, outputs, true, 15, 1f);
+    this(
+        inputType,
+        32.0,
+        outputs,
+        true,
+        new FSTDataOutputWriter(getLegacyDataOutput(DEFAULT_BLOCK_BITS)),
+        1f);
+  }
+
+  static DataOutput getLegacyDataOutput(int blockBits) {
+    return new BytesStore(blockBits);
   }
 
   private FSTCompiler(
       FST.INPUT_TYPE inputType,
       double suffixRAMLimitMB,
       Outputs<T> outputs,
       boolean allowFixedLengthArcs,
-      int bytesPageBits,
+      FSTDataOutputWriter fstWriter,
       float directAddressingMaxOversizingFactor) {
     this.allowFixedLengthArcs = allowFixedLengthArcs;
     this.directAddressingMaxOversizingFactor = directAddressingMaxOversizingFactor;
-    bytes = new BytesStore(bytesPageBits);
     // pad: ensure no node gets address 0 which is reserved to mean
     // the stop state w/ no arcs
-    bytes.writeByte((byte) 0);
-    fst = new FST<>(new FST.FSTMetadata<>(inputType, null, -1, VERSION_CURRENT, 0), outputs, bytes);
+    try {
+      fstWriter.writeByte((byte) 0);
+    } catch (IOException e) {

Review Comment:
   Ah, if the public ctor is be removed in https://github.com/apache/lucene/pull/12715 then we can throw it here. I'll wait for that PR to be merged.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1395069004


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -337,11 +349,23 @@ public long size() {
     return getPosition();
   }
 
+  /** Similar to {@link #truncate(long)} with newLen=0 but keep the first block to reduce GC. */
+  public void reset() {

Review Comment:
   Yeah it seems the main complexity is due to the fact that we need to override the previously written bytes, sometimes when we want to remove the labels, sometimes when we need to update the header after writing everything. So I guess making it forward would not eliminate those backward writing operation.
   
   I simplified the writing operation a bit in the direct addressing mode. Seems like we can convert it to append-only mode. But the binary search one seems to be difficult, as when we fix the arc we need to do in backward (I assume to avoid overriding?). Anyhow, the remaining backward operations are:
   - writeBytes: As mentioned, this is used in the binary search mode. If we first copy the whole buffer to a new one, we can start appending back to back. But this isn't as efficient as the current in-place copy? The whole method seems to be expanding each arc from variable arc length to fixed arc lengths.
   - reverse: This is only done at the end before writing to the DataOutput and the NodeHash.ByteBlockPool



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1391183424


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -120,31 +122,54 @@ public class FSTCompiler<T> {
   final float directAddressingMaxOversizingFactor;
   long directAddressingExpansionCredit;
 
-  final BytesStore bytes;
+  // writer for frozen nodes
+  final FSTDataOutputWriter fstWriter;
+  // reader for the frozen nodes
+  final FSTReader fstReader;
+
+  // buffer to store the scratch bytes before writing to the fstWriter
+  final BytesStore bytes = new BytesStore(DEFAULT_SCRATCH_PAGE_BITS);
 
   /**
    * Instantiates an FST/FSA builder with default settings and pruning options turned off. For more
    * tuning and tweaking, see {@link Builder}.
    */
   // TODO: remove this?  Builder API should be the only entry point?
   public FSTCompiler(FST.INPUT_TYPE inputType, Outputs<T> outputs) {
-    this(inputType, 32.0, outputs, true, 15, 1f);
+    this(
+        inputType,
+        32.0,
+        outputs,
+        true,
+        new FSTDataOutputWriter(getLegacyDataOutput(DEFAULT_BLOCK_BITS)),
+        1f);
+  }
+
+  static DataOutput getLegacyDataOutput(int blockBits) {
+    return new BytesStore(blockBits);
   }
 
   private FSTCompiler(
       FST.INPUT_TYPE inputType,
       double suffixRAMLimitMB,
       Outputs<T> outputs,
       boolean allowFixedLengthArcs,
-      int bytesPageBits,
+      FSTDataOutputWriter fstWriter,
       float directAddressingMaxOversizingFactor) {
     this.allowFixedLengthArcs = allowFixedLengthArcs;
     this.directAddressingMaxOversizingFactor = directAddressingMaxOversizingFactor;
-    bytes = new BytesStore(bytesPageBits);
     // pad: ensure no node gets address 0 which is reserved to mean
     // the stop state w/ no arcs
-    bytes.writeByte((byte) 0);
-    fst = new FST<>(new FST.FSTMetadata<>(inputType, null, -1, VERSION_CURRENT, 0), outputs, bytes);
+    try {
+      fstWriter.writeByte((byte) 0);
+    } catch (IOException e) {

Review Comment:
   Can we do this in a follow-up PR? Having this constructor throwing exception means the public one also needs to throw exception, and that was used in a lot of places :)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1395069004


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -337,11 +349,23 @@ public long size() {
     return getPosition();
   }
 
+  /** Similar to {@link #truncate(long)} with newLen=0 but keep the first block to reduce GC. */
+  public void reset() {

Review Comment:
   Yeah it seems the main complexity is due to the fact that we need to override the previously written bytes, sometimes when we want to remove the labels, sometimes when we need to update the header after writing everything. So I guess making it forward would not eliminate those backward writing operation.
   
   I simplified the writing operation a bit in the direct addressing mode. Seems like we can convert it to append-only mode. But the binary search one seems to be difficult, as when we fix the arc we need to do in backward (I assume to avoid overriding?). Anyhow, the remaining backward operations are:
   - writeBytes: As mentioned, this is used in the binary search mode. If we first copy the whole buffer to a new one, we can start appending back to back. But this isn't as efficient as the current in-place copy?
   - reverse: This is only done at the end before writing to the DataOutput and the NodeHash.ByteBlockPool



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1829144978

   Tested Test2BFST with `-Dtests.seed=D193E7FD4B9E68C4`
   
   **mainline**
   
   ```
   1100000: 432584968 RAM bytes used; 432367203 FST bytes; 211082699 nodes; took 248 seconds
   ```
   
   **pr**
   
   ```
   1100000: 432427020 RAM bytes used; 432367203 FST bytes; 211082699 nodes; took 236 seconds
   ```
   
   **9_x**
   
   ```
   1100000: 432584968 RAM bytes used; 432367203 FST bytes; 211082699 nodes; took 260 seconds
   ```
   
   The FST size, number nodes are exactly the same, while the running time doesn't seem to differ much (running with my personal computer so the background process could be noisy).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1829601799

   > Tested Test2BFST with -Dtests.seed=D193E7FD4B9E68C4
   
   Duh, I forgot to fix the seed!  And the test is indeed random in the inputs it compiles.  Sorry for the false alarm :)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1407526003


##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader, Freezable {
+
+  private final ByteBuffersDataOutput dataOutput;
+  private ByteBuffersDataInput dataInput;
+  private List<ByteBuffer> byteBuffers;
+
+  public ReadWriteDataOutput(ByteBuffersDataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    dataOutput.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) {
+    dataOutput.writeBytes(b, offset, length);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return dataOutput.ramBytesUsed();
+  }
+
+  @Override
+  public void freeze() {
+    // these operations are costly, so we want to compute it once and cache
+    byteBuffers = dataOutput.toWriteableBufferList();

Review Comment:
   I tried several ways to optimize the `getReverseBytesReader()` method, such as bypassing all `asReadOnlyBuffer()` and accessing the byte array directly, but they wouldn't come close compared to BytesStore:
   - The `ByteBuffersDataOutput.blocks` is a Deque and thus not random accessible. We need to copy to a list, and this takes time as the number of blocks grow.
   - It took ~15 seconds to verify (`Util.get()`) 100K times, compared to only ~2 seconds with BytesStore.
   
   Hence I had to re-add `Freezable()` here, so that the costly operations will only be called once. The alternative way is to cache, but `getReverseBytesReader()` can be called concurrently hence we need some kind of ConcurrentHashMap with only 2 items, which is a weird way to cache.
   
   Another way that is not so efficient as Freezable, but avoid the added interface is to simply use a volatile variable and have a simple null check (the volatile is there to enforce a write barrier).
   
   ```
   private volatile ByteBuffersDataInput dataInput;
   private volatile List<ByteBuffer> byteBuffers;
   
   public FST.BytesReader getReverseBytesReader() {
       if (byteBuffers == null) {
           byteBuffers = dataOutput.toWriteableBufferList();
       }
       if (dataInput == null) {
           dataInput = new ByteBuffersDataInput(byteBuffers);
       }
       // use the variables
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1828597325

   Hmm, also oddly -- why do the number of nodes differ between `main` and 9.x?  This PR should not have altered how many nodes are created as a function of FST inputs right?  Or maybe how the RAM accounting is done for NodeHash changes a bit, causing a slightly larger FST on `main` than 9.x?  So many WTFs ("wow that's funny") suddenly :)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1416980241


##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader, Freezable {
+
+  private final ByteBuffersDataOutput dataOutput;
+  // the DataInput to read from in case the DataOutput has multiple blocks
+  private ByteBuffersDataInput dataInput;
+  // the ByteBuffers to read from in case the DataOutput has a single block
+  private ByteBuffer byteBuffer;
+
+  public ReadWriteDataOutput(ByteBuffersDataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    dataOutput.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) {
+    dataOutput.writeBytes(b, offset, length);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return dataOutput.ramBytesUsed();
+  }
+
+  @Override
+  public void freeze() {
+    // these operations are costly, so we want to compute it once and cache
+    List<ByteBuffer> byteBuffers = dataOutput.toWriteableBufferList();
+    if (byteBuffers.size() == 1) {

Review Comment:
   Thanks @dweiss . I think that makes sense. I left this PR to use the simple implementation (i.e just delegate everything to ByteBuffersDataInput). I'll mark the other one as draft, as I still want to play around with the benchmark just for curiosity.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1416379828


##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader, Freezable {
+
+  private final ByteBuffersDataOutput dataOutput;
+  // the DataInput to read from in case the DataOutput has multiple blocks
+  private ByteBuffersDataInput dataInput;
+  // the ByteBuffers to read from in case the DataOutput has a single block
+  private ByteBuffer byteBuffer;
+
+  public ReadWriteDataOutput(ByteBuffersDataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    dataOutput.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) {
+    dataOutput.writeBytes(b, offset, length);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return dataOutput.ramBytesUsed();
+  }
+
+  @Override
+  public void freeze() {
+    // these operations are costly, so we want to compute it once and cache
+    List<ByteBuffer> byteBuffers = dataOutput.toWriteableBufferList();
+    if (byteBuffers.size() == 1) {

Review Comment:
   Sorry for the missing context, I was referring to a regression I found at https://github.com/apache/lucene/issues/10520 instead. 
   
   Maybe it was different for ByteBuffersDataInput, or there would be no regression if the block size is just 1. I'll think we need to run some test to verify. Let me know if there's a process for this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1418738243


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -120,22 +125,44 @@ public class FSTCompiler<T> {
   final float directAddressingMaxOversizingFactor;
   long directAddressingExpansionCredit;
 
-  final BytesStore bytes;
+  // the DataOutput to stream the FST bytes to
+  final DataOutput dataOutput;
+
+  // buffer to store bytes for the one node we are currently writing
+  final GrowableByteArrayDataOutput scratchBytes = new GrowableByteArrayDataOutput();
+
+  private long numBytesWritten;
+
+  /**
+   * Get an on-heap DataOutput that allows the FST to be read immediately after writing.

Review Comment:
   Yay, very exciting we can now build massive FSTs with "trivial" amounts of heap!!  Thank you [Tantivy FST implementation](https://blog.burntsushi.net/transducers/) :)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1845249074

   Thanks everyone! I addressed comments, putting a simpler implementation.
   
   +1 to the FST micro benchmarking


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1350425577


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,19 +21,18 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements Accountable {
+class BytesStore extends FSTWriter {

Review Comment:
   Besides `reverse` most of the methods here also needs to write/update previously written bytes (within the current frontier input).
   
   Moreover, there is also methods which requires reading of previously written bytes that can't simply fit in the DataOutput (getReverseBytesReader e.g).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1350439623


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,19 +21,18 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements Accountable {
+class BytesStore extends FSTWriter {

Review Comment:
   An alternative way is to always use BytesStore for in-mem operator, then delegating the final write to a DataOutput, (which could also be supported by this approach as well). But then, it would have a problem:
   - If someone wants to use an in-mem DataOutput, they would need an extra write (one to the BytesStore and one to the in-mem DataOutput) and thus less efficient.
   - Another complication is that, the NodeHash used for suffix sharing would need to search the FST in both previous (flushed) nodes and frontier (in-writing) nodes, and thus breaking them into 2 different, independent data-structures would make it hard to control the read pointer.
   
   The approach in this PR would allow one to use either in-mem with BytesStore, or hybrid of in-mem & other DataOutput.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1353826324


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,19 +21,18 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements Accountable {
+class BytesStore extends FSTWriter {

Review Comment:
   > Once FST.addNode completes, those written bytes are never altered?
   
   More precisely, those bytes are never altered after `FSTCompiler.add` completes. It seems we need to write all nodes of the input in reverse as a whole. But yeah the BytesStore use the head of the buffer as scratch area.
   
   > Hmm getReverseBytesReader is indeed a problem. I wonder how the [Tantivy FST impl](https://blog.burntsushi.net/transducers/) deals with this?
   
   It seems Tantivy segregate the building and the traverse of FST as 2 different entity. The FST Builder will just write the FST to a DataOutput and not allow it to be read directly. I was thinking of this too, as currently we are mixing up the writing and reading:
   - Load a previously saved FST from a DataInput. This is read-only and is fine, and it's how Tantivy FST is created as well.
   - Construct a FST on-the-fly and use it right away. This is both read & write and it uses BytesStore.
   
   I'm kind of favoring the way Tantivy is doing, it's cleaner and more "facade pattern". Maybe we could first refactor so that the FST created on the fly will be written directly to a DataOutput, and then instead of using it directly, we construct a FST from that DataOutput?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1350439623


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,19 +21,18 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements Accountable {
+class BytesStore extends FSTWriter {

Review Comment:
   An alternative way is to always use BytesStore for in-mem operator, then delegating the final write to a DataOutput, (which could also be supported by this approach as well). But then, it would have a problem:
   - If someone wants to use an in-mem DataOutput, they would need an extra write (one to the BytesStore and one to the in-mem DataOutput) and thus less efficient.
   
   The approach in this PR would allow one to use either in-mem with BytesStore, or hybrid of in-mem & other DataOutput.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1353826324


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,19 +21,18 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements Accountable {
+class BytesStore extends FSTWriter {

Review Comment:
   > Once FST.addNode completes, those written bytes are never altered?
   
   ~~More precisely, those bytes are never altered after `FSTCompiler.add` completes. It seems we need to write all nodes of the input in reverse as a whole.~~ It seems it's possible to flush the scratch bytes after every addNode. yeah the BytesStore use the head of the buffer as scratch area.
   
   > Hmm getReverseBytesReader is indeed a problem. I wonder how the [Tantivy FST impl](https://blog.burntsushi.net/transducers/) deals with this?
   
   It seems Tantivy segregate the building and the traverse of FST as 2 different entity. The FST Builder will just write the FST to a DataOutput and not allow it to be read directly. I was thinking of this too, as currently we are mixing up the writing and reading:
   - Load a previously saved FST from a DataInput. This is read-only and is fine, and it's how Tantivy FST is created as well.
   - Construct a FST on-the-fly and use it right away. This is both read & write and it uses BytesStore.
   
   I'm kind of favoring the way Tantivy is doing, it's cleaner and more "facade pattern". Maybe we could first refactor so that the FST created on the fly will be written directly to a DataOutput, and then instead of using it directly, we construct a FST from that DataOutput? From the Builder point-of-view it will still create the FST eventually, but could pave the way for segregation of building & reading later (i.e Builder will only write to the DataOutput and it's up to the users to create the corresponding DataInput and construct a FST from that).
   
   UPDATE: The above seems to be a breaking change and might not be a small, incremental one, as NodeHash needs the FST to be initialized first so that it can search over the previously written nodes.
   
   If we are doing that, then we can get rid of the `getReverseBytesReader`. However one issue remains: we still need `getReverseBytesReaderForSuffixSharing` for NodeHash. Or at least some operation for random-access. I think Tantivy is using LRU cache for this in write-through mode: write the node into both the DataOutput and the LRU at the same time. This means we don't even need to read from the DataOutput, but it won't be perfectly minimal (as there will be false-negative cache-miss). I understand that there is the trade-off, but we might also need to support the current minimalist mode.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1828936176

   I checked some of the usage in the analysis module. SynonymGraphFilter cache the `BytesReader` on constructor, and I think TokenFilter by default are cached per field? But lots of other places does not have this cache, such as:
   - Stemmer
   - GeneratingSuggester
   - ...
   
   I think the main bottleneck are the non-trivial creation of the BufferList (in `toBufferList` or `toWritableBufferList`) and the ByteBufferDataInput.
   
   In the worst case, how do you think about reviving the BytesStore (with a much simpler implementation as it does have the scratch bytes operation)? Or alternatively, we could create a reusable block-based byte array DataOutput (not sure if there's already something like that).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1829613228

   Thanks @dungba88 -- I will catch up with the latest iterations soon.  I tested just how much slower the `ByteBuffer` based store is than the FST's `BytesStore`:
   
   9.x:
   
   ```
     1> TEST: now verify [fst size=5368709408; nodeCount=2252341463; arcCount=2264078568]
     1> 144.4s 0...
     1> 150.7s 1000000...
     1> 157.1s 2000000...
     1> 163.5s 3000000...
     1> 169.9s 4000000...
     1> 176.2s 5000000...
     1> 182.6s 6000000...
     1> 188.9s 7000000...
     1> 195.3s 8000000...
     1> 201.6s 9000000...
     1> 208.0s 10000000...
     1> 214.3s 11000000...
   ```
   
   This PR:
   
   ```
     1> TEST: now verify [fst size=4622366544; nodeCount=2252341629; arcCount=2264078736]
     1> 0.0s 0...
     1> 599.5s 100000...
     1> 1201.7s 200000...
     1> 1797.6s 300000...
     1> 2392.0s 400000...
     1> 2967.8s 500000...
     1> 3532.1s 600000...
     1> 4110.5s 700000...
     1> 4678.4s 800000...
     1> 5247.9s 900000...
     1> 5815.1s 1000000...
     1> 6382.3s 1100000...
     1> 6950.0s 1200000...
     1> 7514.7s 1300000...
     1> 8087.2s 1400000...
     1> 8657.6s 1500000...
     1> 9227.2s 1600000...
     1> 9795.8s 1700000...
     1> 10358.1s 1800000...
     1> 10923.6s 1900000...
     1> 11491.5s 2000000...
     1> 12057.4s 2100000...
     1> 12649.5s 2200000...
     1> 13231.7s 2300000...
     1> 13797.8s 2400000...
     1> 14382.8s 2500000...
     1> 15017.1s 2600000...
     1> 15633.8s 2700000...
     1> 16200.9s 2800000...
     1> 16762.8s 2900000...
     1> 17329.6s 3000000...
     1> 17891.7s 3100000...
     1> 18462.2s 3200000...
     1> 19025.3s 3300000...
     1> 19609.4s 3400000...
     1> 20171.3s 3500000...
     1> 20738.6s 3600000...
     1> 21306.2s 3700000...
     1> 21872.2s 3800000...
     1> 22439.8s 3900000...
     1> 22998.3s 4000000...
   ```
   
   More than two orders-of-magnitude (base 10) slower!
   
   +1 to `freeze` and do the output -> input conversion only once.  Thanks @dungba88.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1839997544

   >  I tested just how much slower the ByteBuffer based store is than the FST's BytesStore:
   
   I assume this is before the last iteration that does the freeze, is that right? What do you think about the last results?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1416403950


##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader, Freezable {
+
+  private final ByteBuffersDataOutput dataOutput;
+  // the DataInput to read from in case the DataOutput has multiple blocks
+  private ByteBuffersDataInput dataInput;
+  // the ByteBuffers to read from in case the DataOutput has a single block
+  private ByteBuffer byteBuffer;
+
+  public ReadWriteDataOutput(ByteBuffersDataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    dataOutput.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) {
+    dataOutput.writeBytes(b, offset, length);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return dataOutput.ramBytesUsed();
+  }
+
+  @Override
+  public void freeze() {
+    // these operations are costly, so we want to compute it once and cache
+    List<ByteBuffer> byteBuffers = dataOutput.toWriteableBufferList();
+    if (byteBuffers.size() == 1) {

Review Comment:
   Even for ByteBuffersDataInput in multi-block cases, there's a slight regression over the BytesStore implementation (note that we already called `freeze()`). The diff is https://github.com/dungba88/lucene/pull/13/files.
   
   ### ReverseRandomBytesReader with ByteBuffersDataInput
   
   ```
     1> TEST: now verify [fst size=4621076364; nodeCount=2252341486; arcCount=2264078585]
     1> 0...: took 0 seconds
     1> 1000000...: took 27 seconds
     1> 2000000...: took 54 seconds
     1> 3000000...: took 82 seconds
     1> 4000000...: took 109 seconds
     1> 5000000...: took 137 seconds
     1> 6000000...: took 165 seconds
     1> 7000000...: took 192 seconds
     1> 8000000...: took 219 seconds
     1> 9000000...: took 247 seconds
     1> 10000000...: took 275 seconds
     1> 11000000...: took 300 seconds
   ```
   
   ### BytesStore-like BytesReader
   
   ```
     1> TEST: now verify [fst size=4621076364; nodeCount=2252341486; arcCount=2264078585]
     1> 0...: took 0 seconds
     1> 1000000...: took 22 seconds
     1> 2000000...: took 44 seconds
     1> 3000000...: took 66 seconds
     1> 4000000...: took 89 seconds
     1> 5000000...: took 111 seconds
     1> 6000000...: took 133 seconds
     1> 7000000...: took 155 seconds
     1> 8000000...: took 178 seconds
     1> 9000000...: took 200 seconds
     1> 10000000...: took 222 seconds
     1> 11000000...: took 245 seconds
   ```
   
   As this PR is already rather long, I can make the first implementation simple by solely using ByteBuffersDataInput, then open an issue to benchmark different alternatives thoroughly. WDYT?
   
   For this PR, I'll use the more simplified version by solely using ByteBuffersDataInput. I opened another one for the alternative way: https://github.com/apache/lucene/pull/12879. I did see improvement as shown in the log above.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand merged PR #12624:
URL: https://github.com/apache/lucene/pull/12624


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1415505550


##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -153,6 +180,40 @@ private FSTCompiler(
     }
   }
 
+  // Get the respective FSTReader of the DataOutput. If the DataOutput is also a FSTReader then we
+  // will use it, otherwise we will return a NullFSTReader. Attempting to read from a FST with
+  // NullFSTReader
+  // will throw UnsupportedOperationException
+  private FSTReader toFSTReader(DataOutput dataOutput) {
+    if (dataOutput instanceof FSTReader) {
+      return (FSTReader) dataOutput;
+    }
+    return NULL_FST_READER;
+  }
+
+  /**
+   * This class is used for FST backed by non-FSTReader DataOutput. It does not allow getting the
+   * reverse BytesReader nor writing to a DataOutput.
+   */
+  private static final class NullFSTReader implements FSTReader {
+
+    @Override
+    public long ramBytesUsed() {
+      return 0;
+    }
+
+    @Override
+    public FST.BytesReader getReverseBytesReader() {
+      throw new UnsupportedOperationException(
+          "NullFSTReader does not support getReverseBytesReader()");
+    }
+
+    @Override
+    public void writeTo(DataOutput out) {
+      throw new UnsupportedOperationException("NullFSTReader does not support writeTo(DataOutput)");

Review Comment:
   Could we improve this message, e.g. something like `FST was not constructed with getOnHeapReaderWriter` or so?  `NullFSTReader` isn't something the user should even know about (it's a private class implementation detail).



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -218,13 +279,19 @@ public Builder<T> allowFixedLengthArcs(boolean allowFixedLengthArcs) {
     }
 
     /**
-     * How many bits wide to make each byte[] block in the BytesStore; if you know the FST will be
-     * large then make this larger. For example 15 bits = 32768 byte pages.
+     * Set the {@link DataOutput} which is used for low-level writing of FST. If you want the FST to
+     * be immediately readable, you need to use a DataOutput that also implements {@link FSTReader},

Review Comment:
   Does `FSTReader` need to be public and known to users?  Could we make it package private and change this to say "you need to use `FSTCompiler#getOnHeapReaderWriter`"?



##########
lucene/test-framework/src/java/org/apache/lucene/tests/util/fst/FSTTester.java:
##########
@@ -316,6 +313,15 @@ public FST<T> doTest() throws IOException {
     return fst;
   }
 
+  protected FST<T> compile(FSTCompiler<T> fstCompiler) throws IOException {
+    return fstCompiler.compile();
+  }
+
+  protected FSTCompiler.Builder<T> getFSTBuilder() {
+    return new FSTCompiler.Builder<>(

Review Comment:
   Could we randomize whether the "on disk" vs "on heap" `DataOutput` is used, so that `TestFSTs` randomly picks?  We should randomize all three posibilities:
     * Create & use on-heap FST
     * Create on heap, save to disk, load FST from disk
     * Stream to disk, then load FST from disk



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -165,7 +226,7 @@ public static class Builder<T> {
     private final Outputs<T> outputs;
     private double suffixRAMLimitMB = 32.0;
     private boolean allowFixedLengthArcs = true;
-    private int bytesPageBits = 15;
+    private DataOutput dataOutput = getOnHeapReaderWriter(15);

Review Comment:
   Hmm can we avoid even creating this, unless the caller didn't pass a `Builder.dataOutput` themselves?



##########
lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java:
##########
@@ -120,22 +125,44 @@ public class FSTCompiler<T> {
   final float directAddressingMaxOversizingFactor;
   long directAddressingExpansionCredit;
 
-  final BytesStore bytes;
+  // the DataOutput to stream the FST bytes to
+  final DataOutput dataOutput;
+
+  // buffer to store bytes for the one node we are currently writing
+  final GrowableByteArrayDataOutput scratchBytes = new GrowableByteArrayDataOutput();
+
+  private long numBytesWritten;
+
+  /**
+   * Get an on-heap DataOutput that allows the FST to be read immediately after writing.

Review Comment:
   Add `, and also optionally saved to an external DataOutput`?
   
   I.e. with these changes we support these possible FST workflows:
     * Build FST and use it immediately entirely in RAM and then discard it
     * Build FST and use it immediately entirely in RAM and also save it to disk (any other `DataOutput`), and load it later and use it
     * Build FST but stream it immediately to disk (except the `FSTMetaData`, saved at the end) and you cannot use it when done unless you go and open your own `DataInput` on the backing store and make a new FST from that
   
   Could we include this enumeration in `FSTCompiler''s class javadocs?
   
   Also: could you update `Test2BFSTs` to also test the 3rd bullet above?  Right now it only tests the first 2 bullets.  Actually, maybe create a new test, `Test2BFSTsToDisk` or so?  That way we can limit heap size of that new test and confirm RAM is truly capped.  That last bullet fully caps the RAM usage during compilation, yay!



##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader, Freezable {

Review Comment:
   Could we get rid of `Freezable` and instead just cast to `ReadWriteDataOutput` and call its `.freeze()` method?  We have only this concrete, package private class implementing this interface?



##########
lucene/core/src/test/org/apache/lucene/util/fst/TestFSTDataOutputWriter.java:
##########
@@ -0,0 +1,230 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import static org.apache.lucene.tests.util.fst.FSTTester.toIntsRef;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.tests.store.MockDirectoryWrapper;
+import org.apache.lucene.tests.util.LuceneTestCase;
+import org.apache.lucene.tests.util.TestUtil;
+import org.apache.lucene.tests.util.fst.FSTTester;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
+
+public class TestFSTDataOutputWriter extends LuceneTestCase {
+
+  private MockDirectoryWrapper dir;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    dir = newMockDirectory();
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    // can be null if we force simpletext (funky, some kind of bug in test runner maybe)
+    if (dir != null) {
+      dir.close();
+    }
+    super.tearDown();
+  }
+
+  public void testRandom() throws Exception {
+
+    final int iters = atLeast(10);
+    final int maxBytes = TEST_NIGHTLY ? 200000 : 20000;
+    for (int iter = 0; iter < iters; iter++) {
+      final int numBytes = TestUtil.nextInt(random(), 1, maxBytes);
+      final byte[] expected = new byte[numBytes];
+      final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      final DataOutput dataOutput = new OutputStreamDataOutput(baos);

Review Comment:
   Hmm what is this actually testing?  It looks like it's testing `java.io.ByteArrayOutputStream` and our `OutputStreamDataOutput` wrapper on top?  It's not about FST storage at all?  Confused...
   
   Maybe move it to `OutputStreamDataOutput` test class?



##########
lucene/core/src/java/org/apache/lucene/util/fst/Freezable.java:
##########
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+/** Represent a datastructure that can be frozen (i.e., no longer modified). */
+interface Freezable {

Review Comment:
   Note that `FieldType` also implements this interface (it's own version).  But let's not declare that!  Keep this package private.



##########
lucene/core/src/java/org/apache/lucene/util/fst/FST.java:
##########
@@ -500,6 +502,12 @@ public FSTMetadata<T> getMetadata() {
     return metadata;
   }
 
+  /**
+   * Save the FST to DataOutput.
+   *
+   * @param metaOut the DataOutput to write the metadata to
+   * @param out the DataOutput to write the FST bytes to
+   */
   public void save(DataOutput metaOut, DataOutput out) throws IOException {

Review Comment:
   One need not call this if they passed their own `IndexOutput` when creating the compiler?



##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader, Freezable {
+
+  private final ByteBuffersDataOutput dataOutput;
+  // the DataInput to read from in case the DataOutput has multiple blocks
+  private ByteBuffersDataInput dataInput;
+  // the ByteBuffers to read from in case the DataOutput has a single block
+  private ByteBuffer byteBuffer;
+
+  public ReadWriteDataOutput(ByteBuffersDataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    dataOutput.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) {
+    dataOutput.writeBytes(b, offset, length);

Review Comment:
   Can we throw an exception in all of these `write` methods if `.freeze()` was called already?  Let's act frozen!



##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader, Freezable {
+
+  private final ByteBuffersDataOutput dataOutput;
+  // the DataInput to read from in case the DataOutput has multiple blocks
+  private ByteBuffersDataInput dataInput;
+  // the ByteBuffers to read from in case the DataOutput has a single block
+  private ByteBuffer byteBuffer;
+
+  public ReadWriteDataOutput(ByteBuffersDataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    dataOutput.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) {
+    dataOutput.writeBytes(b, offset, length);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return dataOutput.ramBytesUsed();
+  }
+
+  @Override
+  public void freeze() {
+    // these operations are costly, so we want to compute it once and cache
+    List<ByteBuffer> byteBuffers = dataOutput.toWriteableBufferList();
+    if (byteBuffers.size() == 1) {

Review Comment:
   Hmm can we just call `.toDataInput()`?  I think it's already doing this "is it a single buffer" opto?  Or was there even poor performance in that case?



##########
lucene/core/src/java/org/apache/lucene/util/fst/Freezable.java:
##########
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+/** Represent a datastructure that can be frozen (i.e., no longer modified). */

Review Comment:
   `Represent` -> `Represents`



##########
lucene/core/src/java/org/apache/lucene/util/fst/GrowableByteArrayDataOutput.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
+
+// Storing a single contiguous byte[] for the current node of the FST we are writing. The byte[]
+// will only grow, never
+// shrink.

Review Comment:
   Can you further strengthen the comment, something like `This is only safe for usage that is bounded in the number of bytes written.  Do not make this public!  Public users should instead use ByteBuffersDataOutput` or so?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1418728841


##########
lucene/core/src/java/org/apache/lucene/util/fst/ReadWriteDataOutput.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader}. It allows the FST
+ * to be readable immediately after writing
+ */
+final class ReadWriteDataOutput extends DataOutput implements FSTReader, Freezable {
+
+  private final ByteBuffersDataOutput dataOutput;
+  // the DataInput to read from in case the DataOutput has multiple blocks
+  private ByteBuffersDataInput dataInput;
+  // the ByteBuffers to read from in case the DataOutput has a single block
+  private ByteBuffer byteBuffer;
+
+  public ReadWriteDataOutput(ByteBuffersDataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+
+  @Override
+  public void writeByte(byte b) {
+    dataOutput.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) {
+    dataOutput.writeBytes(b, offset, length);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return dataOutput.ramBytesUsed();
+  }
+
+  @Override
+  public void freeze() {
+    // these operations are costly, so we want to compute it once and cache
+    List<ByteBuffer> byteBuffers = dataOutput.toWriteableBufferList();
+    if (byteBuffers.size() == 1) {

Review Comment:
   > I did see improvement as shown in the log above.
   
   I compared 9.7.x and main + this PR (an earlier rev than the current one, the rev where we re-added `.freeze()`), running `Test2BFSTs`.  This PR is a bit faster at building (643 seconds vs 677 for the `NO_OUTPUTS` case), but slower at `get` (89 seconds vs 60 seconds), but there were other changes since 9.7.x so this is not a pure test.  Yet perhaps this `asReadOnly` wrapping (vs FST's dedicated `byte[]`-backed `BytesStore`) is indeed part of that slowdown.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1391973476


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -26,7 +26,8 @@
 // TODO: merge with PagedBytes, except PagedBytes doesn't
 // let you read while writing which FST needs
 
-class BytesStore extends DataOutput implements FSTReader {
+// TODO: Separate the scratch writer and reader functionality
+class BytesStore extends DataOutput implements FSTReader, Freezeable {

Review Comment:
   Now thinking again, I think even the ByteBuffersDataOutput would need freezing as well, otherwise its last block will always be a full block, and might be a waste of RAM. Freezable provides a chance to optimize those datastructure as they are not to be modified again.
   
   Anyhow I removed the interface and the method for now. It's easy to add back.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1391247632


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -337,11 +349,23 @@ public long size() {
     return getPosition();
   }
 
+  /** Similar to {@link #truncate(long)} with newLen=0 but keep the first block to reduce GC. */
+  public void reset() {

Review Comment:
   The public API of this class seems to only allow appending only, but for scratch area we still need to some backward writing right?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1392197630


##########
lucene/core/src/java/org/apache/lucene/util/fst/ByteBuffersFSTReader.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.fst;
+
+import java.io.IOException;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+
+/** An adapter class to use {@link ByteBuffersDataOutput} as a {@link FSTReader} */
+final class ByteBuffersFSTReader extends DataOutput implements FSTReader {

Review Comment:
   I'm nervous having the ByteBuffersDataOutput to implement FSTReader directly, hence the adapter class



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "mikemccand (via GitHub)" <gi...@apache.org>.
mikemccand commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1392647780


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,12 +21,13 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
-// TODO: merge with PagedBytes, except PagedBytes doesn't
-// let you read while writing which FST needs
+// TODO: merge with or use PagedBytes/ByteBuffersDataOutput, as we no longer need to read while
+// writing
 
-class BytesStore extends DataOutput implements FSTReader {
+class BytesStore extends DataOutput implements Accountable {

Review Comment:
   > Hmm where do we do this? I guess it is where we do in-place fixing of arc encoding after we've decided whether it should be sparse/dense/contiguous?
   
   Maybe we should make our own package-private `ByteArrayDataOutput` to use when writing one frozen node?  It just reallocates the `byte[]` as large as needed, and lets us directly access that `byte[]` and muck with its bytes, reverse things, etc.?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dweiss (via GitHub)" <gi...@apache.org>.
dweiss commented on PR #12624:
URL: https://github.com/apache/lucene/pull/12624#issuecomment-1819661097

   I've taken a look - I think this class should be kept package-private (or even class-private). It is a DataOutput but it serves very specific purposes (with the unusual methods to move the write pointer at random, expand the underlying buffer, etc.). 
   
   I also changed the implementation to use OutputStreamDataOutput [1] but it's not any more elegant than written explicitly, so please disregard that suggestion.
   
   [1] [baos.patch](https://github.com/apache/lucene/files/13418649/baos.patch)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1393547261


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -21,12 +21,13 @@
 import java.util.List;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 
-// TODO: merge with PagedBytes, except PagedBytes doesn't
-// let you read while writing which FST needs
+// TODO: merge with or use PagedBytes/ByteBuffersDataOutput, as we no longer need to read while
+// writing
 
-class BytesStore extends DataOutput implements FSTReader {
+class BytesStore extends DataOutput implements Accountable {

Review Comment:
   > Hmm where do we do this
   
   TBH I haven't gone through all of them, but one instance is when we write the arc in backward, another is when we first skip the header and after all is written we go back and write the header, presence bits and label.
   
   Anyhow I changed to a byte[] to simplify its implementation.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] Allow FST builder to use different writer (#12543) [lucene]

Posted by "dungba88 (via GitHub)" <gi...@apache.org>.
dungba88 commented on code in PR #12624:
URL: https://github.com/apache/lucene/pull/12624#discussion_r1395069004


##########
lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java:
##########
@@ -337,11 +349,23 @@ public long size() {
     return getPosition();
   }
 
+  /** Similar to {@link #truncate(long)} with newLen=0 but keep the first block to reduce GC. */
+  public void reset() {

Review Comment:
   Yeah it seems the main complexity is due to the fact that we need to override the previously written bytes, sometimes when we want to remove the labels, sometimes when we need to update the header after writing everything. So I guess making it forward would not eliminate those backward writing operation.
   
   I simplified the writing operation a bit in the direct addressing mode. Seems like we can convert it to append-only mode. But the binary search one seems to be difficult, as when we fix the arc we need to do in backward (I assume to avoid overriding?). Anyhow, the remaining backward operations are:
   - writeBytes: As mentioned, this is used in the binary search mode. If we first copy the whole buffer to a new one, we can start appending back to back. But this isn't as efficient as the current in-place copy? The whole method seems to be expanding each arc from variable arc length to fixed arc lengths. Anyhow there's only a single place this method is used, and the method is pretty much 1-liner, so I embed it to FSTCompiler.
   - reverse: This is only done at the end before writing to the DataOutput and the NodeHash.ByteBlockPool



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org