You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2013/01/14 19:54:24 UTC
svn commit: r1433035 [2/4] - in /lucene/dev/branches/lucene4547: ./
dev-tools/ dev-tools/scripts/ lucene/ lucene/analysis/
lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/dict/
lucene/analysis/kuromoji/src/resources/org/apache/lucene/an...
Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/FST.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/FST.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/FST.java Mon Jan 14 18:54:22 2013
@@ -27,8 +27,14 @@ import java.io.InputStream;
import java.io.OutputStream;
import java.util.HashMap;
import java.util.Map;
+/*
+import java.io.Writer;
+import java.io.OutputStreamWriter;
+import java.io.FileOutputStream;
+*/
import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.store.ByteArrayDataOutput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.InputStreamDataInput;
@@ -51,9 +57,6 @@ import org.apache.lucene.util.packed.Pac
// job, ie, once we are at a 'suffix only', just store the
// completion labels as a string not as a series of arcs.
-// TODO: maybe make an explicit thread state that holds
-// reusable stuff eg BytesReader, a scratch arc
-
// NOTE: while the FST is able to represent a non-final
// dead-end state (NON_FINAL_END_NODE=0), the layers above
// (FSTEnum, Util) have problems with this!!
@@ -65,8 +68,6 @@ import org.apache.lucene.util.packed.Pac
*
* <p> See the {@link org.apache.lucene.util.fst package
* documentation} for some simple examples.
- * <p><b>NOTE</b>: the FST cannot be larger than ~2.1 GB
- * because it uses int to address the byte[].
*
* @lucene.experimental
*/
@@ -93,6 +94,8 @@ public final class FST<T> {
// position:
private final static int BIT_TARGET_DELTA = 1 << 6;
+ // We use this as a marker (because this one flag is
+ // illegal by itself ...):
private final static byte ARCS_AS_FIXED_ARRAY = BIT_ARC_HAS_FINAL_OUTPUT;
/**
@@ -125,24 +128,27 @@ public final class FST<T> {
/** Added optional packed format. */
private final static int VERSION_PACKED = 3;
- private final static int VERSION_CURRENT = VERSION_PACKED;
+ /** Changed from int to vInt for encoding arc targets.
+ * Also changed maxBytesPerArc from int to vInt in the array case. */
+ private final static int VERSION_VINT_TARGET = 4;
+
+ private final static int VERSION_CURRENT = VERSION_VINT_TARGET;
// Never serialized; just used to represent the virtual
// final node w/ no arcs:
- private final static int FINAL_END_NODE = -1;
+ private final static long FINAL_END_NODE = -1;
// Never serialized; just used to represent the virtual
// non-final node w/ no arcs:
- private final static int NON_FINAL_END_NODE = 0;
+ private final static long NON_FINAL_END_NODE = 0;
// if non-null, this FST accepts the empty string and
// produces this output
T emptyOutput;
- // Not private to avoid synthetic access$NNN methods:
- byte[] bytes;
+ final BytesStore bytes;
- private int startNode = -1;
+ private long startNode = -1;
public final Outputs<T> outputs;
@@ -150,13 +156,13 @@ public final class FST<T> {
// instead of storing the address of the target node for
// a given arc, we mark a single bit noting that the next
// node in the byte[] is the target node):
- private int lastFrozenNode;
+ private long lastFrozenNode;
private final T NO_OUTPUT;
- public int nodeCount;
- public int arcCount;
- public int arcWithOutputCount;
+ public long nodeCount;
+ public long arcCount;
+ public long arcWithOutputCount;
private final boolean packed;
private PackedInts.Reader nodeRefToAddress;
@@ -175,19 +181,19 @@ public final class FST<T> {
// From node (ord or address); currently only used when
// building an FST w/ willPackFST=true:
- int node;
+ long node;
/** To node (ord or address) */
- public int target;
+ public long target;
byte flags;
public T nextFinalOutput;
// address (into the byte[]), or ord/address if label == END_LABEL
- int nextArc;
+ long nextArc;
// This is non-zero if current arcs are fixed array:
- int posArcsStart;
+ long posArcsStart;
int bytesPerArc;
int arcIdx;
int numArcs;
@@ -254,8 +260,6 @@ public final class FST<T> {
return (flags & bit) != 0;
}
- private final BytesWriter writer;
-
private GrowableWriter nodeAddress;
// TODO: we could be smarter here, and prune periodically
@@ -263,23 +267,28 @@ public final class FST<T> {
// clear early on:
private GrowableWriter inCounts;
+ private final int version;
+
// make a new empty FST, for building; Builder invokes
// this ctor
- FST(INPUT_TYPE inputType, Outputs<T> outputs, boolean willPackFST, float acceptableOverheadRatio, boolean allowArrayArcs) {
+ FST(INPUT_TYPE inputType, Outputs<T> outputs, boolean willPackFST, float acceptableOverheadRatio, boolean allowArrayArcs, int bytesPageBits) {
this.inputType = inputType;
this.outputs = outputs;
this.allowArrayArcs = allowArrayArcs;
- bytes = new byte[128];
+ version = VERSION_CURRENT;
+ // 32 KB blocks:
+ 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);
NO_OUTPUT = outputs.getNoOutput();
if (willPackFST) {
- nodeAddress = new GrowableWriter(PackedInts.bitsRequired(bytes.length - 1), 8, acceptableOverheadRatio);
+ nodeAddress = new GrowableWriter(15, 8, acceptableOverheadRatio);
inCounts = new GrowableWriter(1, 8, acceptableOverheadRatio);
} else {
nodeAddress = null;
inCounts = null;
}
-
- writer = new DefaultBytesWriter();
emptyOutput = null;
packed = false;
@@ -289,23 +298,29 @@ public final class FST<T> {
/** Load a previously saved FST. */
public FST(DataInput in, Outputs<T> outputs) throws IOException {
this.outputs = outputs;
- writer = null;
// NOTE: only reads most recent format; we don't have
// back-compat promise for FSTs (they are experimental):
- CodecUtil.checkHeader(in, FILE_FORMAT_NAME, VERSION_PACKED, VERSION_PACKED);
+ version = CodecUtil.checkHeader(in, FILE_FORMAT_NAME, VERSION_PACKED, VERSION_VINT_TARGET);
packed = in.readByte() == 1;
if (in.readByte() == 1) {
// accepts empty string
+ // 1 KB blocks:
+ BytesStore emptyBytes = new BytesStore(10);
int numBytes = in.readVInt();
- bytes = new byte[numBytes];
- in.readBytes(bytes, 0, numBytes);
-
+ emptyBytes.copyBytes(in, numBytes);
+
// De-serialize empty-string output:
BytesReader reader;
if (packed) {
- reader = new ForwardBytesReader(bytes, 0);
+ reader = emptyBytes.getForwardReader();
} else {
- reader = new ReverseBytesReader(bytes, bytes.length-1);
+ 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);
+ }
}
emptyOutput = outputs.readFinalOutput(reader);
} else {
@@ -331,12 +346,13 @@ public final class FST<T> {
nodeRefToAddress = null;
}
startNode = in.readVInt();
- nodeCount = in.readVInt();
- arcCount = in.readVInt();
- arcWithOutputCount = in.readVInt();
+ nodeCount = in.readVLong();
+ arcCount = in.readVLong();
+ arcWithOutputCount = in.readVLong();
- bytes = new byte[in.readVInt()];
- in.readBytes(bytes, 0, bytes.length);
+ int numBytes = in.readVInt();
+ bytes = new BytesStore(in, numBytes, Integer.MAX_VALUE);
+
NO_OUTPUT = outputs.getNoOutput();
cacheRootArcs();
@@ -345,6 +361,15 @@ public final class FST<T> {
// building; we need to break out mutable FST from
// immutable
allowArrayArcs = false;
+
+ /*
+ if (bytes.length == 665) {
+ Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"), "UTF-8");
+ Util.toDot(this, w, false, false);
+ w.close();
+ System.out.println("Wrote FST to out.dot");
+ }
+ */
}
public INPUT_TYPE getInputType() {
@@ -352,8 +377,8 @@ public final class FST<T> {
}
/** Returns bytes used to represent the FST */
- public int sizeInBytes() {
- int size = bytes.length;
+ public long sizeInBytes() {
+ long size = bytes.getPosition();
if (packed) {
size += nodeRefToAddress.ramBytesUsed();
} else if (nodeAddress != null) {
@@ -363,25 +388,23 @@ public final class FST<T> {
return size;
}
- void finish(int startNode) throws IOException {
- if (startNode == FINAL_END_NODE && emptyOutput != null) {
- startNode = 0;
- }
+ void finish(long startNode) throws IOException {
if (this.startNode != -1) {
throw new IllegalStateException("already finished");
}
- byte[] finalBytes = new byte[writer.getPosition()];
- System.arraycopy(bytes, 0, finalBytes, 0, writer.getPosition());
- bytes = finalBytes;
+ if (startNode == FINAL_END_NODE && emptyOutput != null) {
+ startNode = 0;
+ }
this.startNode = startNode;
+ bytes.finish();
cacheRootArcs();
}
- private int getNodeAddress(int node) {
+ private long getNodeAddress(long node) {
if (nodeAddress != null) {
// Deref
- return (int) nodeAddress.get(node);
+ return nodeAddress.get((int) node);
} else {
// Straight
return node;
@@ -481,12 +504,13 @@ public final class FST<T> {
if (packed) {
((PackedInts.Mutable) nodeRefToAddress).save(out);
}
- out.writeVInt(startNode);
- out.writeVInt(nodeCount);
- out.writeVInt(arcCount);
- out.writeVInt(arcWithOutputCount);
- out.writeVInt(bytes.length);
- out.writeBytes(bytes, 0, bytes.length);
+ out.writeVLong(startNode);
+ out.writeVLong(nodeCount);
+ out.writeVLong(arcCount);
+ out.writeVLong(arcWithOutputCount);
+ long numBytes = bytes.getPosition();
+ out.writeVLong(numBytes);
+ bytes.writeTo(out);
}
/**
@@ -526,17 +550,16 @@ public final class FST<T> {
}
}
- private void writeLabel(int v) throws IOException {
+ private void writeLabel(DataOutput out, int v) throws IOException {
assert v >= 0: "v=" + v;
if (inputType == INPUT_TYPE.BYTE1) {
assert v <= 255: "v=" + v;
- writer.writeByte((byte) v);
+ out.writeByte((byte) v);
} else if (inputType == INPUT_TYPE.BYTE2) {
assert v <= 65535: "v=" + v;
- writer.writeShort((short) v);
+ out.writeShort((short) v);
} else {
- //writeInt(v);
- writer.writeVInt(v);
+ out.writeVInt(v);
}
}
@@ -562,8 +585,9 @@ public final class FST<T> {
// serializes new node by appending its bytes to the end
// of the current byte[]
- int addNode(Builder.UnCompiledNode<T> nodeIn) throws IOException {
- //System.out.println("FST.addNode pos=" + writer.posWrite + " numArcs=" + nodeIn.numArcs);
+ long addNode(Builder.UnCompiledNode<T> nodeIn) throws IOException {
+
+ //System.out.println("FST.addNode pos=" + bytes.getPosition() + " numArcs=" + nodeIn.numArcs);
if (nodeIn.numArcs == 0) {
if (nodeIn.isFinal) {
return FINAL_END_NODE;
@@ -572,38 +596,28 @@ public final class FST<T> {
}
}
- int startAddress = writer.getPosition();
+ final long startAddress = bytes.getPosition();
//System.out.println(" startAddr=" + startAddress);
final boolean doFixedArray = shouldExpand(nodeIn);
- final int fixedArrayStart;
if (doFixedArray) {
+ //System.out.println(" fixedArray");
if (bytesPerArc.length < nodeIn.numArcs) {
bytesPerArc = new int[ArrayUtil.oversize(nodeIn.numArcs, 1)];
}
- // write a "false" first arc:
- writer.writeByte(ARCS_AS_FIXED_ARRAY);
- writer.writeVInt(nodeIn.numArcs);
- // placeholder -- we'll come back and write the number
- // of bytes per arc (int) here:
- // TODO: we could make this a vInt instead
- writer.writeInt(0);
- fixedArrayStart = writer.getPosition();
- //System.out.println(" do fixed arcs array arcsStart=" + fixedArrayStart);
- } else {
- fixedArrayStart = 0;
}
arcCount += nodeIn.numArcs;
final int lastArc = nodeIn.numArcs-1;
- int lastArcStart = writer.getPosition();
+ long lastArcStart = bytes.getPosition();
int maxBytesPerArc = 0;
for(int arcIdx=0;arcIdx<nodeIn.numArcs;arcIdx++) {
final Builder.Arc<T> arc = nodeIn.arcs[arcIdx];
final Builder.CompiledNode target = (Builder.CompiledNode) arc.target;
int flags = 0;
+ //System.out.println(" arc " + arcIdx + " label=" + arc.label + " -> target=" + target.node);
if (arcIdx == lastArc) {
flags += BIT_LAST_ARC;
@@ -630,111 +644,135 @@ public final class FST<T> {
if (!targetHasArcs) {
flags += BIT_STOP_NODE;
} else if (inCounts != null) {
- inCounts.set(target.node, inCounts.get(target.node) + 1);
+ inCounts.set((int) target.node, inCounts.get((int) target.node) + 1);
}
if (arc.output != NO_OUTPUT) {
flags += BIT_ARC_HAS_OUTPUT;
}
- writer.writeByte((byte) flags);
- writeLabel(arc.label);
+ bytes.writeByte((byte) flags);
+ writeLabel(bytes, arc.label);
- // System.out.println(" write arc: label=" + (char) arc.label + " flags=" + flags + " target=" + target.node + " pos=" + writer.posWrite + " output=" + outputs.outputToString(arc.output));
+ // System.out.println(" write arc: label=" + (char) arc.label + " flags=" + flags + " target=" + target.node + " pos=" + bytes.getPosition() + " output=" + outputs.outputToString(arc.output));
if (arc.output != NO_OUTPUT) {
- outputs.write(arc.output, writer);
+ outputs.write(arc.output, bytes);
//System.out.println(" write output");
arcWithOutputCount++;
}
if (arc.nextFinalOutput != NO_OUTPUT) {
//System.out.println(" write final output");
- outputs.writeFinalOutput(arc.nextFinalOutput, writer);
+ outputs.writeFinalOutput(arc.nextFinalOutput, bytes);
}
if (targetHasArcs && (flags & BIT_TARGET_NEXT) == 0) {
assert target.node > 0;
//System.out.println(" write target");
- writer.writeInt(target.node);
+ bytes.writeVLong(target.node);
}
// just write the arcs "like normal" on first pass,
// but record how many bytes each one took, and max
// byte size:
if (doFixedArray) {
- bytesPerArc[arcIdx] = writer.getPosition() - lastArcStart;
- lastArcStart = writer.getPosition();
+ bytesPerArc[arcIdx] = (int) (bytes.getPosition() - lastArcStart);
+ lastArcStart = bytes.getPosition();
maxBytesPerArc = Math.max(maxBytesPerArc, bytesPerArc[arcIdx]);
//System.out.println(" bytes=" + bytesPerArc[arcIdx]);
}
}
-
- // TODO: if arc'd arrays will be "too wasteful" by some
- // measure, eg if arcs have vastly different sized
- // outputs, then we should selectively disable array for
- // such cases
+
+ // TODO: try to avoid wasteful cases: disable doFixedArray in that case
+ /*
+ *
+ * LUCENE-4682: what is a fair heuristic here?
+ * It could involve some of these:
+ * 1. how "busy" the node is: nodeIn.inputCount relative to frontier[0].inputCount?
+ * 2. how much binSearch saves over scan: nodeIn.numArcs
+ * 3. waste: numBytes vs numBytesExpanded
+ *
+ * the one below just looks at #3
+ if (doFixedArray) {
+ // rough heuristic: make this 1.25 "waste factor" a parameter to the phd ctor????
+ int numBytes = lastArcStart - startAddress;
+ int numBytesExpanded = maxBytesPerArc * nodeIn.numArcs;
+ if (numBytesExpanded > numBytes*1.25) {
+ doFixedArray = false;
+ }
+ }
+ */
if (doFixedArray) {
- //System.out.println(" doFixedArray");
+ final int MAX_HEADER_SIZE = 11; // header(byte) + numArcs(vint) + numBytes(vint)
assert maxBytesPerArc > 0;
// 2nd pass just "expands" all arcs to take up a fixed
// byte size
- final int sizeNeeded = fixedArrayStart + nodeIn.numArcs * maxBytesPerArc;
- assert ((long) fixedArrayStart) + ((long) nodeIn.numArcs) * maxBytesPerArc < Integer.MAX_VALUE: "FST too large (> 2.1 GB)";
- bytes = ArrayUtil.grow(bytes, sizeNeeded);
- // TODO: we could make this a vInt instead
- bytes[fixedArrayStart-4] = (byte) (maxBytesPerArc >> 24);
- bytes[fixedArrayStart-3] = (byte) (maxBytesPerArc >> 16);
- bytes[fixedArrayStart-2] = (byte) (maxBytesPerArc >> 8);
- bytes[fixedArrayStart-1] = (byte) maxBytesPerArc;
+ //System.out.println("write int @pos=" + (fixedArrayStart-4) + " numArcs=" + nodeIn.numArcs);
+ // create the header
+ // TODO: clean this up: or just rewind+reuse and deal with it
+ byte header[] = new byte[MAX_HEADER_SIZE];
+ ByteArrayDataOutput bad = new ByteArrayDataOutput(header);
+ // write a "false" first arc:
+ bad.writeByte(ARCS_AS_FIXED_ARRAY);
+ bad.writeVInt(nodeIn.numArcs);
+ bad.writeVInt(maxBytesPerArc);
+ int headerLen = bad.getPosition();
+
+ final long fixedArrayStart = startAddress + headerLen;
// expand the arcs in place, backwards
- int srcPos = writer.getPosition();
- int destPos = fixedArrayStart + nodeIn.numArcs*maxBytesPerArc;
- writer.setPosition(destPos);
- for(int arcIdx=nodeIn.numArcs-1;arcIdx>=0;arcIdx--) {
- //System.out.println(" repack arcIdx=" + arcIdx + " srcPos=" + srcPos + " destPos=" + destPos);
- destPos -= maxBytesPerArc;
- srcPos -= bytesPerArc[arcIdx];
- if (srcPos != destPos) {
- assert destPos > srcPos: "destPos=" + destPos + " srcPos=" + srcPos + " arcIdx=" + arcIdx + " maxBytesPerArc=" + maxBytesPerArc + " bytesPerArc[arcIdx]=" + bytesPerArc[arcIdx] + " nodeIn.numArcs=" + nodeIn.numArcs;
- System.arraycopy(bytes, srcPos, bytes, destPos, bytesPerArc[arcIdx]);
+ long srcPos = bytes.getPosition();
+ long destPos = fixedArrayStart + nodeIn.numArcs*maxBytesPerArc;
+ assert destPos >= srcPos;
+ if (destPos > srcPos) {
+ bytes.skipBytes((int) (destPos - srcPos));
+ for(int arcIdx=nodeIn.numArcs-1;arcIdx>=0;arcIdx--) {
+ destPos -= maxBytesPerArc;
+ srcPos -= bytesPerArc[arcIdx];
+ //System.out.println(" repack arcIdx=" + arcIdx + " srcPos=" + srcPos + " destPos=" + destPos);
+ if (srcPos != destPos) {
+ //System.out.println(" copy len=" + bytesPerArc[arcIdx]);
+ assert destPos > srcPos: "destPos=" + destPos + " srcPos=" + srcPos + " arcIdx=" + arcIdx + " maxBytesPerArc=" + maxBytesPerArc + " bytesPerArc[arcIdx]=" + bytesPerArc[arcIdx] + " nodeIn.numArcs=" + nodeIn.numArcs;
+ bytes.copyBytes(srcPos, destPos, bytesPerArc[arcIdx]);
+ }
}
}
+
+ // now write the header
+ bytes.writeBytes(startAddress, header, 0, headerLen);
}
- // reverse bytes in-place; we do this so that the
- // "BIT_TARGET_NEXT" opto can work, ie, it reads the
- // node just before the current one
- final int endAddress = writer.getPosition() - 1;
-
- int left = startAddress;
- int right = endAddress;
- while (left < right) {
- final byte b = bytes[left];
- bytes[left++] = bytes[right];
- bytes[right--] = b;
+ final long thisNodeAddress = bytes.getPosition()-1;
+
+ bytes.reverse(startAddress, thisNodeAddress);
+
+ // PackedInts uses int as the index, so we cannot handle
+ // > 2.1B nodes when packing:
+ if (nodeAddress != null && nodeCount == Integer.MAX_VALUE) {
+ throw new IllegalStateException("cannot create a packed FST with more than 2.1 billion nodes");
}
- //System.out.println(" endAddress=" + endAddress);
nodeCount++;
- final int node;
+ final long node;
if (nodeAddress != null) {
+
// Nodes are addressed by 1+ord:
- if (nodeCount == nodeAddress.size()) {
+ if ((int) nodeCount == nodeAddress.size()) {
nodeAddress = nodeAddress.resize(ArrayUtil.oversize(nodeAddress.size() + 1, nodeAddress.getBitsPerValue()));
inCounts = inCounts.resize(ArrayUtil.oversize(inCounts.size() + 1, inCounts.getBitsPerValue()));
}
- nodeAddress.set(nodeCount, endAddress);
+ nodeAddress.set((int) nodeCount, thisNodeAddress);
// System.out.println(" write nodeAddress[" + nodeCount + "] = " + endAddress);
node = nodeCount;
} else {
- node = endAddress;
+ node = thisNodeAddress;
}
lastFrozenNode = node;
+ //System.out.println(" ret node=" + node + " address=" + thisNodeAddress + " nodeAddress=" + nodeAddress);
return node;
}
@@ -763,7 +801,7 @@ public final class FST<T> {
*
* @return Returns the second argument
* (<code>arc</code>). */
- public Arc<T> readLastTargetArc(Arc<T> follow, Arc<T> arc, FST.BytesReader in) throws IOException {
+ public Arc<T> readLastTargetArc(Arc<T> follow, Arc<T> arc, BytesReader in) throws IOException {
//System.out.println("readLast");
if (!targetHasArcs(follow)) {
//System.out.println(" end node");
@@ -774,19 +812,19 @@ public final class FST<T> {
arc.flags = BIT_LAST_ARC;
return arc;
} else {
- in.pos = getNodeAddress(follow.target);
+ in.setPosition(getNodeAddress(follow.target));
arc.node = follow.target;
final byte b = in.readByte();
if (b == ARCS_AS_FIXED_ARRAY) {
// array: jump straight to end
arc.numArcs = in.readVInt();
- if (packed) {
+ if (packed || version >= VERSION_VINT_TARGET) {
arc.bytesPerArc = in.readVInt();
} else {
arc.bytesPerArc = in.readInt();
}
//System.out.println(" array numArcs=" + arc.numArcs + " bpa=" + arc.bytesPerArc);
- arc.posArcsStart = in.pos;
+ arc.posArcsStart = in.getPosition();
arc.arcIdx = arc.numArcs - 2;
} else {
arc.flags = b;
@@ -804,18 +842,16 @@ public final class FST<T> {
}
if (arc.flag(BIT_STOP_NODE)) {
} else if (arc.flag(BIT_TARGET_NEXT)) {
+ } else if (packed) {
+ in.readVLong();
} else {
- if (packed) {
- in.readVInt();
- } else {
- in.skip(4);
- }
+ readUnpackedNodeTarget(in);
}
arc.flags = in.readByte();
}
- // Undo the byte flags we read:
- in.skip(-1);
- arc.nextArc = in.pos;
+ // Undo the byte flags we read:
+ in.skipBytes(-1);
+ arc.nextArc = in.getPosition();
}
readNextRealArc(arc, in);
assert arc.isLast();
@@ -823,6 +859,16 @@ public final class FST<T> {
}
}
+ private long readUnpackedNodeTarget(BytesReader in) throws IOException {
+ long target;
+ if (version < VERSION_VINT_TARGET) {
+ target = in.readInt();
+ } else {
+ target = in.readVLong();
+ }
+ return target;
+ }
+
/**
* Follow the <code>follow</code> arc and read the first arc of its target;
* this changes the provided <code>arc</code> (2nd arg) in-place and returns
@@ -853,10 +899,9 @@ public final class FST<T> {
}
}
- public Arc<T> readFirstRealTargetArc(int node, Arc<T> arc, final BytesReader in) throws IOException {
- assert in.bytes == bytes;
- final int address = getNodeAddress(node);
- in.pos = address;
+ public Arc<T> readFirstRealTargetArc(long node, Arc<T> arc, final BytesReader in) throws IOException {
+ final long address = getNodeAddress(node);
+ in.setPosition(address);
//System.out.println(" readFirstRealTargtArc address="
//+ address);
//System.out.println(" flags=" + arc.flags);
@@ -866,13 +911,13 @@ public final class FST<T> {
//System.out.println(" fixedArray");
// this is first arc in a fixed-array
arc.numArcs = in.readVInt();
- if (packed) {
+ if (packed || version >= VERSION_VINT_TARGET) {
arc.bytesPerArc = in.readVInt();
} else {
arc.bytesPerArc = in.readInt();
}
arc.arcIdx = -1;
- arc.nextArc = arc.posArcsStart = in.pos;
+ arc.nextArc = arc.posArcsStart = in.getPosition();
//System.out.println(" bytesPer=" + arc.bytesPerArc + " numArcs=" + arc.numArcs + " arcsStart=" + pos);
} else {
//arc.flags = b;
@@ -889,11 +934,11 @@ public final class FST<T> {
* @return Returns <code>true</code> if <code>arc</code> points to a state in an
* expanded array format.
*/
- boolean isExpandedTarget(Arc<T> follow, FST.BytesReader in) throws IOException {
+ boolean isExpandedTarget(Arc<T> follow, BytesReader in) throws IOException {
if (!targetHasArcs(follow)) {
return false;
} else {
- in.pos = getNodeAddress(follow.target);
+ in.setPosition(getNodeAddress(follow.target));
return in.readByte() == ARCS_AS_FIXED_ARRAY;
}
}
@@ -917,30 +962,36 @@ public final class FST<T> {
assert !arc.isLast();
if (arc.label == END_LABEL) {
- //System.out.println(" nextArc fake " + arc.nextArc);
- int pos = in.pos = getNodeAddress(arc.nextArc);
+ //System.out.println(" nextArc fake " +
+ //arc.nextArc);
+
+ long pos = getNodeAddress(arc.nextArc);
+ in.setPosition(pos);
+
final byte b = in.readByte();
if (b == ARCS_AS_FIXED_ARRAY) {
- //System.out.println(" nextArc fake array");
+ //System.out.println(" nextArc fixed array");
in.readVInt();
- if (packed) {
+
+ // Skip bytesPerArc:
+ if (packed || version >= VERSION_VINT_TARGET) {
in.readVInt();
} else {
in.readInt();
}
} else {
- in.pos = pos;
+ in.setPosition(pos);
}
} else {
if (arc.bytesPerArc != 0) {
//System.out.println(" nextArc real array");
// arcs are at fixed entries
- in.pos = arc.posArcsStart;
- in.skip((1+arc.arcIdx)*arc.bytesPerArc);
+ in.setPosition(arc.posArcsStart);
+ in.skipBytes((1+arc.arcIdx)*arc.bytesPerArc);
} else {
// arcs are packed
//System.out.println(" nextArc real packed");
- in.pos = arc.nextArc;
+ in.setPosition(arc.nextArc);
}
}
// skip flags
@@ -951,7 +1002,6 @@ public final class FST<T> {
/** Never returns null, but you should never call this if
* arc.isLast() is true. */
public Arc<T> readNextRealArc(Arc<T> arc, final BytesReader in) throws IOException {
- assert in.bytes == bytes;
// TODO: can't assert this because we call from readFirstArc
// assert !flag(arc.flags, BIT_LAST_ARC);
@@ -961,10 +1011,11 @@ public final class FST<T> {
// arcs are at fixed entries
arc.arcIdx++;
assert arc.arcIdx < arc.numArcs;
- in.skip(arc.posArcsStart, arc.arcIdx*arc.bytesPerArc);
+ in.setPosition(arc.posArcsStart);
+ in.skipBytes(arc.arcIdx*arc.bytesPerArc);
} else {
// arcs are packed
- in.pos = arc.nextArc;
+ in.setPosition(arc.nextArc);
}
arc.flags = in.readByte();
arc.label = readLabel(in);
@@ -987,9 +1038,9 @@ public final class FST<T> {
} else {
arc.target = NON_FINAL_END_NODE;
}
- arc.nextArc = in.pos;
+ arc.nextArc = in.getPosition();
} else if (arc.flag(BIT_TARGET_NEXT)) {
- arc.nextArc = in.pos;
+ arc.nextArc = in.getPosition();
// TODO: would be nice to make this lazy -- maybe
// caller doesn't need the target and is scanning arcs...
if (nodeAddress == null) {
@@ -998,35 +1049,36 @@ public final class FST<T> {
// must scan
seekToNextNode(in);
} else {
- in.skip(arc.posArcsStart, arc.bytesPerArc * arc.numArcs);
+ in.setPosition(arc.posArcsStart);
+ in.skipBytes(arc.bytesPerArc * arc.numArcs);
}
}
- arc.target = in.pos;
+ arc.target = in.getPosition();
} else {
arc.target = arc.node - 1;
assert arc.target > 0;
}
} else {
if (packed) {
- final int pos = in.pos;
- final int code = in.readVInt();
+ final long pos = in.getPosition();
+ final long code = in.readVLong();
if (arc.flag(BIT_TARGET_DELTA)) {
// Address is delta-coded from current address:
arc.target = pos + code;
//System.out.println(" delta pos=" + pos + " delta=" + code + " target=" + arc.target);
} else if (code < nodeRefToAddress.size()) {
// Deref
- arc.target = (int) nodeRefToAddress.get(code);
+ arc.target = nodeRefToAddress.get((int) code);
//System.out.println(" deref code=" + code + " target=" + arc.target);
} else {
// Absolute
arc.target = code;
- //System.out.println(" abs code=" + code + " derefLen=" + nodeRefToAddress.length);
+ //System.out.println(" abs code=" + code);
}
} else {
- arc.target = in.readInt();
+ arc.target = readUnpackedNodeTarget(in);
}
- arc.nextArc = in.pos;
+ arc.nextArc = in.getPosition();
}
return arc;
}
@@ -1035,7 +1087,6 @@ public final class FST<T> {
* This returns null if the arc was not found, else the incoming arc. */
public Arc<T> findTargetArc(int labelToMatch, Arc<T> follow, Arc<T> arc, BytesReader in) throws IOException {
assert cachedRootArcs != null;
- assert in.bytes == bytes;
if (labelToMatch == END_LABEL) {
if (follow.isFinal()) {
@@ -1070,7 +1121,7 @@ public final class FST<T> {
return null;
}
- in.pos = getNodeAddress(follow.target);
+ in.setPosition(getNodeAddress(follow.target));
arc.node = follow.target;
@@ -1079,18 +1130,19 @@ public final class FST<T> {
if (in.readByte() == ARCS_AS_FIXED_ARRAY) {
// Arcs are full array; do binary search:
arc.numArcs = in.readVInt();
- if (packed) {
+ if (packed || version >= VERSION_VINT_TARGET) {
arc.bytesPerArc = in.readVInt();
} else {
arc.bytesPerArc = in.readInt();
}
- arc.posArcsStart = in.pos;
+ arc.posArcsStart = in.getPosition();
int low = 0;
int high = arc.numArcs-1;
while (low <= high) {
//System.out.println(" cycle");
int mid = (low + high) >>> 1;
- in.skip(arc.posArcsStart, arc.bytesPerArc*mid + 1);
+ in.setPosition(arc.posArcsStart);
+ in.skipBytes(arc.bytesPerArc*mid + 1);
int midLabel = readLabel(in);
final int cmp = midLabel - labelToMatch;
if (cmp < 0) {
@@ -1145,9 +1197,9 @@ public final class FST<T> {
if (!flag(flags, BIT_STOP_NODE) && !flag(flags, BIT_TARGET_NEXT)) {
if (packed) {
- in.readVInt();
+ in.readVLong();
} else {
- in.readInt();
+ readUnpackedNodeTarget(in);
}
}
@@ -1157,16 +1209,16 @@ public final class FST<T> {
}
}
- public int getNodeCount() {
+ public long getNodeCount() {
// 1+ in order to count the -1 implicit final node
return 1+nodeCount;
}
- public int getArcCount() {
+ public long getArcCount() {
return arcCount;
}
- public int getArcWithOutputCount() {
+ public long getArcWithOutputCount() {
return arcWithOutputCount;
}
@@ -1191,56 +1243,6 @@ public final class FST<T> {
node.numArcs >= FIXED_ARRAY_NUM_ARCS_DEEP);
}
- static abstract class BytesWriter extends DataOutput {
- public abstract void setPosition(int posWrite);
- public abstract int getPosition();
- }
-
- // Non-static: writes to FST's byte[]
- class DefaultBytesWriter extends BytesWriter {
- int posWrite;
-
- public DefaultBytesWriter() {
- // pad: ensure no node gets address 0 which is reserved to mean
- // the stop state w/ no arcs
- posWrite = 1;
- }
-
- @Override
- public void writeByte(byte b) {
- assert posWrite <= bytes.length;
- if (bytes.length == posWrite) {
- assert bytes.length < Integer.MAX_VALUE: "FST too large (> 2.1 GB)";
- bytes = ArrayUtil.grow(bytes);
- }
- assert posWrite < bytes.length: "posWrite=" + posWrite + " bytes.length=" + bytes.length;
- bytes[posWrite++] = b;
- }
-
- @Override
- public int getPosition() {
- return posWrite;
- }
-
- @Override
- public void setPosition(int posWrite) {
- this.posWrite = posWrite;
- if (bytes.length < posWrite) {
- assert bytes.length < Integer.MAX_VALUE: "FST too large (> 2.1 GB)";
- bytes = ArrayUtil.grow(bytes, posWrite);
- }
- }
-
- @Override
- public void writeBytes(byte[] b, int offset, int length) {
- final int size = posWrite + length;
- assert bytes.length < Integer.MAX_VALUE: "FST too large (> 2.1 GB)";
- bytes = ArrayUtil.grow(bytes, size);
- System.arraycopy(b, offset, bytes, posWrite, length);
- posWrite += length;
- }
- }
-
/** Returns a {@link BytesReader} for this FST, positioned at
* position 0. */
public BytesReader getBytesReader() {
@@ -1249,87 +1251,34 @@ public final class FST<T> {
/** Returns a {@link BytesReader} for this FST, positioned at
* the provided position. */
- public BytesReader getBytesReader(int pos) {
+ public BytesReader getBytesReader(long pos) {
// TODO: maybe re-use via ThreadLocal?
+ BytesReader in;
if (packed) {
- return new ForwardBytesReader(bytes, pos);
+ in = bytes.getForwardReader();
} else {
- return new ReverseBytesReader(bytes, pos);
- }
- }
-
- /** Reads the bytes from this FST. Use {@link
- * #getBytesReader(int)} to obtain an instance for this
- * FST; re-use across calls (but only within a single
- * thread) for better performance. */
- public static abstract class BytesReader extends DataInput {
- protected int pos;
- protected final byte[] bytes;
- protected BytesReader(byte[] bytes, int pos) {
- this.bytes = bytes;
- this.pos = pos;
- }
- abstract void skip(int byteCount);
- abstract void skip(int base, int byteCount);
- }
-
- final static class ReverseBytesReader extends BytesReader {
-
- public ReverseBytesReader(byte[] bytes, int pos) {
- super(bytes, pos);
- }
-
- @Override
- public byte readByte() {
- return bytes[pos--];
- }
-
- @Override
- public void readBytes(byte[] b, int offset, int len) {
- for(int i=0;i<len;i++) {
- b[offset+i] = bytes[pos--];
- }
- }
-
- @Override
- public void skip(int count) {
- pos -= count;
+ in = bytes.getReverseReader();
}
-
- @Override
- public void skip(int base, int count) {
- pos = base - count;
+ if (pos != 0) {
+ in.setPosition(pos);
}
+ return in;
}
- // TODO: can we use just ByteArrayDataInput...? need to
- // add a .skipBytes to DataInput.. hmm and .setPosition
- final static class ForwardBytesReader extends BytesReader {
-
- public ForwardBytesReader(byte[] bytes, int pos) {
- super(bytes, pos);
- }
-
- @Override
- public byte readByte() {
- return bytes[pos++];
- }
+ /** Reads bytes stored in an FST. */
+ public static abstract class BytesReader extends DataInput {
+ /** Get current read position. */
+ public abstract long getPosition();
- @Override
- public void readBytes(byte[] b, int offset, int len) {
- System.arraycopy(bytes, pos, b, offset, len);
- pos += len;
- }
+ /** Set current read position. */
+ public abstract void setPosition(long pos);
- @Override
- public void skip(int count) {
- pos += count;
- }
+ /** Returns true if this reader uses reversed bytes
+ * under-the-hood. */
+ public abstract boolean reversed();
- @Override
- public void skip(int base, int count) {
- pos = base + count;
- }
+ /** Skips bytes. */
+ public abstract void skipBytes(int count);
}
private static class ArcAndState<T> {
@@ -1451,14 +1400,13 @@ public final class FST<T> {
*/
// Creates a packed FST
- private FST(INPUT_TYPE inputType, PackedInts.Reader nodeRefToAddress, Outputs<T> outputs) {
+ private FST(INPUT_TYPE inputType, Outputs<T> outputs, int bytesPageBits) {
+ version = VERSION_CURRENT;
packed = true;
this.inputType = inputType;
- bytes = new byte[128];
- this.nodeRefToAddress = nodeRefToAddress;
+ bytes = new BytesStore(bytesPageBits);
this.outputs = outputs;
NO_OUTPUT = outputs.getNoOutput();
- writer = new DefaultBytesWriter();
// NOTE: bogus because this is only used during
// building; we need to break out mutable FST from
@@ -1480,6 +1428,9 @@ public final class FST<T> {
*/
FST<T> pack(int minInCountDeref, int maxDerefNodes, float acceptableOverheadRatio) throws IOException {
+ // NOTE: maxDerefNodes is intentionally int: we cannot
+ // support > 2.1B deref nodes
+
// TODO: other things to try
// - renumber the nodes to get more next / better locality?
// - allow multiple input labels on an arc, so
@@ -1529,17 +1480,13 @@ public final class FST<T> {
//System.out.println("map node=" + n.node + " inCount=" + n.count + " to newID=" + downTo);
}
- final FST<T> fst = new FST<T>(inputType, null, outputs);
-
- final BytesWriter writer = fst.writer;
-
// +1 because node ords start at 1 (0 is reserved as stop node):
final GrowableWriter newNodeAddress = new GrowableWriter(
- PackedInts.bitsRequired(bytes.length), 1 + nodeCount, acceptableOverheadRatio);
+ PackedInts.bitsRequired(this.bytes.getPosition()), (int) (1 + nodeCount), acceptableOverheadRatio);
// Fill initial coarse guess:
for(int node=1;node<=nodeCount;node++) {
- newNodeAddress.set(node, 1 + bytes.length - nodeAddress.get(node));
+ newNodeAddress.set(node, 1 + this.bytes.getPosition() - nodeAddress.get(node));
}
int absCount;
@@ -1547,6 +1494,8 @@ public final class FST<T> {
int topCount;
int nextCount;
+ FST<T> fst;
+
// Iterate until we converge:
while(true) {
@@ -1556,7 +1505,10 @@ public final class FST<T> {
// for assert:
boolean negDelta = false;
- writer.setPosition(0);
+ fst = new FST<T>(inputType, outputs, bytes.getBlockBits());
+
+ final BytesStore writer = fst.bytes;
+
// Skip 0 byte since 0 is reserved target:
writer.writeByte((byte) 0);
@@ -1568,19 +1520,20 @@ public final class FST<T> {
int changedCount = 0;
- int addressError = 0;
+ long addressError = 0;
//int totWasted = 0;
// Since we re-reverse the bytes, we now write the
// nodes backwards, so that BIT_TARGET_NEXT is
// unchanged:
- for(int node=nodeCount;node>=1;node--) {
+ for(int node=(int)nodeCount;node>=1;node--) {
fst.nodeCount++;
- final int address = writer.getPosition();
+ final long address = writer.getPosition();
+
//System.out.println(" node: " + node + " address=" + address);
if (address != newNodeAddress.get(node)) {
- addressError = address - (int) newNodeAddress.get(node);
+ addressError = address - newNodeAddress.get(node);
//System.out.println(" change: " + (address - newNodeAddress[node]));
changed = true;
newNodeAddress.set(node, address);
@@ -1600,6 +1553,7 @@ public final class FST<T> {
writeNode:
while(true) { // retry writing this node
+ //System.out.println(" cycle: retry");
readFirstRealTargetArc(node, arc, r);
final boolean useArcArray = arc.bytesPerArc != 0;
@@ -1617,9 +1571,9 @@ public final class FST<T> {
int maxBytesPerArc = 0;
//int wasted = 0;
while(true) { // iterate over all arcs for this node
+ //System.out.println(" cycle next arc");
- //System.out.println(" arc label=" + arc.label + " target=" + arc.target + " pos=" + writer.posWrite);
- final int arcStartPos = writer.getPosition();
+ final long arcStartPos = writer.getPosition();
nodeArcCount++;
byte flags = 0;
@@ -1654,19 +1608,18 @@ public final class FST<T> {
flags += BIT_ARC_HAS_OUTPUT;
}
- final Integer ptr;
- final int absPtr;
+ final long absPtr;
final boolean doWriteTarget = targetHasArcs(arc) && (flags & BIT_TARGET_NEXT) == 0;
if (doWriteTarget) {
- ptr = topNodeMap.get(arc.target);
+ final Integer ptr = topNodeMap.get(arc.target);
if (ptr != null) {
absPtr = ptr;
} else {
- absPtr = topNodeMap.size() + (int) newNodeAddress.get(arc.target) + addressError;
+ absPtr = topNodeMap.size() + newNodeAddress.get((int) arc.target) + addressError;
}
- int delta = (int) newNodeAddress.get(arc.target) + addressError - writer.getPosition() - 2;
+ long delta = newNodeAddress.get((int) arc.target) + addressError - writer.getPosition() - 2;
if (delta < 0) {
//System.out.println("neg: " + delta);
anyNegDelta = true;
@@ -1677,12 +1630,13 @@ public final class FST<T> {
flags |= BIT_TARGET_DELTA;
}
} else {
- ptr = null;
absPtr = 0;
}
+ assert flags != ARCS_AS_FIXED_ARRAY;
writer.writeByte(flags);
- fst.writeLabel(arc.label);
+
+ fst.writeLabel(writer, arc.label);
if (arc.output != NO_OUTPUT) {
outputs.write(arc.output, writer);
@@ -1696,7 +1650,7 @@ public final class FST<T> {
if (doWriteTarget) {
- int delta = (int) newNodeAddress.get(arc.target) + addressError - writer.getPosition();
+ long delta = newNodeAddress.get((int) arc.target) + addressError - writer.getPosition();
if (delta < 0) {
anyNegDelta = true;
//System.out.println("neg: " + delta);
@@ -1705,7 +1659,7 @@ public final class FST<T> {
if (flag(flags, BIT_TARGET_DELTA)) {
//System.out.println(" delta");
- writer.writeVInt(delta);
+ writer.writeVLong(delta);
if (!retry) {
deltaCount++;
}
@@ -1717,7 +1671,7 @@ public final class FST<T> {
System.out.println(" abs");
}
*/
- writer.writeVInt(absPtr);
+ writer.writeVLong(absPtr);
if (!retry) {
if (absPtr >= topNodeMap.size()) {
absCount++;
@@ -1729,7 +1683,7 @@ public final class FST<T> {
}
if (useArcArray) {
- final int arcBytes = writer.getPosition() - arcStartPos;
+ final int arcBytes = (int) (writer.getPosition() - arcStartPos);
//System.out.println(" " + arcBytes + " bytes");
maxBytesPerArc = Math.max(maxBytesPerArc, arcBytes);
// NOTE: this may in fact go "backwards", if
@@ -1739,7 +1693,7 @@ public final class FST<T> {
// will retry (below) so it's OK to ovewrite
// bytes:
//wasted += bytesPerArc - arcBytes;
- writer.setPosition(arcStartPos + bytesPerArc);
+ writer.skipBytes((int) (arcStartPos + bytesPerArc - writer.getPosition()));
}
if (arc.isLast()) {
@@ -1764,11 +1718,12 @@ public final class FST<T> {
// Retry:
bytesPerArc = maxBytesPerArc;
- writer.setPosition(address);
+ writer.truncate(address);
nodeArcCount = 0;
retry = true;
anyNegDelta = false;
}
+
negDelta |= anyNegDelta;
fst.arcCount += nodeArcCount;
@@ -1788,8 +1743,8 @@ public final class FST<T> {
}
long maxAddress = 0;
- for (int key : topNodeMap.keySet()) {
- maxAddress = Math.max(maxAddress, newNodeAddress.get(key));
+ for (long key : topNodeMap.keySet()) {
+ maxAddress = Math.max(maxAddress, newNodeAddress.get((int) key));
}
PackedInts.Mutable nodeRefToAddressIn = PackedInts.getMutable(topNodeMap.size(),
@@ -1799,8 +1754,7 @@ public final class FST<T> {
}
fst.nodeRefToAddress = nodeRefToAddressIn;
-
- fst.startNode = (int) newNodeAddress.get(startNode);
+ fst.startNode = newNodeAddress.get((int) startNode);
//System.out.println("new startNode=" + fst.startNode + " old startNode=" + startNode);
if (emptyOutput != null) {
@@ -1810,11 +1764,8 @@ public final class FST<T> {
assert fst.nodeCount == nodeCount: "fst.nodeCount=" + fst.nodeCount + " nodeCount=" + nodeCount;
assert fst.arcCount == arcCount;
assert fst.arcWithOutputCount == arcWithOutputCount: "fst.arcWithOutputCount=" + fst.arcWithOutputCount + " arcWithOutputCount=" + arcWithOutputCount;
-
- final byte[] finalBytes = new byte[writer.getPosition()];
- //System.out.println("resize " + fst.bytes.length + " down to " + writer.posWrite);
- System.arraycopy(fst.bytes, 0, finalBytes, 0, writer.getPosition());
- fst.bytes = finalBytes;
+
+ fst.bytes.finish();
fst.cacheRootArcs();
//final int size = fst.sizeInBytes();
Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java Mon Jan 14 18:54:22 2013
@@ -17,11 +17,11 @@ package org.apache.lucene.util.fst;
* limitations under the License.
*/
+import java.io.IOException;
+
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
-import java.io.IOException;
-
/** Can next() and advance() through the terms in an FST
*
* @lucene.experimental
@@ -153,8 +153,8 @@ abstract class FSTEnum<T> {
boolean found = false;
while (low <= high) {
mid = (low + high) >>> 1;
- in.pos = arc.posArcsStart;
- in.skip(arc.bytesPerArc*mid+1);
+ in.setPosition(arc.posArcsStart);
+ in.skipBytes(arc.bytesPerArc*mid+1);
final int midLabel = fst.readLabel(in);
final int cmp = midLabel - targetLabel;
//System.out.println(" cycle low=" + low + " high=" + high + " mid=" + mid + " midLabel=" + midLabel + " cmp=" + cmp);
@@ -292,8 +292,8 @@ abstract class FSTEnum<T> {
boolean found = false;
while (low <= high) {
mid = (low + high) >>> 1;
- in.pos = arc.posArcsStart;
- in.skip(arc.bytesPerArc*mid+1);
+ in.setPosition(arc.posArcsStart);
+ in.skipBytes(arc.bytesPerArc*mid+1);
final int midLabel = fst.readLabel(in);
final int cmp = midLabel - targetLabel;
//System.out.println(" cycle low=" + low + " high=" + high + " mid=" + mid + " midLabel=" + midLabel + " cmp=" + cmp);
Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java Mon Jan 14 18:54:22 2013
@@ -19,22 +19,27 @@ package org.apache.lucene.util.fst;
import java.io.IOException;
+import org.apache.lucene.util.packed.GrowableWriter;
+import org.apache.lucene.util.packed.PackedInts;
+
// Used to dedup states (lookup already-frozen states)
final class NodeHash<T> {
- private int[] table;
+ private GrowableWriter table;
private int count;
private int mask;
private final FST<T> fst;
private final FST.Arc<T> scratchArc = new FST.Arc<T>();
+ private final FST.BytesReader in;
- public NodeHash(FST<T> fst) {
- table = new int[16];
+ public NodeHash(FST<T> fst, FST.BytesReader in) {
+ table = new GrowableWriter(8, 16, PackedInts.COMPACT);
mask = 15;
this.fst = fst;
+ this.in = in;
}
- private boolean nodesEqual(Builder.UnCompiledNode<T> node, int address, FST.BytesReader in) throws IOException {
+ private boolean nodesEqual(Builder.UnCompiledNode<T> node, long address) throws IOException {
fst.readFirstRealTargetArc(address, scratchArc, in);
if (scratchArc.bytesPerArc != 0 && node.numArcs != scratchArc.numArcs) {
return false;
@@ -73,7 +78,8 @@ final class NodeHash<T> {
final Builder.Arc<T> arc = node.arcs[arcIdx];
//System.out.println(" label=" + arc.label + " target=" + ((Builder.CompiledNode) arc.target).node + " h=" + h + " output=" + fst.outputs.outputToString(arc.output) + " isFinal?=" + arc.isFinal);
h = PRIME * h + arc.label;
- h = PRIME * h + ((Builder.CompiledNode) arc.target).node;
+ long n = ((Builder.CompiledNode) arc.target).node;
+ h = PRIME * h + (int) (n^(n>>32));
h = PRIME * h + arc.output.hashCode();
h = PRIME * h + arc.nextFinalOutput.hashCode();
if (arc.isFinal) {
@@ -85,16 +91,15 @@ final class NodeHash<T> {
}
// hash code for a frozen node
- private int hash(int node) throws IOException {
+ private int hash(long node) throws IOException {
final int PRIME = 31;
- final FST.BytesReader in = fst.getBytesReader(0);
//System.out.println("hash frozen node=" + node);
int h = 0;
fst.readFirstRealTargetArc(node, scratchArc, in);
while(true) {
- //System.out.println(" label=" + scratchArc.label + " target=" + scratchArc.target + " h=" + h + " output=" + fst.outputs.outputToString(scratchArc.output) + " next?=" + scratchArc.flag(4) + " final?=" + scratchArc.isFinal());
+ //System.out.println(" label=" + scratchArc.label + " target=" + scratchArc.target + " h=" + h + " output=" + fst.outputs.outputToString(scratchArc.output) + " next?=" + scratchArc.flag(4) + " final?=" + scratchArc.isFinal() + " pos=" + in.getPosition());
h = PRIME * h + scratchArc.label;
- h = PRIME * h + scratchArc.target;
+ h = PRIME * h + (int) (scratchArc.target^(scratchArc.target>>32));
h = PRIME * h + scratchArc.output.hashCode();
h = PRIME * h + scratchArc.nextFinalOutput.hashCode();
if (scratchArc.isFinal()) {
@@ -109,26 +114,25 @@ final class NodeHash<T> {
return h & Integer.MAX_VALUE;
}
- public int add(Builder.UnCompiledNode<T> nodeIn) throws IOException {
- // System.out.println("hash: add count=" + count + " vs " + table.length);
- final FST.BytesReader in = fst.getBytesReader(0);
+ public long add(Builder.UnCompiledNode<T> nodeIn) throws IOException {
+ // System.out.println("hash: add count=" + count + " vs " + table.size());
final int h = hash(nodeIn);
int pos = h & mask;
int c = 0;
while(true) {
- final int v = table[pos];
+ final long v = table.get(pos);
if (v == 0) {
// freeze & add
- final int node = fst.addNode(nodeIn);
+ final long node = fst.addNode(nodeIn);
//System.out.println(" now freeze node=" + node);
assert hash(node) == h : "frozenHash=" + hash(node) + " vs h=" + h;
count++;
- table[pos] = node;
- if (table.length < 2*count) {
+ table.set(pos, node);
+ if (table.size() < 2*count) {
rehash();
}
return node;
- } else if (nodesEqual(nodeIn, v, in)) {
+ } else if (nodesEqual(nodeIn, v)) {
// same node is already here
return v;
}
@@ -139,12 +143,12 @@ final class NodeHash<T> {
}
// called only by rehash
- private void addNew(int address) throws IOException {
+ private void addNew(long address) throws IOException {
int pos = hash(address) & mask;
int c = 0;
while(true) {
- if (table[pos] == 0) {
- table[pos] = address;
+ if (table.get(pos) == 0) {
+ table.set(pos, address);
break;
}
@@ -154,16 +158,16 @@ final class NodeHash<T> {
}
private void rehash() throws IOException {
- final int[] oldTable = table;
+ final GrowableWriter oldTable = table;
- if (oldTable.length >= Integer.MAX_VALUE/2) {
+ if (oldTable.size() >= Integer.MAX_VALUE/2) {
throw new IllegalStateException("FST too large (> 2.1 GB)");
}
- table = new int[2*table.length];
- mask = table.length-1;
- for(int idx=0;idx<oldTable.length;idx++) {
- final int address = oldTable[idx];
+ table = new GrowableWriter(oldTable.getBitsPerValue(), 2*oldTable.size(), PackedInts.COMPACT);
+ mask = table.size()-1;
+ for(int idx=0;idx<oldTable.size();idx++) {
+ final long address = oldTable.get(idx);
if (address != 0) {
addNew(address);
}
Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/Util.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/Util.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/util/fst/Util.java Mon Jan 14 18:54:22 2013
@@ -39,7 +39,7 @@ public final class Util {
// TODO: would be nice not to alloc this on every lookup
final FST.Arc<T> arc = fst.getFirstArc(new FST.Arc<T>());
- final FST.BytesReader fstReader = fst.getBytesReader(0);
+ final BytesReader fstReader = fst.getBytesReader(0);
// Accumulate output as we go
T output = fst.outputs.getNoOutput();
@@ -64,7 +64,7 @@ public final class Util {
public static<T> T get(FST<T> fst, BytesRef input) throws IOException {
assert fst.inputType == FST.INPUT_TYPE.BYTE1;
- final FST.BytesReader fstReader = fst.getBytesReader(0);
+ final BytesReader fstReader = fst.getBytesReader(0);
// TODO: would be nice not to alloc this on every lookup
final FST.Arc<T> arc = fst.getFirstArc(new FST.Arc<T>());
@@ -101,7 +101,7 @@ public final class Util {
* fit this. */
public static IntsRef getByOutput(FST<Long> fst, long targetOutput) throws IOException {
- final FST.BytesReader in = fst.getBytesReader(0);
+ final BytesReader in = fst.getBytesReader(0);
// TODO: would be nice not to alloc this on every lookup
FST.Arc<Long> arc = fst.getFirstArc(new FST.Arc<Long>());
@@ -147,8 +147,8 @@ public final class Util {
boolean exact = false;
while (low <= high) {
mid = (low + high) >>> 1;
- in.pos = arc.posArcsStart;
- in.skip(arc.bytesPerArc*mid);
+ in.setPosition(arc.posArcsStart);
+ in.skipBytes(arc.bytesPerArc*mid);
final byte flags = in.readByte();
fst.readLabel(in);
final long minArcOutput;
@@ -273,7 +273,7 @@ public final class Util {
public static class TopNSearcher<T> {
private final FST<T> fst;
- private final FST.BytesReader bytesReader;
+ private final BytesReader bytesReader;
private final int topN;
private final int maxQueueDepth;
@@ -374,7 +374,7 @@ public final class Util {
//System.out.println("search topN=" + topN);
- final FST.BytesReader fstReader = fst.getBytesReader(0);
+ final BytesReader fstReader = fst.getBytesReader(0);
final T NO_OUTPUT = fst.outputs.getNoOutput();
// TODO: we could enable FST to sorting arcs by weight
@@ -544,7 +544,9 @@ public final class Util {
* </pre>
*
* <p>
- * Note: larger FSTs (a few thousand nodes) won't even render, don't bother.
+ * Note: larger FSTs (a few thousand nodes) won't even
+ * render, don't bother. If the FST is > 2.1 GB in size
+ * then this method will throw strange exceptions.
*
* @param sameRank
* If <code>true</code>, the resulting <code>dot</code> file will try
@@ -578,7 +580,7 @@ public final class Util {
// A bitset of already seen states (target offset).
final BitSet seen = new BitSet();
- seen.set(startArc.target);
+ seen.set((int) startArc.target);
// Shape for states.
final String stateShape = "circle";
@@ -595,7 +597,7 @@ public final class Util {
emitDotState(out, "initial", "point", "white", "");
final T NO_OUTPUT = fst.outputs.getNoOutput();
- final FST.BytesReader r = fst.getBytesReader(0);
+ final BytesReader r = fst.getBytesReader(0);
// final FST.Arc<T> scratchArc = new FST.Arc<T>();
@@ -617,7 +619,7 @@ public final class Util {
finalOutput = null;
}
- emitDotState(out, Integer.toString(startArc.target), isFinal ? finalStateShape : stateShape, stateColor, finalOutput == null ? "" : fst.outputs.outputToString(finalOutput));
+ emitDotState(out, Long.toString(startArc.target), isFinal ? finalStateShape : stateShape, stateColor, finalOutput == null ? "" : fst.outputs.outputToString(finalOutput));
}
out.write(" initial -> " + startArc.target + "\n");
@@ -638,7 +640,8 @@ public final class Util {
if (FST.targetHasArcs(arc)) {
// scan all target arcs
//System.out.println(" readFirstTarget...");
- final int node = arc.target;
+
+ final long node = arc.target;
fst.readFirstRealTargetArc(arc.target, arc, r);
@@ -648,7 +651,7 @@ public final class Util {
//System.out.println(" cycle arc=" + arc);
// Emit the unseen state and add it to the queue for the next level.
- if (arc.target >= 0 && !seen.get(arc.target)) {
+ if (arc.target >= 0 && !seen.get((int) arc.target)) {
/*
boolean isFinal = false;
@@ -675,12 +678,12 @@ public final class Util {
finalOutput = "";
}
- emitDotState(out, Integer.toString(arc.target), stateShape, stateColor, finalOutput);
+ emitDotState(out, Long.toString(arc.target), stateShape, stateColor, finalOutput);
// To see the node address, use this instead:
//emitDotState(out, Integer.toString(arc.target), stateShape, stateColor, String.valueOf(arc.target));
- seen.set(arc.target);
+ seen.set((int) arc.target);
nextLevelQueue.add(new FST.Arc<T>().copyFrom(arc));
- sameLevelStates.add(arc.target);
+ sameLevelStates.add((int) arc.target);
}
String outs;
@@ -893,8 +896,8 @@ public final class Util {
// " targetLabel=" + targetLabel);
while (low <= high) {
mid = (low + high) >>> 1;
- in.pos = arc.posArcsStart;
- in.skip(arc.bytesPerArc * mid + 1);
+ in.setPosition(arc.posArcsStart);
+ in.skipBytes(arc.bytesPerArc * mid + 1);
final int midLabel = fst.readLabel(in);
final int cmp = midLabel - label;
// System.out.println(" cycle low=" + low + " high=" + high + " mid=" +
Modified: lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Mon Jan 14 18:54:22 2013
@@ -99,7 +99,7 @@ public class TestBackwardsCompatibility
createIndex("index.nocfs", false, false);
}
*/
-
+
/*
// These are only needed for the special upgrade test to verify
// that also single-segment indexes are correctly upgraded by IndexUpgrader.
@@ -115,8 +115,40 @@ public class TestBackwardsCompatibility
}
*/
+
+ /*
+ public void testCreateMoreTermsIndex() throws Exception {
+ // we use a real directory name that is not cleaned up,
+ // because this method is only used to create backwards
+ // indexes:
+ File indexDir = new File("moreterms");
+ _TestUtil.rmDir(indexDir);
+ Directory dir = newFSDirectory(indexDir);
+
+ LogByteSizeMergePolicy mp = new LogByteSizeMergePolicy();
+ mp.setUseCompoundFile(false);
+ mp.setNoCFSRatio(1.0);
+ mp.setMaxCFSSegmentSizeMB(Double.POSITIVE_INFINITY);
+ // TODO: remove randomness
+ IndexWriterConfig conf = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()))
+ .setMergePolicy(mp);
+ conf.setCodec(Codec.forName("Lucene40"));
+ IndexWriter writer = new IndexWriter(dir, conf);
+ LineFileDocs docs = new LineFileDocs(null, true);
+ for(int i=0;i<50;i++) {
+ writer.addDocument(docs.nextDoc());
+ }
+ writer.close();
+ dir.close();
+
+ // Gives you time to copy the index out!: (there is also
+ // a test option to not remove temp dir...):
+ Thread.sleep(100000);
+ }
+ */
+
final static String[] oldNames = {"40.cfs",
- "40.nocfs",
+ "40.nocfs",
};
final String[] unsupportedNames = {"19.cfs",
@@ -144,7 +176,7 @@ public class TestBackwardsCompatibility
};
final static String[] oldSingleSegmentNames = {"40.optimized.cfs",
- "40.optimized.nocfs",
+ "40.optimized.nocfs",
};
static Map<String,Directory> oldIndexDirs;
@@ -916,4 +948,15 @@ public class TestBackwardsCompatibility
dir.close();
}
}
+
+ public static final String moreTermsIndex = "moreterms.40.zip";
+
+ public void testMoreTerms() throws Exception {
+ File oldIndexDir = _TestUtil.getTempDir("moreterms");
+ _TestUtil.unzip(getDataFile(moreTermsIndex), oldIndexDir);
+ Directory dir = newFSDirectory(oldIndexDir);
+ // TODO: more tests
+ _TestUtil.checkIndex(dir);
+ dir.close();
+ }
}
Modified: lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java Mon Jan 14 18:54:22 2013
@@ -22,7 +22,6 @@ import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
-import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
@@ -31,12 +30,14 @@ import org.apache.lucene.search.TermStat
import org.apache.lucene.search.similarities.DefaultSimilarity;
import org.apache.lucene.search.similarities.PerFieldSimilarityWrapper;
import org.apache.lucene.search.similarities.Similarity;
+import org.apache.lucene.search.similarities.Similarity.ExactSimScorer;
+import org.apache.lucene.search.similarities.Similarity.SimWeight;
+import org.apache.lucene.search.similarities.Similarity.SloppySimScorer;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LineFileDocs;
import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util._TestUtil;
/**
*
@@ -86,39 +87,6 @@ public class TestCustomNorms extends Luc
dir.close();
docs.close();
}
-
- public void testPackedNorms() throws IOException {
- Directory dir = newDirectory();
- IndexWriterConfig config = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
- config.setSimilarity(new PackedNormSimilarity());
- RandomIndexWriter writer = new RandomIndexWriter(random(), dir, config);
- int num = _TestUtil.nextInt(random(), 1, 1000);
- for (int i = 0; i < num; i++) {
- Document doc = new Document();
- doc.add(new StringField("len", Integer.toString(i), Field.Store.YES));
- StringBuilder sb = new StringBuilder();
- for (int j = 0; j < i; j++) {
- sb.append(" token");
- }
- doc.add(new TextField("content", sb.toString(), Field.Store.NO));
- writer.addDocument(doc);
- }
-
- DirectoryReader ir = writer.getReader();
- writer.close();
- for (AtomicReaderContext context : ir.leaves()) {
- AtomicReader reader = context.reader();
- DocValues norms = reader.normValues("content");
- assertNotNull(norms);
- Source source = norms.getSource();
- assertEquals(Type.VAR_INTS, source.getType());
- for (int i = 0; i < reader.maxDoc(); i++) {
- assertEquals(source.getInt(i), Long.parseLong(reader.document(i).get("len")));
- }
- }
- ir.close();
- dir.close();
- }
public void testExceptionOnRandomType() throws IOException {
Directory dir = newDirectory();
@@ -334,28 +302,5 @@ public class TestCustomNorms extends Luc
throw new UnsupportedOperationException();
}
}
-
- class PackedNormSimilarity extends Similarity {
-
- @Override
- public void computeNorm(FieldInvertState state, Norm norm) {
- norm.setPackedLong(state.getLength());
- }
-
- @Override
- public SimWeight computeWeight(float queryBoost, CollectionStatistics collectionStats, TermStatistics... termStats) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public ExactSimScorer exactSimScorer(SimWeight weight, AtomicReaderContext context) throws IOException {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public SloppySimScorer sloppySimScorer(SimWeight weight, AtomicReaderContext context) throws IOException {
- throw new UnsupportedOperationException();
- }
- }
}
Modified: lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java Mon Jan 14 18:54:22 2013
@@ -310,7 +310,7 @@ public class TestFSTs extends LuceneTest
final boolean doRewrite = random().nextBoolean();
- Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, doRewrite, true);
+ Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, doRewrite, PackedInts.DEFAULT, true, 15);
boolean storeOrd = random().nextBoolean();
if (VERBOSE) {
@@ -453,7 +453,7 @@ public class TestFSTs extends LuceneTest
this.outputs = outputs;
this.doPack = doPack;
- builder = new Builder<T>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs, null, doPack, !noArcArrays);
+ builder = new Builder<T>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs, null, doPack, PackedInts.DEFAULT, !noArcArrays, 15);
}
protected abstract T getOutput(IntsRef input, int ord) throws IOException;
@@ -484,8 +484,13 @@ public class TestFSTs extends LuceneTest
}
}
+ long tMid = System.currentTimeMillis();
+ System.out.println(((tMid-tStart) / 1000.0) + " sec to add all terms");
+
assert builder.getTermCount() == ord;
FST<T> fst = builder.finish();
+ long tEnd = System.currentTimeMillis();
+ System.out.println(((tEnd-tMid) / 1000.0) + " sec to finish/pack");
if (fst == null) {
System.out.println("FST was fully pruned!");
System.exit(0);
@@ -513,6 +518,12 @@ public class TestFSTs extends LuceneTest
return;
}
+ /*
+ IndexInput in = dir.openInput("fst.bin", IOContext.DEFAULT);
+ fst = new FST<T>(in, outputs);
+ in.close();
+ */
+
System.out.println("\nNow verify...");
while(true) {
@@ -576,7 +587,7 @@ public class TestFSTs extends LuceneTest
}
}
- // java -cp build/classes/test:build/classes/test-framework:build/classes/java:lib/junit-4.10.jar org.apache.lucene.util.fst.TestFSTs /x/tmp/allTerms3.txt out
+ // java -cp ../build/codecs/classes/java:../test-framework/lib/randomizedtesting-runner-2.0.8.jar:../build/core/classes/test:../build/core/classes/test-framework:../build/core/classes/java:../build/test-framework/classes/java:../test-framework/lib/junit-4.10.jar org.apache.lucene.util.fst.TestFSTs /xold/tmp/allTerms3.txt out
public static void main(String[] args) throws IOException {
int prune = 0;
int limit = Integer.MAX_VALUE;
@@ -1022,7 +1033,7 @@ public class TestFSTs extends LuceneTest
throws IOException {
if (FST.targetHasArcs(arc)) {
int childCount = 0;
- FST.BytesReader fstReader = fst.getBytesReader(0);
+ BytesReader fstReader = fst.getBytesReader(0);
for (arc = fst.readFirstTargetArc(arc, arc, fstReader);;
arc = fst.readNextArc(arc, fstReader), childCount++)
{
@@ -1062,7 +1073,7 @@ public class TestFSTs extends LuceneTest
public void testFinalOutputOnEndState() throws Exception {
final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
- final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, null, random().nextBoolean(), true);
+ final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, null, random().nextBoolean(), PackedInts.DEFAULT, true, 15);
builder.add(Util.toUTF32("stat", new IntsRef()), 17L);
builder.add(Util.toUTF32("station", new IntsRef()), 10L);
final FST<Long> fst = builder.finish();
@@ -1077,7 +1088,7 @@ public class TestFSTs extends LuceneTest
public void testInternalFinalState() throws Exception {
final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
final boolean willRewrite = random().nextBoolean();
- final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, willRewrite, true);
+ final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, willRewrite, PackedInts.DEFAULT, true, 15);
builder.add(Util.toIntsRef(new BytesRef("stat"), new IntsRef()), outputs.getNoOutput());
builder.add(Util.toIntsRef(new BytesRef("station"), new IntsRef()), outputs.getNoOutput());
final FST<Long> fst = builder.finish();
@@ -1100,7 +1111,7 @@ public class TestFSTs extends LuceneTest
final Long nothing = outputs.getNoOutput();
final Builder<Long> b = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
- final FST<Long> fst = new FST<Long>(FST.INPUT_TYPE.BYTE1, outputs, false, PackedInts.COMPACT, true);
+ final FST<Long> fst = new FST<Long>(FST.INPUT_TYPE.BYTE1, outputs, false, PackedInts.COMPACT, true, 15);
final Builder.UnCompiledNode<Long> rootNode = new Builder.UnCompiledNode<Long>(b, 0);
Modified: lucene/dev/branches/lucene4547/lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java (original)
+++ lucene/dev/branches/lucene4547/lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java Mon Jan 14 18:54:22 2013
@@ -46,7 +46,7 @@ public class SearchFiles {
/** Simple command-line based search demo. */
public static void main(String[] args) throws Exception {
String usage =
- "Usage:\tjava org.apache.lucene.demo.SearchFiles [-index dir] [-field f] [-repeat n] [-queries file] [-query string] [-raw] [-paging hitsPerPage]\n\nSee http://lucene.apache.org/java/4_0/demo.html for details.";
+ "Usage:\tjava org.apache.lucene.demo.SearchFiles [-index dir] [-field f] [-repeat n] [-queries file] [-query string] [-raw] [-paging hitsPerPage]\n\nSee http://lucene.apache.org/core/4_1_0/demo/ for details.";
if (args.length > 0 && ("-h".equals(args[0]) || "-help".equals(args[0]))) {
System.out.println(usage);
System.exit(0);
Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsPayloadIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsPayloadIterator.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsPayloadIterator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsPayloadIterator.java Mon Jan 14 18:54:22 2013
@@ -3,7 +3,7 @@ package org.apache.lucene.facet.associat
import java.io.IOException;
import org.apache.lucene.facet.search.PayloadIterator;
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.util.BytesRef;
@@ -46,13 +46,22 @@ public abstract class AssociationsPayloa
* It is assumed that all association values can be deserialized with the
* given {@link CategoryAssociation}.
*/
- public AssociationsPayloadIterator(IndexReader reader, String field, T association) throws IOException {
- pi = new PayloadIterator(reader, new Term(field, association.getCategoryListID()));
- hasAssociations = pi.init();
+ public AssociationsPayloadIterator(String field, T association) throws IOException {
+ pi = new PayloadIterator(new Term(field, association.getCategoryListID()));
this.association = association;
}
/**
+ * Sets the {@link AtomicReaderContext} for which {@link #setNextDoc(int)}
+ * calls will be made. Returns true iff this reader has associations for any
+ * of the documents belonging to the association given to the constructor.
+ */
+ public final boolean setNextReader(AtomicReaderContext context) throws IOException {
+ hasAssociations = pi.setNextReader(context);
+ return hasAssociations;
+ }
+
+ /**
* Skip to the requested document. Returns true iff the document has category
* association values and they were read successfully. Associations are
* handled through {@link #handleAssociation(int, CategoryAssociation)} by
Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsPayloadIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsPayloadIterator.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsPayloadIterator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsPayloadIterator.java Mon Jan 14 18:54:22 2013
@@ -2,7 +2,6 @@ package org.apache.lucene.facet.associat
import java.io.IOException;
-import org.apache.lucene.index.IndexReader;
import org.apache.lucene.util.collections.IntToFloatMap;
/*
@@ -31,9 +30,8 @@ public class FloatAssociationsPayloadIte
private final IntToFloatMap ordinalAssociations = new IntToFloatMap();
- public FloatAssociationsPayloadIterator(IndexReader reader, String field, CategoryFloatAssociation association)
- throws IOException {
- super(reader, field, association);
+ public FloatAssociationsPayloadIterator(String field, CategoryFloatAssociation association) throws IOException {
+ super(field, association);
}
@Override
Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsPayloadIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsPayloadIterator.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsPayloadIterator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsPayloadIterator.java Mon Jan 14 18:54:22 2013
@@ -2,7 +2,6 @@ package org.apache.lucene.facet.associat
import java.io.IOException;
-import org.apache.lucene.index.IndexReader;
import org.apache.lucene.util.collections.IntToIntMap;
/*
@@ -31,9 +30,8 @@ public class IntAssociationsPayloadItera
private final IntToIntMap ordinalAssociations = new IntToIntMap();
- public IntAssociationsPayloadIterator(IndexReader reader, String field, CategoryIntAssociation association)
- throws IOException {
- super(reader, field, association);
+ public IntAssociationsPayloadIterator(String field, CategoryIntAssociation association) throws IOException {
+ super(field, association);
}
@Override
Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java Mon Jan 14 18:54:22 2013
@@ -3,13 +3,10 @@ package org.apache.lucene.facet.index.pa
import java.io.IOException;
import java.io.Serializable;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.Term;
-
import org.apache.lucene.facet.search.CategoryListIterator;
import org.apache.lucene.facet.search.PayloadCategoryListIteraor;
-import org.apache.lucene.facet.search.TotalFacetCounts;
import org.apache.lucene.facet.util.PartitionsUtils;
+import org.apache.lucene.index.Term;
import org.apache.lucene.util.encoding.DGapIntEncoder;
import org.apache.lucene.util.encoding.IntDecoder;
import org.apache.lucene.util.encoding.IntEncoder;
@@ -98,11 +95,6 @@ public class CategoryListParams implemen
return new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder())));
}
- /**
- * Equality is defined by the 'term' that defines this category list.
- * Sub-classes should override this method if a more complex calculation
- * is needed to ensure equality.
- */
@Override
public boolean equals(Object o) {
if (o == this) {
@@ -121,29 +113,16 @@ public class CategoryListParams implemen
return this.term.equals(other.term);
}
- /**
- * Hashcode is similar to {@link #equals(Object)}, in that it uses
- * the term that defines this category list to derive the hashcode.
- * Subclasses need to ensure that equality/hashcode is correctly defined,
- * or there could be side-effects in the {@link TotalFacetCounts} caching
- * mechanism (as the filename for a Total Facet Counts array cache
- * is dependent on the hashCode, so it should consistently return the same
- * hash for identity).
- */
@Override
public int hashCode() {
return this.hashCode;
}
- /**
- * Create the category list iterator for the specified partition.
- */
- public CategoryListIterator createCategoryListIterator(IndexReader reader,
- int partition) throws IOException {
+ /** Create the {@link CategoryListIterator} for the specified partition. */
+ public CategoryListIterator createCategoryListIterator(int partition) throws IOException {
String categoryListTermStr = PartitionsUtils.partitionName(this, partition);
Term payloadTerm = new Term(term.field(), categoryListTermStr);
- return new PayloadCategoryListIteraor(reader, payloadTerm,
- createEncoder().createMatchingDecoder());
+ return new PayloadCategoryListIteraor(payloadTerm, createEncoder().createMatchingDecoder());
}
}
\ No newline at end of file
Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java Mon Jan 14 18:54:22 2013
@@ -50,7 +50,7 @@ public final class AdaptiveFacetsAccumul
* Create an {@link AdaptiveFacetsAccumulator}
* @see StandardFacetsAccumulator#StandardFacetsAccumulator(FacetSearchParams, IndexReader, TaxonomyReader)
*/
- public AdaptiveFacetsAccumulator(FacetSearchParams searchParams, IndexReader indexReader,
+ public AdaptiveFacetsAccumulator(FacetSearchParams searchParams, IndexReader indexReader,
TaxonomyReader taxonomyReader) {
super(searchParams, indexReader, taxonomyReader);
}
Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java Mon Jan 14 18:54:22 2013
@@ -2,6 +2,7 @@ package org.apache.lucene.facet.search;
import java.io.IOException;
+import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.util.IntsRef;
/*
@@ -23,6 +24,8 @@ import org.apache.lucene.util.IntsRef;
/**
* An interface for obtaining the category ordinals of documents.
+ * {@link #getOrdinals(int, IntsRef)} calls are done with document IDs that are
+ * local to the reader given to {@link #setNextReader(AtomicReaderContext)}.
* <p>
* <b>NOTE:</b> this class operates as a key to a map, and therefore you should
* implement {@code equals()} and {@code hashCode()} for proper behavior.
@@ -32,19 +35,20 @@ import org.apache.lucene.util.IntsRef;
public interface CategoryListIterator {
/**
- * Initializes the iterator. This method must be called before any calls to
- * {@link #getOrdinals(int, IntsRef)}, and its return value indicates whether there are
- * any relevant documents for this iterator.
+ * Sets the {@link AtomicReaderContext} for which
+ * {@link #getOrdinals(int, IntsRef)} calls will be made. Returns true iff any
+ * of the documents in this reader have category ordinals. This method must be
+ * called before any calls to {@link #getOrdinals(int, IntsRef)}.
*/
- public boolean init() throws IOException;
-
+ public boolean setNextReader(AtomicReaderContext context) throws IOException;
+
/**
* Stores the category ordinals of the given document ID in the given
* {@link IntsRef}, starting at position 0 upto {@link IntsRef#length}. Grows
* the {@link IntsRef} if it is not large enough.
*
* <p>
- * <b>NOTE:</b> if the requested document does not category ordinals
+ * <b>NOTE:</b> if the requested document does not have category ordinals
* associated with it, {@link IntsRef#length} is set to zero.
*/
public void getOrdinals(int docID, IntsRef ints) throws IOException;
Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadCategoryListIteraor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadCategoryListIteraor.java?rev=1433035&r1=1433034&r2=1433035&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadCategoryListIteraor.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadCategoryListIteraor.java Mon Jan 14 18:54:22 2013
@@ -2,7 +2,7 @@ package org.apache.lucene.facet.search;
import java.io.IOException;
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
@@ -34,17 +34,15 @@ import org.apache.lucene.util.encoding.I
public class PayloadCategoryListIteraor implements CategoryListIterator {
private final IntDecoder decoder;
- private final IndexReader indexReader;
private final Term term;
private final PayloadIterator pi;
private final int hashCode;
- public PayloadCategoryListIteraor(IndexReader indexReader, Term term, IntDecoder decoder) throws IOException {
- pi = new PayloadIterator(indexReader, term);
+ public PayloadCategoryListIteraor(Term term, IntDecoder decoder) throws IOException {
+ pi = new PayloadIterator(term);
this.decoder = decoder;
- hashCode = indexReader.hashCode() ^ term.hashCode();
+ hashCode = term.hashCode();
this.term = term;
- this.indexReader = indexReader;
}
@Override
@@ -58,7 +56,7 @@ public class PayloadCategoryListIteraor
}
// Hash codes are the same, check equals() to avoid cases of hash-collisions.
- return indexReader.equals(that.indexReader) && term.equals(that.term);
+ return term.equals(that.term);
}
@Override
@@ -67,8 +65,8 @@ public class PayloadCategoryListIteraor
}
@Override
- public boolean init() throws IOException {
- return pi.init();
+ public boolean setNextReader(AtomicReaderContext context) throws IOException {
+ return pi.setNextReader(context);
}
@Override