blob: c6b0426234bdb23a7e3ca98309efae4727e172c7 [file] [log] [blame]
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
index b0091fd..0076f14 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
@@ -102,6 +102,8 @@ public final class BlockTreeTermsReader extends FieldsProducer {
// Open input to the main terms dict file (_X.tib)
final IndexInput termsIn;
+ // Open input to the terms index file (_X.tip)
+ final IndexInput indexIn;
//private static final boolean DEBUG = BlockTreeTermsWriter.DEBUG;
@@ -118,7 +120,6 @@ public final class BlockTreeTermsReader extends FieldsProducer {
/** Sole constructor. */
public BlockTreeTermsReader(PostingsReaderBase postingsReader, SegmentReadState state) throws IOException {
boolean success = false;
- IndexInput indexIn = null;
this.postingsReader = postingsReader;
this.segment = state.segmentInfo.name;
@@ -197,13 +198,11 @@ public final class BlockTreeTermsReader extends FieldsProducer {
throw new CorruptIndexException("duplicate field: " + fieldInfo.name, termsIn);
}
}
-
- indexIn.close();
success = true;
} finally {
if (!success) {
// this.close() will close in:
- IOUtils.closeWhileHandlingException(indexIn, this);
+ IOUtils.closeWhileHandlingException(this);
}
}
}
@@ -237,7 +236,7 @@ public final class BlockTreeTermsReader extends FieldsProducer {
@Override
public void close() throws IOException {
try {
- IOUtils.close(termsIn, postingsReader);
+ IOUtils.close(indexIn, termsIn, postingsReader);
} finally {
// Clear so refs to terms index is GCable even if
// app hangs onto us:
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java
index 46aee6e..09b6078 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java
@@ -34,6 +34,7 @@ import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.OffHeapFSTStore;
/**
* BlockTree's implementation of {@link Terms}.
@@ -88,7 +89,7 @@ public final class FieldReader extends Terms implements Accountable {
final IndexInput clone = indexIn.clone();
//System.out.println("start=" + indexStartFP + " field=" + fieldInfo.name);
clone.seek(indexStartFP);
- index = new FST<>(clone, ByteSequenceOutputs.getSingleton());
+ index = new FST<>(clone, ByteSequenceOutputs.getSingleton(), new OffHeapFSTStore());
/*
if (false) {
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java b/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java
index 428edd3..91e46c5 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java
@@ -20,6 +20,7 @@ package org.apache.lucene.util.fst;
import java.io.IOException;
import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.IntsRefBuilder;
import org.apache.lucene.util.fst.FST.INPUT_TYPE; // javadoc
@@ -210,7 +211,6 @@ public class Builder<T> {
assert bytesPosEnd > bytesPosStart;
lastFrozenNode = node;
}
-
nodeIn.clear();
final CompiledNode fn = new CompiledNode();
@@ -344,9 +344,9 @@ public class Builder<T> {
}
b.length = input.length;
if (output == NO_OUTPUT) {
- System.out.println("\nFST ADD: input=" + toString(b) + " " + b);
+ System.out.println("\nFST ADD: input=" + b);
} else {
- System.out.println("\nFST ADD: input=" + toString(b) + " " + b + " output=" + fst.outputs.outputToString(output));
+ System.out.println("\nFST ADD: input=" + b + " output=" + fst.outputs.outputToString(output));
}
}
*/
@@ -480,7 +480,9 @@ public class Builder<T> {
}
}
//if (DEBUG) System.out.println(" builder.finish root.isFinal=" + root.isFinal + " root.output=" + root.output);
- fst.finish(compileNode(root, lastInput.length()).node);
+ long newStartNode = compileNode(root, lastInput.length()).node;
+ assert newStartNode == fst.bytes.getPosition() - 1 || newStartNode == -1;
+ fst.finish(newStartNode);
return fst;
}
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java b/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java
index 96f5732..f574717 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java
@@ -407,11 +407,6 @@ class BytesStore extends DataOutput implements Accountable {
nextRead = (int) (pos & blockMask);
assert getPosition() == pos;
}
-
- @Override
- public boolean reversed() {
- return false;
- }
};
}
@@ -466,11 +461,6 @@ class BytesStore extends DataOutput implements Accountable {
nextRead = (int) (pos & blockMask);
assert getPosition() == pos: "pos=" + pos + " getPos()=" + getPosition();
}
-
- @Override
- public boolean reversed() {
- return true;
- }
};
}
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/FST.java b/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
index 8e84e3c..4a0e48a 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
@@ -24,6 +24,8 @@ import java.io.InputStream;
import java.io.OutputStream;
import java.nio.file.Files;
import java.nio.file.Path;
+import java.util.Collection;
+import java.util.Collections;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.store.ByteArrayDataOutput;
@@ -116,6 +118,11 @@ public final class FST<T> implements Accountable {
/** If arc has this label then that arc is final/accepted */
public static final int END_LABEL = -1;
+ // FSTs are serialized in reverse order while building them, but un-reversed when written to disk,
+ // so finished FSTs, and those loaded from disk are in forward order (not reversed)
+ private boolean forward;
+ private long numBytes; // used to reverse arc target addresses
+
public final INPUT_TYPE inputType;
// if non-null, this FST accepts the empty string and
@@ -127,8 +134,7 @@ public final class FST<T> implements Accountable {
* GB then bytesArray is set instead. */
final BytesStore bytes;
- /** Used at read time when the FST fits into a single byte[]. */
- final byte[] bytesArray;
+ private final FSTStore fstStore;
private long startNode = -1;
@@ -238,30 +244,30 @@ public final class FST<T> implements Accountable {
this.inputType = inputType;
this.outputs = outputs;
version = VERSION_CURRENT;
- bytesArray = null;
+ fstStore = null;
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);
emptyOutput = null;
+ forward = false;
}
public static final int DEFAULT_MAX_BLOCK_BITS = Constants.JRE_IS_64BIT ? 30 : 28;
/** Load a previously saved FST. */
public FST(DataInput in, Outputs<T> outputs) throws IOException {
- this(in, outputs, DEFAULT_MAX_BLOCK_BITS);
+ this(in, outputs, new OnHeapFSTStore(DEFAULT_MAX_BLOCK_BITS));
}
/** Load a previously saved FST; maxBlockBits allows you to
* control the size of the byte[] pages used to hold the FST bytes. */
- public FST(DataInput in, Outputs<T> outputs, int maxBlockBits) throws IOException {
+ public FST(DataInput in, Outputs<T> outputs, FSTStore fstStore) throws IOException {
+ bytes = null;
+ this.fstStore = fstStore;
this.outputs = outputs;
-
- if (maxBlockBits < 1 || maxBlockBits > 30) {
- throw new IllegalArgumentException("maxBlockBits should be 1 .. 30; got " + maxBlockBits);
- }
+ forward = true;
// NOTE: only reads most recent format; we don't have
// back-compat promise for FSTs (they are experimental):
@@ -274,13 +280,7 @@ public final class FST<T> implements Accountable {
emptyBytes.copyBytes(in, numBytes);
// De-serialize empty-string output:
- BytesReader reader = emptyBytes.getReverseReader();
- // NoOutputs uses 0 bytes when writing its output,
- // so we have to check here else BytesStore gets
- // angry:
- if (numBytes > 0) {
- reader.setPosition(numBytes-1);
- }
+ BytesReader reader = emptyBytes.getForwardReader();
emptyOutput = outputs.readFinalOutput(reader);
} else {
emptyOutput = null;
@@ -300,26 +300,11 @@ public final class FST<T> implements Accountable {
throw new IllegalStateException("invalid input type " + t);
}
startNode = in.readVLong();
-
- long numBytes = in.readVLong();
- if (numBytes > 1 << maxBlockBits) {
- // FST is big: we need multiple pages
- bytes = new BytesStore(in, numBytes, 1<<maxBlockBits);
- bytesArray = null;
- } else {
- // FST fits into a single block: use ByteArrayBytesStoreReader for less overhead
- bytes = null;
- bytesArray = new byte[(int) numBytes];
- in.readBytes(bytesArray, 0, bytesArray.length);
- }
-
+ numBytes = in.readVLong();
+ this.fstStore.init(in, numBytes);
cacheRootArcs();
}
- public INPUT_TYPE getInputType() {
- return inputType;
- }
-
private long ramBytesUsed(Arc<T>[] arcs) {
long size = 0;
if (arcs != null) {
@@ -344,11 +329,12 @@ public final class FST<T> implements Accountable {
@Override
public long ramBytesUsed() {
long size = BASE_RAM_BYTES_USED;
- if (bytesArray != null) {
- size += bytesArray.length;
+ if (this.fstStore != null) {
+ size += this.fstStore.ramBytesUsed();
} else {
size += bytes.ramBytesUsed();
}
+
size += cachedArcsBytesUsed;
return size;
}
@@ -358,16 +344,28 @@ public final class FST<T> implements Accountable {
return getClass().getSimpleName() + "(input=" + inputType + ",output=" + outputs;
}
+ @Override
+ public Collection<Accountable> getChildResources() {
+ return Collections.emptySet();
+ }
+
void finish(long newStartNode) throws IOException {
assert newStartNode <= bytes.getPosition();
if (startNode != -1) {
throw new IllegalStateException("already finished");
}
if (newStartNode == FINAL_END_NODE && emptyOutput != null) {
- newStartNode = 0;
+ startNode = 0;
+ } else {
+ startNode = bytes.getPosition() - newStartNode;
}
- startNode = newStartNode;
bytes.finish();
+ // nocommit: expensive, but let's see if it helps us at query time
+ numBytes = bytes.getPosition();
+ if (numBytes > 2) {
+ bytes.reverse(1, numBytes - 1);
+ }
+ forward = true;
cacheRootArcs();
}
@@ -436,15 +434,6 @@ public final class FST<T> implements Accountable {
outputs.writeFinalOutput(emptyOutput, ros);
byte[] emptyOutputBytes = ros.toArrayCopy();
- // reverse
- final int stopAt = emptyOutputBytes.length/2;
- int upto = 0;
- while (upto < stopAt) {
- final byte b = emptyOutputBytes[upto];
- emptyOutputBytes[upto] = emptyOutputBytes[emptyOutputBytes.length-upto-1];
- emptyOutputBytes[emptyOutputBytes.length-upto-1] = b;
- upto++;
- }
out.writeVInt(emptyOutputBytes.length);
out.writeBytes(emptyOutputBytes, 0, emptyOutputBytes.length);
} else {
@@ -465,9 +454,8 @@ public final class FST<T> implements Accountable {
out.writeVLong(numBytes);
bytes.writeTo(out);
} else {
- assert bytesArray != null;
- out.writeVLong(bytesArray.length);
- out.writeBytes(bytesArray, 0, bytesArray.length);
+ assert fstStore != null;
+ fstStore.writeTo(out);
}
}
@@ -523,8 +511,8 @@ public final class FST<T> implements Accountable {
return arc.target > 0;
}
- // serializes new node by appending its bytes to the end
- // of the current byte[]
+ // serializes new node by appending its bytes to the end of the current byte[]
+ // for each arc: { byte -- flags, VInt -- label, output?, final-output?, target.node (unless implicitly next node) }
long addNode(Builder<T> builder, Builder.UnCompiledNode<T> nodeIn) throws IOException {
T NO_OUTPUT = outputs.getNoOutput();
@@ -683,11 +671,10 @@ public final class FST<T> implements Accountable {
builder.bytes.writeBytes(startAddress, header, 0, headerLen);
}
- final long thisNodeAddress = builder.bytes.getPosition()-1;
+ builder.nodeCount++;
+ final long thisNodeAddress = builder.bytes.getPosition()-1;
builder.bytes.reverse(startAddress, thisNodeAddress);
-
- builder.nodeCount++;
return thisNodeAddress;
}
@@ -807,8 +794,7 @@ public final class FST<T> implements Accountable {
public Arc<T> readFirstRealTargetArc(long node, Arc<T> arc, final BytesReader in) throws IOException {
final long address = node;
in.setPosition(address);
- //System.out.println(" readFirstRealTargtArc address="
- //+ address);
+ //System.out.println(" readFirstRealTargetArc address=" + address);
//System.out.println(" flags=" + arc.flags);
if (in.readByte() == ARCS_AS_FIXED_ARRAY) {
@@ -818,7 +804,7 @@ public final class FST<T> implements Accountable {
arc.bytesPerArc = in.readVInt();
arc.arcIdx = -1;
arc.nextArc = arc.posArcsStart = in.getPosition();
- //System.out.println(" bytesPer=" + arc.bytesPerArc + " numArcs=" + arc.numArcs + " arcsStart=" + pos);
+ // System.out.println(" bytesPer=" + arc.bytesPerArc + " numArcs=" + arc.numArcs + " arcsStart=" + arc.posArcsStart);
} else {
//arc.flags = b;
arc.nextArc = address;
@@ -951,6 +937,9 @@ public final class FST<T> implements Accountable {
arc.target = in.getPosition();
} else {
arc.target = readUnpackedNodeTarget(in);
+ if (forward && arc.target > 0) {
+ arc.target = numBytes - arc.target; // minimum value is 1
+ }
arc.nextArc = in.getPosition();
}
return arc;
@@ -1137,10 +1126,10 @@ public final class FST<T> implements Accountable {
/** Returns a {@link BytesReader} for this FST, positioned at
* position 0. */
public BytesReader getBytesReader() {
- if (bytesArray != null) {
- return new ReverseBytesReader(bytesArray);
+ if (this.fstStore != null) {
+ return this.fstStore.getBytesReader();
} else {
- return bytes.getReverseReader();
+ return bytes.getForwardReader();
}
}
@@ -1151,10 +1140,6 @@ public final class FST<T> implements Accountable {
/** Set current read position. */
public abstract void setPosition(long pos);
-
- /** Returns true if this reader uses reversed bytes
- * under-the-hood. */
- public abstract boolean reversed();
}
/*
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/FSTStore.java b/lucene/core/src/java/org/apache/lucene/util/fst/FSTStore.java
new file mode 100644
index 0000000..517a0d6
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/FSTStore.java
@@ -0,0 +1,32 @@
+/*
+ * 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 org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.Accountable;
+
+import java.io.IOException;
+
+public interface FSTStore extends Accountable {
+
+ void init(DataInput in, long numBytes) throws IOException;
+
+ FST.BytesReader getBytesReader();
+
+ void writeTo(DataOutput out) throws IOException;
+}
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/ForwardBytesReader.java b/lucene/core/src/java/org/apache/lucene/util/fst/ForwardBytesReader.java
index 9d450b8..809e459 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/ForwardBytesReader.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/ForwardBytesReader.java
@@ -54,9 +54,4 @@ final class ForwardBytesReader extends FST.BytesReader {
public void setPosition(long pos) {
this.pos = (int) pos;
}
-
- @Override
- public boolean reversed() {
- return false;
- }
}
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/IndexInputReader.java b/lucene/core/src/java/org/apache/lucene/util/fst/IndexInputReader.java
new file mode 100644
index 0000000..0ebc445
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/IndexInputReader.java
@@ -0,0 +1,64 @@
+/*
+ * 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.IndexInput;
+
+/** Implements FST.BytesReader read from an IndexInput. */
+final class IndexInputReader extends FST.BytesReader {
+ private final IndexInput in;
+ private long offset;
+
+ public IndexInputReader(IndexInput in, long offset) {
+ this.in = in.clone();
+ this.offset = offset;
+ }
+
+ @Override
+ public byte readByte() throws IOException {
+ return in.readByte();
+ }
+
+ @Override
+ public void readBytes(byte[] b, int offset, int len) throws IOException {
+ in.readBytes(b, offset, len);
+ }
+
+ @Override
+ public void skipBytes(long count) {
+ try {
+ in.seek(in.getFilePointer() + count);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public long getPosition() {
+ return in.getFilePointer() - offset;
+ }
+
+ @Override
+ public void setPosition(long pos) {
+ try {
+ in.seek(pos + offset);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java b/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java
index 6704cf6..284d76c 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java
@@ -23,6 +23,7 @@ import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PagedGrowableWriter;
// Used to dedup states (lookup already-frozen states)
+// The hash of a node incorporates the arc's label, target, output, final-output, and is-final flag
final class NodeHash<T> {
private PagedGrowableWriter table;
@@ -110,7 +111,7 @@ final class NodeHash<T> {
}
fst.readNextRealArc(scratchArc, in);
}
- //System.out.println(" ret " + (h&Integer.MAX_VALUE));
+ // System.out.println(" ret " + (h&Integer.MAX_VALUE));
return h & Long.MAX_VALUE;
}
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/OffHeapFSTStore.java b/lucene/core/src/java/org/apache/lucene/util/fst/OffHeapFSTStore.java
new file mode 100644
index 0000000..caed2e7
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/OffHeapFSTStore.java
@@ -0,0 +1,66 @@
+/*
+ * 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 org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.RandomAccessInput;
+import org.apache.lucene.util.RamUsageEstimator;
+
+import java.io.IOException;
+
+/** Provides off heap storage of finite state machine (FST),
+ * using underlying index input instead of byte store on heap
+ *
+ * @lucene.experimental
+ */
+public final class OffHeapFSTStore implements FSTStore {
+
+ private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(OffHeapFSTStore.class);
+
+ private IndexInput in;
+ private long offset;
+ private long numBytes;
+
+ @Override
+ public void init(DataInput in, long numBytes) throws IOException {
+ if (in instanceof IndexInput) {
+ this.in = (IndexInput) in;
+ this.numBytes = numBytes;
+ this.offset = this.in.getFilePointer();
+ } else {
+ throw new IllegalArgumentException("parameter:in should be an instance of RandomAccessInput for using OffHeapFSTStore, not a "
+ + in.getClass().getName());
+ }
+ }
+
+ @Override
+ public long ramBytesUsed() {
+ return BASE_RAM_BYTES_USED;
+ }
+
+ @Override
+ public FST.BytesReader getBytesReader() {
+ return new IndexInputReader(in, offset);
+ }
+
+ @Override
+ public void writeTo(DataOutput out) throws IOException {
+ throw new UnsupportedOperationException("writeToOutput operation is not supported for OffHeapFSTStore");
+ }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/OnHeapFSTStore.java b/lucene/core/src/java/org/apache/lucene/util/fst/OnHeapFSTStore.java
new file mode 100644
index 0000000..4a3d8ae
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/OnHeapFSTStore.java
@@ -0,0 +1,97 @@
+/*
+ * 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 org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.RamUsageEstimator;
+
+import java.io.IOException;
+
+/** Provides storage of finite state machine (FST),
+ * using byte array or byte store allocated on heap.
+ *
+ * @lucene.experimental
+ */
+public final class OnHeapFSTStore implements FSTStore {
+
+ private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(OnHeapFSTStore.class);
+
+ /** 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. */
+ private BytesStore bytes;
+
+ /** Used at read time when the FST fits into a single byte[]. */
+ private byte[] bytesArray;
+
+ private final int maxBlockBits;
+
+ public OnHeapFSTStore(int maxBlockBits) {
+ if (maxBlockBits < 1 || maxBlockBits > 30) {
+ throw new IllegalArgumentException("maxBlockBits should be 1 .. 30; got " + maxBlockBits);
+ }
+
+ this.maxBlockBits = maxBlockBits;
+ }
+
+ @Override
+ public void init(DataInput in, long numBytes) throws IOException {
+ if (numBytes > 1 << this.maxBlockBits) {
+ // FST is big: we need multiple pages
+ bytes = new BytesStore(in, numBytes, 1<<this.maxBlockBits);
+ } else {
+ // FST fits into a single block: use ByteArrayBytesStoreReader for less overhead
+ bytesArray = new byte[(int) numBytes];
+ in.readBytes(bytesArray, 0, bytesArray.length);
+ }
+ }
+
+ @Override
+ public long ramBytesUsed() {
+ long size = BASE_RAM_BYTES_USED;
+ if (bytesArray != null) {
+ size += bytesArray.length;
+ } else {
+ size += bytes.ramBytesUsed();
+ }
+
+ return size;
+ }
+
+ @Override
+ public FST.BytesReader getBytesReader() {
+ if (bytesArray != null) {
+ return new ForwardBytesReader(bytesArray);
+ } else {
+ return bytes.getReverseReader();
+ }
+ }
+
+ @Override
+ public void writeTo(DataOutput out) throws IOException {
+ if (bytes != null) {
+ long numBytes = bytes.getPosition();
+ out.writeVLong(numBytes);
+ bytes.writeTo(out);
+ } else {
+ assert bytesArray != null;
+ out.writeVLong(bytesArray.length);
+ out.writeBytes(bytesArray, 0, bytesArray.length);
+ }
+ }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/ReverseBytesReader.java b/lucene/core/src/java/org/apache/lucene/util/fst/ReverseBytesReader.java
index 01341cf..f98f5b6 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/ReverseBytesReader.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/ReverseBytesReader.java
@@ -52,10 +52,5 @@ final class ReverseBytesReader extends FST.BytesReader {
public void setPosition(long pos) {
this.pos = (int) pos;
}
-
- @Override
- public boolean reversed() {
- return true;
- }
}
diff --git a/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java b/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java
index ce5a52b..f735fae 100644
--- a/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java
+++ b/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java
@@ -247,7 +247,6 @@ public class TestBytesStore extends LuceneTestCase {
}
// reversed
FST.BytesReader r = bytes.getReverseReader();
- assertTrue(r.reversed());
r.setPosition(totalLength-1);
r.readBytes(actual, 0, actual.length);
int start = 0;
@@ -265,7 +264,6 @@ public class TestBytesStore extends LuceneTestCase {
System.out.println(" bulk: forward");
}
FST.BytesReader r = bytes.getForwardReader();
- assertFalse(r.reversed());
r.readBytes(actual, 0, actual.length);
}
diff --git a/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java b/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
index d882fae..3bc8c74 100644
--- a/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
+++ b/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
@@ -53,6 +53,8 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.ByteArrayDataOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.IOContext;
@@ -263,7 +265,6 @@ public class TestFSTs extends LuceneTestCase {
}
-
public void testRandomWords() throws IOException {
if (TEST_NIGHTLY) {
testRandomWords(1000, atLeast(2));
@@ -303,7 +304,44 @@ public class TestFSTs extends LuceneTestCase {
public void testBigSet() throws IOException {
testRandomWords(TestUtil.nextInt(random(), 50000, 60000), 1);
}
-
+
+ // test some degenerate cases
+ public void testEmpty() throws IOException {
+ Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, PositiveIntOutputs.getSingleton());
+ assertNull(builder.finish());
+ builder = new Builder<>(FST.INPUT_TYPE.BYTE1, PositiveIntOutputs.getSingleton());
+ builder.add(new IntsRef(), 7L);
+ FST<Long> fst = builder.finish();
+ assertNotNull(fst);
+ assertEquals(7L, (long) Util.get(fst, new IntsRef()));
+ }
+
+ // Simple test to make sure we save and restore properly
+ public void testReadWrite() throws IOException {
+ Builder<Long> builder = new Builder<>(FST.INPUT_TYPE.BYTE1, PositiveIntOutputs.getSingleton());
+ Set<IntsRef> words = new HashSet<>();
+ if (random().nextBoolean()) {
+ words.add(new IntsRef()); // include empty
+ }
+ for (int i = 0; i < atLeast(100); i++) {
+ IntsRef word = toIntsRef(getRandomString(random()), 0);
+ words.add(word);
+ }
+ List<IntsRef> wordList = new ArrayList<>(words);
+ Collections.sort(wordList);
+ for (int i = 0; i < wordList.size(); i++) {
+ builder.add(wordList.get(i), (long) i);
+ }
+ FST<Long> fst = builder.finish();
+ byte[] buf = new byte[(int) fst.bytes.getPosition() + 100];
+ ByteArrayDataOutput bado = new ByteArrayDataOutput(buf);
+ fst.save(bado);
+ fst = new FST<>(new ByteArrayDataInput(buf), PositiveIntOutputs.getSingleton());
+ for (IntsRef word : words) {
+ assertNotNull(Util.get(fst, word));
+ }
+ }
+
// Build FST for all unique terms in the test line docs
// file, up until a doc limit
public void testRealTerms() throws Exception {
@@ -988,7 +1026,7 @@ public class TestFSTs extends LuceneTestCase {
}
final Set<String> allTerms = new HashSet<>();
- while(allTerms.size() < NUM_TERMS) {
+ while(allTerms.size() < 10000) {
allTerms.add(simpleRandomString(random()));
}
@@ -1001,6 +1039,7 @@ public class TestFSTs extends LuceneTestCase {
if (VERBOSE) {
System.out.println("TEST: get reader");
}
+ w.commit();
IndexReader r = w.getReader();
if (VERBOSE) {
System.out.println("TEST: got reader=" + r);
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java b/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java
index 8e6a4ea..7d2b417 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java
@@ -268,18 +268,25 @@ public class FSTTester<T> {
FST<T> doTest(int prune1, int prune2, boolean allowRandomSuffixSharing) throws IOException {
+ boolean shareSuffixes = allowRandomSuffixSharing ? random.nextBoolean() : true;
+ int suffixSharingLimit = allowRandomSuffixSharing ? TestUtil.nextInt(random, 1, 10) : Integer.MAX_VALUE;
+
if (LuceneTestCase.VERBOSE) {
- System.out.println("\nTEST: prune1=" + prune1 + " prune2=" + prune2);
+ System.out.println("\nTEST: prune1=" + prune1 + " prune2=" + prune2 + " shareSuffixes=" + shareSuffixes + " suffixSharingLimit=" + suffixSharingLimit + ", inputMode=" + inputMode);
+ System.out.println(" expected input/output pairs:");
+ for(InputOutput<T> pair : pairs) {
+ System.out.println(" " + pair.input + " -> " + pair.output);
+ }
}
final Builder<T> builder = new Builder<>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4,
- prune1, prune2,
- prune1==0 && prune2==0,
- allowRandomSuffixSharing ? random.nextBoolean() : true,
- allowRandomSuffixSharing ? TestUtil.nextInt(random, 1, 10) : Integer.MAX_VALUE,
- outputs,
- true,
- 15);
+ prune1, prune2,
+ prune1==0 && prune2==0,
+ shareSuffixes,
+ suffixSharingLimit,
+ outputs,
+ true,
+ 15);
for(InputOutput<T> pair : pairs) {
if (pair.output instanceof List) {
@@ -388,18 +395,22 @@ public class FSTTester<T> {
}
{
IntsRefFSTEnum<T> fstEnum = new IntsRefFSTEnum<>(fst);
+ int i = 0;
for(InputOutput<T> pair : pairs) {
+ ++i;
IntsRef term = pair.input;
if (LuceneTestCase.VERBOSE) {
System.out.println("TEST: check term=" + inputToString(inputMode, term) + " output=" + fst.outputs.outputToString(pair.output));
}
T output = run(fst, term, null);
assertNotNull("term " + inputToString(inputMode, term) + " is not accepted", output);
- assertTrue(outputsEqual(pair.output, output));
+ assertEquals("unexpected output for term " + term, pair.output, output);
// verify enum's next
IntsRefFSTEnum.InputOutput<T> t = fstEnum.next();
- assertNotNull(t);
+ if (t == null) {
+ assertNotNull("got null for input [" + i + "] = " + inputToString(inputMode, term), t);
+ }
assertEquals("expected input=" + inputToString(inputMode, term) + " but fstEnum returned " + inputToString(inputMode, t.input), term, t.input);
assertTrue(outputsEqual(pair.output, t.output));
}
@@ -783,9 +794,9 @@ public class FSTTester<T> {
assertTrue(cmo.isLeaf || cmo.isFinal);
//if (cmo.isFinal && !cmo.isLeaf) {
if (cmo.isFinal) {
- assertEquals(cmo.finalOutput, current.output);
+ assertEquals("unexpected final output for input " + current.input, cmo.finalOutput, current.output);
} else {
- assertEquals(cmo.output, current.output);
+ assertEquals("unexpected output for input " + current.input, cmo.output, current.output);
}
}