You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2012/01/30 00:19:07 UTC
svn commit: r1237505 [2/3] - in /lucene/dev/branches/lucene2858: ./ lucene/
lucene/contrib/ lucene/contrib/misc/ lucene/contrib/misc/src/java/
lucene/contrib/misc/src/java/org/apache/lucene/store/
lucene/src/java/org/apache/lucene/codecs/ lucene/src/ja...
Modified: lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/FST.java?rev=1237505&r1=1237504&r2=1237505&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/FST.java (original)
+++ lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/FST.java Sun Jan 29 23:19:05 2012
@@ -25,6 +25,8 @@ import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.Map;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
@@ -33,12 +35,23 @@ import org.apache.lucene.store.OutputStr
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.PriorityQueue;
+import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.fst.Builder.UnCompiledNode;
+// TODO: break this into WritableFST and ReadOnlyFST.. then
+// we can have subclasses of ReadOnlyFST to handle the
+// different byte[] level encodings (packed or
+// not)... and things like nodeCount, arcCount are read only
+
// TODO: if FST is pure prefix trie we can do a more compact
// 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!!
@@ -52,13 +65,15 @@ import org.apache.lucene.util.fst.Builde
*
* @lucene.experimental
*/
-public class FST<T> {
+public final class FST<T> {
public static enum INPUT_TYPE {BYTE1, BYTE2, BYTE4};
public final INPUT_TYPE inputType;
private final static int BIT_FINAL_ARC = 1 << 0;
private final static int BIT_LAST_ARC = 1 << 1;
- private final static int BIT_TARGET_NEXT = 1 << 2;
+ final static int BIT_TARGET_NEXT = 1 << 2;
+
+ // TODO: we can free up a bit if we can nuke this:
private final static int BIT_STOP_NODE = 1 << 3;
private final static int BIT_ARC_HAS_OUTPUT = 1 << 4;
private final static int BIT_ARC_HAS_FINAL_OUTPUT = 1 << 5;
@@ -66,7 +81,12 @@ public class FST<T> {
// Arcs are stored as fixed-size (per entry) array, so
// that we can find an arc using binary search. We do
// this when number of arcs is > NUM_ARCS_ARRAY:
- private final static int BIT_ARCS_AS_FIXED_ARRAY = 1 << 6;
+
+ // If set, the target node is delta coded vs current
+ // position:
+ private final static int BIT_TARGET_DELTA = 1 << 6;
+
+ private final static byte ARCS_AS_FIXED_ARRAY = BIT_ARC_HAS_FINAL_OUTPUT;
/**
* @see #shouldExpand(UnCompiledNode)
@@ -95,7 +115,10 @@ public class FST<T> {
/** Write BYTE2 labels as 2-byte short, not vInt. */
private final static int VERSION_SHORT_BYTE2_LABELS = 2;
- private final static int VERSION_CURRENT = VERSION_SHORT_BYTE2_LABELS;
+ /** Added optional packed format. */
+ private final static int VERSION_PACKED = 3;
+
+ private final static int VERSION_CURRENT = VERSION_PACKED;
// Never serialized; just used to represent the virtual
// final node w/ no arcs:
@@ -126,6 +149,9 @@ public class FST<T> {
public int arcCount;
public int arcWithOutputCount;
+ private final boolean packed;
+ private final int[] nodeRefToAddress;
+
// If arc has this label then that arc is final/accepted
public static final int END_LABEL = -1;
@@ -137,10 +163,17 @@ public class FST<T> {
public int label;
public T output;
+ // From node (ord or address); currently only used when
+ // building an FST w/ willPackFST=true:
+ int node;
+
+ // To node (ord or address):
public int target;
byte flags;
public T nextFinalOutput;
+
+ // address (into the byte[]), or ord/address if label == END_LABEL
int nextArc;
// This is non-zero if current arcs are fixed array:
@@ -151,19 +184,18 @@ public class FST<T> {
/** Returns this */
public Arc<T> copyFrom(Arc<T> other) {
+ node = other.node;
label = other.label;
target = other.target;
flags = other.flags;
output = other.output;
nextFinalOutput = other.nextFinalOutput;
nextArc = other.nextArc;
- if (other.bytesPerArc != 0) {
- bytesPerArc = other.bytesPerArc;
+ bytesPerArc = other.bytesPerArc;
+ if (bytesPerArc != 0) {
posArcsStart = other.posArcsStart;
arcIdx = other.arcIdx;
numArcs = other.numArcs;
- } else {
- bytesPerArc = 0;
}
return this;
}
@@ -179,40 +211,91 @@ public class FST<T> {
public boolean isFinal() {
return flag(BIT_FINAL_ARC);
}
+
+ @Override
+ public String toString() {
+ StringBuilder b = new StringBuilder();
+ b.append("node=" + node);
+ b.append(" target=" + target);
+ b.append(" label=" + label);
+ if (flag(BIT_LAST_ARC)) {
+ b.append(" last");
+ }
+ if (flag(BIT_FINAL_ARC)) {
+ b.append(" final");
+ }
+ if (flag(BIT_TARGET_NEXT)) {
+ b.append(" targetNext");
+ }
+ if (flag(BIT_ARC_HAS_OUTPUT)) {
+ b.append(" hasOutput");
+ }
+ if (flag(BIT_ARC_HAS_FINAL_OUTPUT)) {
+ b.append(" hasOutput");
+ }
+ if (bytesPerArc != 0) {
+ b.append(" arcArray(idx=" + arcIdx + " of " + numArcs + ")");
+ }
+ return b.toString();
+ }
};
- static boolean flag(int flags, int bit) {
+ private final static boolean flag(int flags, int bit) {
return (flags & bit) != 0;
}
private final BytesWriter writer;
- // make a new empty FST, for building
- public FST(INPUT_TYPE inputType, Outputs<T> outputs) {
+ // TODO: we can save RAM here by using growable packed
+ // ints...:
+ private int[] nodeAddress;
+
+ // TODO: we could be smarter here, and prune periodically
+ // as we go; high in-count nodes will "usually" become
+ // clear early on:
+ private int[] inCounts;
+
+ // make a new empty FST, for building; Builder invokes
+ // this ctor
+ FST(INPUT_TYPE inputType, Outputs<T> outputs, boolean willPackFST) {
this.inputType = inputType;
this.outputs = outputs;
bytes = new byte[128];
NO_OUTPUT = outputs.getNoOutput();
+ if (willPackFST) {
+ nodeAddress = new int[8];
+ inCounts = new int[8];
+ } else {
+ nodeAddress = null;
+ inCounts = null;
+ }
writer = new BytesWriter();
emptyOutput = null;
+ packed = false;
+ nodeRefToAddress = null;
}
- // create an existing FST
+ /** 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_SHORT_BYTE2_LABELS, VERSION_SHORT_BYTE2_LABELS);
+ CodecUtil.checkHeader(in, FILE_FORMAT_NAME, VERSION_PACKED, VERSION_PACKED);
+ packed = in.readByte() == 1;
if (in.readByte() == 1) {
// accepts empty string
int numBytes = in.readVInt();
// messy
bytes = new byte[numBytes];
in.readBytes(bytes, 0, numBytes);
- emptyOutput = outputs.read(getBytesReader(numBytes-1));
+ if (packed) {
+ emptyOutput = outputs.read(getBytesReader(0));
+ } else {
+ emptyOutput = outputs.read(getBytesReader(numBytes-1));
+ }
} else {
emptyOutput = null;
}
@@ -230,6 +313,15 @@ public class FST<T> {
default:
throw new IllegalStateException("invalid input type " + t);
}
+ if (packed) {
+ final int nodeRefCount = in.readVInt();
+ nodeRefToAddress = new int[nodeRefCount];
+ for(int idx=0;idx<nodeRefCount;idx++) {
+ nodeRefToAddress[idx] = in.readVInt();
+ }
+ } else {
+ nodeRefToAddress = null;
+ }
startNode = in.readVInt();
nodeCount = in.readVInt();
arcCount = in.readVInt();
@@ -248,7 +340,14 @@ public class FST<T> {
/** Returns bytes used to represent the FST */
public int sizeInBytes() {
- return bytes.length;
+ int size = bytes.length;
+ if (packed) {
+ size += nodeRefToAddress.length * RamUsageEstimator.NUM_BYTES_INT;
+ } else if (nodeAddress != null) {
+ size += nodeAddress.length * RamUsageEstimator.NUM_BYTES_INT;
+ size += inCounts.length * RamUsageEstimator.NUM_BYTES_INT;
+ }
+ return size;
}
void finish(int startNode) throws IOException {
@@ -266,15 +365,25 @@ public class FST<T> {
cacheRootArcs();
}
+ private int getNodeAddress(int node) {
+ if (nodeAddress != null) {
+ // Deref
+ return nodeAddress[node];
+ } else {
+ // Straight
+ return node;
+ }
+ }
+
// Caches first 128 labels
@SuppressWarnings("unchecked")
private void cacheRootArcs() throws IOException {
- cachedRootArcs = (FST.Arc<T>[]) new FST.Arc[0x80];
- final FST.Arc<T> arc = new FST.Arc<T>();
+ cachedRootArcs = (Arc<T>[]) new Arc[0x80];
+ final Arc<T> arc = new Arc<T>();
getFirstArc(arc);
final BytesReader in = getBytesReader(0);
if (targetHasArcs(arc)) {
- readFirstRealArc(arc.target, arc, in);
+ readFirstRealTargetArc(arc.target, arc, in);
while(true) {
assert arc.label != END_LABEL;
if (arc.label < cachedRootArcs.length) {
@@ -307,14 +416,16 @@ public class FST<T> {
outputs.write(emptyOutput, writer);
emptyOutputBytes = new byte[writer.posWrite-posSave];
- // reverse
- final int stopAt = (writer.posWrite - posSave)/2;
- int upto = 0;
- while(upto < stopAt) {
- final byte b = bytes[posSave + upto];
- bytes[posSave+upto] = bytes[writer.posWrite-upto-1];
- bytes[writer.posWrite-upto-1] = b;
- upto++;
+ if (!packed) {
+ // reverse
+ final int stopAt = (writer.posWrite - posSave)/2;
+ int upto = 0;
+ while(upto < stopAt) {
+ final byte b = bytes[posSave + upto];
+ bytes[posSave+upto] = bytes[writer.posWrite-upto-1];
+ bytes[writer.posWrite-upto-1] = b;
+ upto++;
+ }
}
System.arraycopy(bytes, posSave, emptyOutputBytes, 0, writer.posWrite-posSave);
writer.posWrite = posSave;
@@ -324,7 +435,15 @@ public class FST<T> {
if (startNode == -1) {
throw new IllegalStateException("call finish first");
}
+ if (nodeAddress != null) {
+ throw new IllegalStateException("cannot save an FST pre-packed FST; it must first be packed");
+ }
CodecUtil.writeHeader(out, FILE_FORMAT_NAME, VERSION_CURRENT);
+ if (packed) {
+ out.writeByte((byte) 1);
+ } else {
+ out.writeByte((byte) 0);
+ }
// TODO: really we should encode this as an arc, arriving
// to the root node, instead of special casing here:
if (emptyOutput != null) {
@@ -343,6 +462,13 @@ public class FST<T> {
t = 2;
}
out.writeByte(t);
+ if (packed) {
+ assert nodeRefToAddress != null;
+ out.writeVInt(nodeRefToAddress.length);
+ for(int idx=0;idx<nodeRefToAddress.length;idx++) {
+ out.writeVInt(nodeRefToAddress[idx]);
+ }
+ }
out.writeVInt(startNode);
out.writeVInt(nodeCount);
out.writeVInt(arcCount);
@@ -418,16 +544,16 @@ public class FST<T> {
// returns true if the node at this address has any
// outgoing arcs
- public boolean targetHasArcs(Arc<T> arc) {
+ public static<T> boolean targetHasArcs(Arc<T> arc) {
return arc.target > 0;
}
// serializes new node by appending its bytes to the end
// of the current byte[]
- int addNode(Builder.UnCompiledNode<T> node) throws IOException {
- //System.out.println("FST.addNode pos=" + posWrite + " numArcs=" + node.numArcs);
- if (node.numArcs == 0) {
- if (node.isFinal) {
+ int addNode(Builder.UnCompiledNode<T> nodeIn) throws IOException {
+ //System.out.println("FST.addNode pos=" + writer.posWrite + " numArcs=" + nodeIn.numArcs);
+ if (nodeIn.numArcs == 0) {
+ if (nodeIn.isFinal) {
return FINAL_END_NODE;
} else {
return NON_FINAL_END_NODE;
@@ -437,15 +563,15 @@ public class FST<T> {
int startAddress = writer.posWrite;
//System.out.println(" startAddr=" + startAddress);
- final boolean doFixedArray = shouldExpand(node);
+ final boolean doFixedArray = shouldExpand(nodeIn);
final int fixedArrayStart;
if (doFixedArray) {
- if (bytesPerArc.length < node.numArcs) {
- bytesPerArc = new int[ArrayUtil.oversize(node.numArcs, 1)];
+ if (bytesPerArc.length < nodeIn.numArcs) {
+ bytesPerArc = new int[ArrayUtil.oversize(nodeIn.numArcs, 1)];
}
// write a "false" first arc:
- writer.writeByte((byte) BIT_ARCS_AS_FIXED_ARRAY);
- writer.writeVInt(node.numArcs);
+ 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
@@ -456,15 +582,14 @@ public class FST<T> {
fixedArrayStart = 0;
}
- nodeCount++;
- arcCount += node.numArcs;
+ arcCount += nodeIn.numArcs;
- final int lastArc = node.numArcs-1;
+ final int lastArc = nodeIn.numArcs-1;
int lastArcStart = writer.posWrite;
int maxBytesPerArc = 0;
- for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
- final Builder.Arc<T> arc = node.arcs[arcIdx];
+ 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;
@@ -472,7 +597,10 @@ public class FST<T> {
flags += BIT_LAST_ARC;
}
- if (lastFrozenNode == target.address && !doFixedArray) {
+ if (lastFrozenNode == target.node && !doFixedArray) {
+ // TODO: for better perf (but more RAM used) we
+ // could avoid this except when arc is "near" the
+ // last arc:
flags += BIT_TARGET_NEXT;
}
@@ -485,10 +613,12 @@ public class FST<T> {
assert arc.nextFinalOutput == NO_OUTPUT;
}
- boolean targetHasArcs = target.address > 0;
+ boolean targetHasArcs = target.node > 0;
if (!targetHasArcs) {
flags += BIT_STOP_NODE;
+ } else if (inCounts != null) {
+ inCounts[target.node]++;
}
if (arc.output != NO_OUTPUT) {
@@ -498,19 +628,23 @@ public class FST<T> {
writer.writeByte((byte) flags);
writeLabel(arc.label);
- //System.out.println(" write arc: label=" + arc.label + " flags=" + flags);
+ // System.out.println(" write arc: label=" + (char) arc.label + " flags=" + flags + " target=" + target.node + " pos=" + writer.posWrite + " output=" + outputs.outputToString(arc.output));
if (arc.output != NO_OUTPUT) {
outputs.write(arc.output, writer);
+ //System.out.println(" write output");
arcWithOutputCount++;
}
+
if (arc.nextFinalOutput != NO_OUTPUT) {
+ //System.out.println(" write final output");
outputs.write(arc.nextFinalOutput, writer);
}
- if (targetHasArcs && (doFixedArray || lastFrozenNode != target.address)) {
- assert target.address > 0;
- writer.writeInt(target.address);
+ if (targetHasArcs && (flags & BIT_TARGET_NEXT) == 0) {
+ assert target.node > 0;
+ //System.out.println(" write target");
+ writer.writeInt(target.node);
}
// just write the arcs "like normal" on first pass,
@@ -530,10 +664,11 @@ public class FST<T> {
// such cases
if (doFixedArray) {
+ //System.out.println(" doFixedArray");
assert maxBytesPerArc > 0;
// 2nd pass just "expands" all arcs to take up a fixed
// byte size
- final int sizeNeeded = fixedArrayStart + node.numArcs * maxBytesPerArc;
+ final int sizeNeeded = fixedArrayStart + nodeIn.numArcs * maxBytesPerArc;
bytes = ArrayUtil.grow(bytes, sizeNeeded);
// TODO: we could make this a vInt instead
bytes[fixedArrayStart-4] = (byte) (maxBytesPerArc >> 24);
@@ -543,9 +678,9 @@ public class FST<T> {
// expand the arcs in place, backwards
int srcPos = writer.posWrite;
- int destPos = fixedArrayStart + node.numArcs*maxBytesPerArc;
+ int destPos = fixedArrayStart + nodeIn.numArcs*maxBytesPerArc;
writer.posWrite = destPos;
- for(int arcIdx=node.numArcs-1;arcIdx>=0;arcIdx--) {
+ for(int arcIdx=nodeIn.numArcs-1;arcIdx>=0;arcIdx--) {
//System.out.println(" repack arcIdx=" + arcIdx + " srcPos=" + srcPos + " destPos=" + destPos);
destPos -= maxBytesPerArc;
srcPos -= bytesPerArc[arcIdx];
@@ -559,7 +694,7 @@ public class FST<T> {
// 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 = lastFrozenNode = writer.posWrite - 1;
+ final int endAddress = writer.posWrite - 1;
int left = startAddress;
int right = endAddress;
@@ -568,13 +703,31 @@ public class FST<T> {
bytes[left++] = bytes[right];
bytes[right--] = b;
}
+ //System.out.println(" endAddress=" + endAddress);
- return endAddress;
+ nodeCount++;
+ final int node;
+ if (nodeAddress != null) {
+ // Nodes are addressed by 1+ord:
+ if (nodeCount == nodeAddress.length) {
+ nodeAddress = ArrayUtil.grow(nodeAddress);
+ inCounts = ArrayUtil.grow(inCounts);
+ }
+ nodeAddress[nodeCount] = endAddress;
+ // System.out.println(" write nodeAddress[" + nodeCount + "] = " + endAddress);
+ node = nodeCount;
+ } else {
+ node = endAddress;
+ }
+ lastFrozenNode = node;
+
+ return node;
}
/** Fills virtual 'start' arc, ie, an empty incoming arc to
* the FST's start node */
public Arc<T> getFirstArc(Arc<T> arc) {
+
if (emptyOutput != null) {
arc.flags = BIT_FINAL_ARC | BIT_LAST_ARC;
arc.nextFinalOutput = emptyOutput;
@@ -585,7 +738,7 @@ public class FST<T> {
arc.output = NO_OUTPUT;
// If there are no nodes, ie, the FST only accepts the
- // empty string, then startNode is 0, and then readFirstTargetArc
+ // empty string, then startNode is 0
arc.target = startNode;
return arc;
}
@@ -602,20 +755,27 @@ public class FST<T> {
//System.out.println(" end node");
assert follow.isFinal();
arc.label = END_LABEL;
+ arc.target = FINAL_END_NODE;
arc.output = follow.nextFinalOutput;
arc.flags = BIT_LAST_ARC;
return arc;
} else {
- final BytesReader in = getBytesReader(follow.target);
- arc.flags = in.readByte();
- if (arc.flag(BIT_ARCS_AS_FIXED_ARRAY)) {
+ final BytesReader in = getBytesReader(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();
- arc.bytesPerArc = in.readInt();
+ if (packed) {
+ arc.bytesPerArc = in.readVInt();
+ } else {
+ arc.bytesPerArc = in.readInt();
+ }
//System.out.println(" array numArcs=" + arc.numArcs + " bpa=" + arc.bytesPerArc);
arc.posArcsStart = in.pos;
arc.arcIdx = arc.numArcs - 2;
} else {
+ arc.flags = b;
// non-array: linear scan
arc.bytesPerArc = 0;
//System.out.println(" scan");
@@ -631,11 +791,17 @@ public class FST<T> {
if (arc.flag(BIT_STOP_NODE)) {
} else if (arc.flag(BIT_TARGET_NEXT)) {
} else {
- in.pos -= 4;
+ if (packed) {
+ in.readVInt();
+ } else {
+ in.skip(4);
+ }
}
arc.flags = in.readByte();
}
- arc.nextArc = in.pos+1;
+ // Undo the byte flags we read:
+ in.skip(-1);
+ arc.nextArc = in.pos;
}
readNextRealArc(arc, in);
assert arc.isLast();
@@ -657,35 +823,48 @@ public class FST<T> {
// Insert "fake" final first arc:
arc.label = END_LABEL;
arc.output = follow.nextFinalOutput;
+ arc.flags = BIT_FINAL_ARC;
if (follow.target <= 0) {
- arc.flags = BIT_LAST_ARC | BIT_FINAL_ARC;
+ arc.flags |= BIT_LAST_ARC;
} else {
- arc.flags = BIT_FINAL_ARC;
+ arc.node = follow.target;
+ // NOTE: nextArc is a node (not an address!) in this case:
arc.nextArc = follow.target;
}
+ arc.target = FINAL_END_NODE;
//System.out.println(" insert isFinal; nextArc=" + follow.target + " isLast=" + arc.isLast() + " output=" + outputs.outputToString(arc.output));
return arc;
} else {
- return readFirstRealArc(follow.target, arc, getBytesReader(0));
+ return readFirstRealTargetArc(follow.target, arc, getBytesReader(0));
}
}
- public Arc<T> readFirstRealArc(int address, Arc<T> arc, final BytesReader in) throws IOException {
+ public Arc<T> readFirstRealTargetArc(int node, Arc<T> arc, final BytesReader in) throws IOException {
+ final int address = getNodeAddress(node);
in.pos = address;
- arc.flags = in.readByte();
+ //System.out.println(" readFirstRealTargtArc address="
+ //+ address);
+ //System.out.println(" flags=" + arc.flags);
+ arc.node = node;
- if (arc.flag(BIT_ARCS_AS_FIXED_ARRAY)) {
+ if (in.readByte() == ARCS_AS_FIXED_ARRAY) {
//System.out.println(" fixedArray");
// this is first arc in a fixed-array
arc.numArcs = in.readVInt();
- arc.bytesPerArc = in.readInt();
+ if (packed) {
+ arc.bytesPerArc = in.readVInt();
+ } else {
+ arc.bytesPerArc = in.readInt();
+ }
arc.arcIdx = -1;
arc.nextArc = arc.posArcsStart = in.pos;
//System.out.println(" bytesPer=" + arc.bytesPerArc + " numArcs=" + arc.numArcs + " arcsStart=" + pos);
} else {
+ //arc.flags = b;
arc.nextArc = address;
arc.bytesPerArc = 0;
}
+
return readNextRealArc(arc, in);
}
@@ -699,9 +878,8 @@ public class FST<T> {
if (!targetHasArcs(follow)) {
return false;
} else {
- final BytesReader in = getBytesReader(follow.target);
- final byte b = in.readByte();
- return (b & BIT_ARCS_AS_FIXED_ARRAY) != 0;
+ final BytesReader in = getBytesReader(getNodeAddress(follow.target));
+ return in.readByte() == ARCS_AS_FIXED_ARRAY;
}
}
@@ -710,10 +888,9 @@ public class FST<T> {
if (arc.label == END_LABEL) {
// This was a fake inserted "final" arc
if (arc.nextArc <= 0) {
- // This arc went to virtual final node, ie has no outgoing arcs
- return null;
+ throw new IllegalArgumentException("cannot readNextArc when arc.isLast()=true");
}
- return readFirstRealArc(arc.nextArc, arc, getBytesReader(0));
+ return readFirstRealTargetArc(arc.nextArc, arc, getBytesReader(0));
} else {
return readNextRealArc(arc, getBytesReader(0));
}
@@ -727,19 +904,24 @@ public class FST<T> {
final BytesReader in;
if (arc.label == END_LABEL) {
//System.out.println(" nextArc fake " + arc.nextArc);
- in = getBytesReader(arc.nextArc);
- byte flags = bytes[in.pos];
- if (flag(flags, BIT_ARCS_AS_FIXED_ARRAY)) {
+ in = getBytesReader(getNodeAddress(arc.nextArc));
+ final byte b = bytes[in.pos];
+ if (b == ARCS_AS_FIXED_ARRAY) {
//System.out.println(" nextArc fake array");
- in.pos--;
+ in.skip(1);
in.readVInt();
- in.readInt();
+ if (packed) {
+ in.readVInt();
+ } else {
+ in.readInt();
+ }
}
} else {
if (arc.bytesPerArc != 0) {
//System.out.println(" nextArc real array");
// arcs are at fixed entries
- in = getBytesReader(arc.posArcsStart - (1+arc.arcIdx)*arc.bytesPerArc);
+ in = getBytesReader(arc.posArcsStart);
+ in.skip((1+arc.arcIdx)*arc.bytesPerArc);
} else {
// arcs are packed
//System.out.println(" nextArc real packed");
@@ -754,12 +936,16 @@ public 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 {
+
+ // TODO: can't assert this because we call from readFirstArc
+ // assert !flag(arc.flags, BIT_LAST_ARC);
+
// this is a continuing arc in a fixed array
if (arc.bytesPerArc != 0) {
// arcs are at fixed entries
arc.arcIdx++;
assert arc.arcIdx < arc.numArcs;
- in.pos = arc.posArcsStart - arc.arcIdx*arc.bytesPerArc;
+ in.skip(arc.posArcsStart, arc.arcIdx*arc.bytesPerArc);
} else {
// arcs are packed
in.pos = arc.nextArc;
@@ -788,45 +974,61 @@ public class FST<T> {
arc.nextArc = in.pos;
} else if (arc.flag(BIT_TARGET_NEXT)) {
arc.nextArc = in.pos;
- if (!arc.flag(BIT_LAST_ARC)) {
- if (arc.bytesPerArc == 0) {
- // must scan
- seekToNextNode(in);
- } else {
- in.pos = arc.posArcsStart - arc.bytesPerArc * arc.numArcs;
+ // TODO: would be nice to make this lazy -- maybe
+ // caller doesn't need the target and is scanning arcs...
+ if (nodeAddress == null) {
+ if (!arc.flag(BIT_LAST_ARC)) {
+ if (arc.bytesPerArc == 0) {
+ // must scan
+ seekToNextNode(in);
+ } else {
+ in.skip(arc.posArcsStart, arc.bytesPerArc * arc.numArcs);
+ }
}
+ arc.target = in.pos;
+ } else {
+ arc.target = arc.node - 1;
+ assert arc.target > 0;
}
- arc.target = in.pos;
} else {
- arc.target = in.readInt();
+ if (packed) {
+ final int pos = in.pos;
+ final int code = in.readVInt();
+ 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.length) {
+ // Deref
+ arc.target = nodeRefToAddress[code];
+ //System.out.println(" deref code=" + code + " target=" + arc.target);
+ } else {
+ // Absolute
+ arc.target = code;
+ //System.out.println(" abs code=" + code + " derefLen=" + nodeRefToAddress.length);
+ }
+ } else {
+ arc.target = in.readInt();
+ }
arc.nextArc = in.pos;
}
-
return arc;
}
/** Finds an arc leaving the incoming arc, replacing the arc in place.
* This returns null if the arc was not found, else the incoming arc. */
- public Arc<T> findTargetArc(int labelToMatch, Arc<T> follow, Arc<T> arc) throws IOException {
+ public Arc<T> findTargetArc(int labelToMatch, Arc<T> follow, Arc<T> arc, BytesReader in) throws IOException {
assert cachedRootArcs != null;
- // Short-circuit if this arc is in the root arc cache:
- if (follow.target == startNode && labelToMatch != END_LABEL && labelToMatch < cachedRootArcs.length) {
- final Arc<T> result = cachedRootArcs[labelToMatch];
- if (result == null) {
- return result;
- } else {
- arc.copyFrom(result);
- return arc;
- }
- }
-
+
if (labelToMatch == END_LABEL) {
if (follow.isFinal()) {
if (follow.target <= 0) {
arc.flags = BIT_LAST_ARC;
} else {
arc.flags = 0;
+ // NOTE: nextArc is a node (not an address!) in this case:
arc.nextArc = follow.target;
+ arc.node = follow.target;
}
arc.output = follow.nextFinalOutput;
arc.label = END_LABEL;
@@ -836,35 +1038,49 @@ public class FST<T> {
}
}
+ // Short-circuit if this arc is in the root arc cache:
+ if (follow.target == startNode && labelToMatch < cachedRootArcs.length) {
+ final Arc<T> result = cachedRootArcs[labelToMatch];
+ if (result == null) {
+ return result;
+ } else {
+ arc.copyFrom(result);
+ return arc;
+ }
+ }
+
if (!targetHasArcs(follow)) {
return null;
}
- // TODO: maybe make an explicit thread state that holds
- // reusable stuff eg BytesReader:
- final BytesReader in = getBytesReader(follow.target);
+ in.pos = getNodeAddress(follow.target);
+
+ arc.node = follow.target;
// System.out.println("fta label=" + (char) labelToMatch);
- if ((in.readByte() & BIT_ARCS_AS_FIXED_ARRAY) != 0) {
+ if (in.readByte() == ARCS_AS_FIXED_ARRAY) {
// Arcs are full array; do binary search:
arc.numArcs = in.readVInt();
- //System.out.println(" bs " + arc.numArcs);
- arc.bytesPerArc = in.readInt();
+ if (packed) {
+ arc.bytesPerArc = in.readVInt();
+ } else {
+ arc.bytesPerArc = in.readInt();
+ }
arc.posArcsStart = in.pos;
int low = 0;
int high = arc.numArcs-1;
while (low <= high) {
//System.out.println(" cycle");
int mid = (low + high) >>> 1;
- in.pos = arc.posArcsStart - arc.bytesPerArc*mid - 1;
+ in.skip(arc.posArcsStart, arc.bytesPerArc*mid + 1);
int midLabel = readLabel(in);
final int cmp = midLabel - labelToMatch;
- if (cmp < 0)
+ if (cmp < 0) {
low = mid + 1;
- else if (cmp > 0)
+ } else if (cmp > 0) {
high = mid - 1;
- else {
+ } else {
arc.arcIdx = mid-1;
//System.out.println(" found!");
return readNextRealArc(arc, in);
@@ -875,7 +1091,8 @@ public class FST<T> {
}
// Linear scan
- readFirstTargetArc(follow, arc);
+ readFirstRealTargetArc(follow.target, arc, in);
+
while(true) {
//System.out.println(" non-bs cycle");
// TODO: we should fix this code to not have to create
@@ -889,7 +1106,7 @@ public class FST<T> {
} else if (arc.isLast()) {
return null;
} else {
- readNextArc(arc);
+ readNextRealArc(arc, in);
}
}
}
@@ -910,7 +1127,11 @@ public class FST<T> {
}
if (!flag(flags, BIT_STOP_NODE) && !flag(flags, BIT_TARGET_NEXT)) {
- in.readInt();
+ if (packed) {
+ in.readVInt();
+ } else {
+ in.readInt();
+ }
}
if (flag(flags, BIT_LAST_ARC)) {
@@ -969,6 +1190,7 @@ public class FST<T> {
@Override
public void writeByte(byte b) {
+ assert posWrite <= bytes.length;
if (bytes.length == posWrite) {
bytes = ArrayUtil.grow(bytes);
}
@@ -976,6 +1198,13 @@ public class FST<T> {
bytes[posWrite++] = b;
}
+ public void setPosWrite(int posWrite) {
+ this.posWrite = posWrite;
+ if (bytes.length < posWrite) {
+ bytes = ArrayUtil.grow(bytes, posWrite);
+ }
+ }
+
@Override
public void writeBytes(byte[] b, int offset, int length) {
final int size = posWrite + length;
@@ -987,15 +1216,24 @@ public class FST<T> {
public final BytesReader getBytesReader(int pos) {
// TODO: maybe re-use via ThreadLocal?
- return new BytesReader(bytes, pos);
+ if (packed) {
+ return new ForwardBytesReader(bytes, pos);
+ } else {
+ return new ReverseBytesReader(bytes, pos);
+ }
}
/** Expert */
- public final static class BytesReader extends DataInput {
- final byte[] bytes;
+ public static abstract class BytesReader extends DataInput {
int pos;
+ abstract void skip(int byteCount);
+ abstract void skip(int base, int byteCount);
+ }
- public BytesReader(byte[] bytes, int pos) {
+ final static class ReverseBytesReader extends BytesReader {
+ final byte[] bytes;
+
+ public ReverseBytesReader(byte[] bytes, int pos) {
this.bytes = bytes;
this.pos = pos;
}
@@ -1011,5 +1249,541 @@ public class FST<T> {
b[offset+i] = bytes[pos--];
}
}
+
+ public void skip(int count) {
+ pos -= count;
+ }
+
+ public void skip(int base, int count) {
+ pos = base - count;
+ }
+ }
+
+ // TODO: can we use just ByteArrayDataInput...? need to
+ // add a .skipBytes to DataInput.. hmm and .setPosition
+ final static class ForwardBytesReader extends BytesReader {
+ final byte[] bytes;
+
+ public ForwardBytesReader(byte[] bytes, int pos) {
+ this.bytes = bytes;
+ this.pos = pos;
+ }
+
+ @Override
+ public byte readByte() {
+ return bytes[pos++];
+ }
+
+ @Override
+ public void readBytes(byte[] b, int offset, int len) {
+ System.arraycopy(bytes, pos, b, offset, len);
+ pos += len;
+ }
+
+ public void skip(int count) {
+ pos += count;
+ }
+
+ public void skip(int base, int count) {
+ pos = base + count;
+ }
+ }
+
+ private static class ArcAndState<T> {
+ final Arc<T> arc;
+ final IntsRef chain;
+
+ public ArcAndState(Arc<T> arc, IntsRef chain) {
+ this.arc = arc;
+ this.chain = chain;
+ }
+ }
+
+ /*
+ public void countSingleChains() throws IOException {
+ // TODO: must assert this FST was built with
+ // "willRewrite"
+
+ final List<ArcAndState<T>> queue = new ArrayList<ArcAndState<T>>();
+
+ // TODO: use bitset to not revisit nodes already
+ // visited
+
+ FixedBitSet seen = new FixedBitSet(1+nodeCount);
+ int saved = 0;
+
+ queue.add(new ArcAndState<T>(getFirstArc(new Arc<T>()), new IntsRef()));
+ Arc<T> scratchArc = new Arc<T>();
+ while(queue.size() > 0) {
+ //System.out.println("cycle size=" + queue.size());
+ //for(ArcAndState<T> ent : queue) {
+ // System.out.println(" " + Util.toBytesRef(ent.chain, new BytesRef()));
+ // }
+ final ArcAndState<T> arcAndState = queue.get(queue.size()-1);
+ seen.set(arcAndState.arc.node);
+ final BytesRef br = Util.toBytesRef(arcAndState.chain, new BytesRef());
+ if (br.length > 0 && br.bytes[br.length-1] == -1) {
+ br.length--;
+ }
+ //System.out.println(" top node=" + arcAndState.arc.target + " chain=" + br.utf8ToString());
+ if (targetHasArcs(arcAndState.arc) && !seen.get(arcAndState.arc.target)) {
+ // push
+ readFirstTargetArc(arcAndState.arc, scratchArc);
+ //System.out.println(" push label=" + (char) scratchArc.label);
+ //System.out.println(" tonode=" + scratchArc.target + " last?=" + scratchArc.isLast());
+
+ final IntsRef chain = IntsRef.deepCopyOf(arcAndState.chain);
+ chain.grow(1+chain.length);
+ // TODO
+ //assert scratchArc.label != END_LABEL;
+ chain.ints[chain.length] = scratchArc.label;
+ chain.length++;
+
+ if (scratchArc.isLast()) {
+ if (scratchArc.target != -1 && inCounts[scratchArc.target] == 1) {
+ //System.out.println(" append");
+ } else {
+ if (arcAndState.chain.length > 1) {
+ saved += chain.length-2;
+ try {
+ System.out.println("chain: " + Util.toBytesRef(chain, new BytesRef()).utf8ToString());
+ } catch (AssertionError ae) {
+ System.out.println("chain: " + Util.toBytesRef(chain, new BytesRef()));
+ }
+ }
+ chain.length = 0;
+ }
+ } else {
+ //System.out.println(" reset");
+ if (arcAndState.chain.length > 1) {
+ saved += arcAndState.chain.length-2;
+ try {
+ System.out.println("chain: " + Util.toBytesRef(arcAndState.chain, new BytesRef()).utf8ToString());
+ } catch (AssertionError ae) {
+ System.out.println("chain: " + Util.toBytesRef(arcAndState.chain, new BytesRef()));
+ }
+ }
+ if (scratchArc.target != -1 && inCounts[scratchArc.target] != 1) {
+ chain.length = 0;
+ } else {
+ chain.ints[0] = scratchArc.label;
+ chain.length = 1;
+ }
+ }
+ // TODO: instead of new Arc() we can re-use from
+ // a by-depth array
+ queue.add(new ArcAndState<T>(new Arc<T>().copyFrom(scratchArc), chain));
+ } else if (!arcAndState.arc.isLast()) {
+ // next
+ readNextArc(arcAndState.arc);
+ //System.out.println(" next label=" + (char) arcAndState.arc.label + " len=" + arcAndState.chain.length);
+ if (arcAndState.chain.length != 0) {
+ arcAndState.chain.ints[arcAndState.chain.length-1] = arcAndState.arc.label;
+ }
+ } else {
+ if (arcAndState.chain.length > 1) {
+ saved += arcAndState.chain.length-2;
+ System.out.println("chain: " + Util.toBytesRef(arcAndState.chain, new BytesRef()).utf8ToString());
+ }
+ // pop
+ //System.out.println(" pop");
+ queue.remove(queue.size()-1);
+ while(queue.size() > 0 && queue.get(queue.size()-1).arc.isLast()) {
+ queue.remove(queue.size()-1);
+ }
+ if (queue.size() > 0) {
+ final ArcAndState<T> arcAndState2 = queue.get(queue.size()-1);
+ readNextArc(arcAndState2.arc);
+ //System.out.println(" read next=" + (char) arcAndState2.arc.label + " queue=" + queue.size());
+ assert arcAndState2.arc.label != END_LABEL;
+ if (arcAndState2.chain.length != 0) {
+ arcAndState2.chain.ints[arcAndState2.chain.length-1] = arcAndState2.arc.label;
+ }
+ }
+ }
+ }
+
+ System.out.println("TOT saved " + saved);
+ }
+ */
+
+ // Creates a packed FST
+ private FST(INPUT_TYPE inputType, int[] nodeRefToAddress, Outputs<T> outputs) {
+ packed = true;
+ this.inputType = inputType;
+ bytes = new byte[128];
+ this.nodeRefToAddress = nodeRefToAddress;
+ this.outputs = outputs;
+ NO_OUTPUT = outputs.getNoOutput();
+ writer = new BytesWriter();
+ }
+
+ /** Expert: creates an FST by packing this one. This
+ * process requires substantial additional RAM (currently
+ * ~8 bytes per node), but then should produce a smaller FST. */
+ public FST<T> pack(int minInCountDeref, int maxDerefNodes) throws IOException {
+
+ // TODO: other things to try
+ // - renumber the nodes to get more next / better locality?
+ // - allow multiple input labels on an arc, so
+ // singular chain of inputs can take one arc (on
+ // wikipedia terms this could save another ~6%)
+ // - in the ord case, the output '1' is presumably
+ // very common (after NO_OUTPUT)... maybe use a bit
+ // for it..?
+ // - use spare bits in flags.... for top few labels /
+ // outputs / targets
+
+ if (nodeAddress == null) {
+ throw new IllegalArgumentException("this FST was not built with willPackFST=true");
+ }
+
+ Arc<T> arc = new Arc<T>();
+
+ final BytesReader r = getBytesReader(0);
+
+ final int topN = Math.min(maxDerefNodes, inCounts.length);
+
+ // Find top nodes with highest number of incoming arcs:
+ NodeQueue q = new NodeQueue(topN);
+
+ NodeAndInCount bottom = null;
+ for(int node=0;node<inCounts.length;node++) {
+ if (inCounts[node] >= minInCountDeref) {
+ if (bottom == null) {
+ q.add(new NodeAndInCount(node, inCounts[node]));
+ if (q.size() == topN) {
+ bottom = q.top();
+ }
+ } else if (inCounts[node] > bottom.count) {
+ q.insertWithOverflow(new NodeAndInCount(node, inCounts[node]));
+ }
+ }
+ }
+
+ // Free up RAM:
+ inCounts = null;
+
+ final Map<Integer,Integer> topNodeMap = new HashMap<Integer,Integer>();
+ for(int downTo=q.size()-1;downTo>=0;downTo--) {
+ NodeAndInCount n = q.pop();
+ topNodeMap.put(n.node, downTo);
+ //System.out.println("map node=" + n.node + " inCount=" + n.count + " to newID=" + downTo);
+ }
+
+ // TODO: we can use packed ints:
+ // +1 because node ords start at 1 (0 is reserved as
+ // stop node):
+ final int[] nodeRefToAddressIn = new int[topNodeMap.size()];
+
+ final FST<T> fst = new FST<T>(inputType, nodeRefToAddressIn, outputs);
+
+ final BytesWriter writer = fst.writer;
+
+ final int[] newNodeAddress = new int[1+nodeCount];
+
+ // Fill initial coarse guess:
+ for(int node=1;node<=nodeCount;node++) {
+ newNodeAddress[node] = 1 + bytes.length - nodeAddress[node];
+ }
+
+ int absCount;
+ int deltaCount;
+ int topCount;
+ int nextCount;
+
+ // Iterate until we converge:
+ while(true) {
+
+ //System.out.println("\nITER");
+ boolean changed = false;
+
+ // for assert:
+ boolean negDelta = false;
+
+ writer.posWrite = 0;
+ // Skip 0 byte since 0 is reserved target:
+ writer.writeByte((byte) 0);
+
+ fst.arcWithOutputCount = 0;
+ fst.nodeCount = 0;
+ fst.arcCount = 0;
+
+ absCount = deltaCount = topCount = nextCount = 0;
+
+ int changedCount = 0;
+
+ int addressError = 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--) {
+ fst.nodeCount++;
+ final int address = writer.posWrite;
+ //System.out.println(" node: " + node + " address=" + address);
+ if (address != newNodeAddress[node]) {
+ addressError = address - newNodeAddress[node];
+ //System.out.println(" change: " + (address - newNodeAddress[node]));
+ changed = true;
+ newNodeAddress[node] = address;
+ changedCount++;
+ }
+
+ int nodeArcCount = 0;
+ int bytesPerArc = 0;
+
+ boolean retry = false;
+
+ // for assert:
+ boolean anyNegDelta = false;
+
+ // Retry loop: possibly iterate more than once, if
+ // this is an array'd node and bytesPerArc changes:
+ writeNode:
+ while(true) { // retry writing this node
+
+ readFirstRealTargetArc(node, arc, r);
+
+ final boolean useArcArray = arc.bytesPerArc != 0;
+ if (useArcArray) {
+ // Write false first arc:
+ if (bytesPerArc == 0) {
+ bytesPerArc = arc.bytesPerArc;
+ }
+ writer.writeByte(ARCS_AS_FIXED_ARRAY);
+ writer.writeVInt(arc.numArcs);
+ writer.writeVInt(bytesPerArc);
+ }
+
+ int maxBytesPerArc = 0;
+
+ while(true) { // iterate over all arcs for this node
+
+ //System.out.println(" arc label=" + arc.label + " target=" + arc.target + " pos=" + writer.posWrite);
+ final int arcStartPos = writer.posWrite;
+ nodeArcCount++;
+
+ byte flags = 0;
+
+ if (arc.isLast()) {
+ flags += BIT_LAST_ARC;
+ }
+ /*
+ if (!useArcArray && nodeUpto < nodes.length-1 && arc.target == nodes[nodeUpto+1]) {
+ flags += BIT_TARGET_NEXT;
+ }
+ */
+ if (!useArcArray && node != 1 && arc.target == node-1) {
+ flags += BIT_TARGET_NEXT;
+ if (!retry) {
+ nextCount++;
+ }
+ }
+ if (arc.isFinal()) {
+ flags += BIT_FINAL_ARC;
+ if (arc.nextFinalOutput != NO_OUTPUT) {
+ flags += BIT_ARC_HAS_FINAL_OUTPUT;
+ }
+ } else {
+ assert arc.nextFinalOutput == NO_OUTPUT;
+ }
+ if (!targetHasArcs(arc)) {
+ flags += BIT_STOP_NODE;
+ }
+
+ if (arc.output != NO_OUTPUT) {
+ flags += BIT_ARC_HAS_OUTPUT;
+ }
+
+ final Integer ptr;
+ final int absPtr;
+ final boolean doWriteTarget = targetHasArcs(arc) && (flags & BIT_TARGET_NEXT) == 0;
+ if (doWriteTarget) {
+
+ ptr = topNodeMap.get(arc.target);
+ if (ptr != null) {
+ absPtr = ptr;
+ } else {
+ absPtr = topNodeMap.size() + newNodeAddress[arc.target] + addressError;
+ }
+
+ int delta = newNodeAddress[arc.target] + addressError - writer.posWrite - 2;
+ if (delta < 0) {
+ //System.out.println("neg: " + delta);
+ anyNegDelta = true;
+ delta = 0;
+ }
+
+ if (delta < absPtr) {
+ flags |= BIT_TARGET_DELTA;
+ }
+ } else {
+ ptr = null;
+ absPtr = 0;
+ }
+
+ writer.writeByte(flags);
+ fst.writeLabel(arc.label);
+
+ if (arc.output != NO_OUTPUT) {
+ outputs.write(arc.output, writer);
+ if (!retry) {
+ fst.arcWithOutputCount++;
+ }
+ }
+ if (arc.nextFinalOutput != NO_OUTPUT) {
+ outputs.write(arc.nextFinalOutput, writer);
+ }
+
+ if (doWriteTarget) {
+
+ int delta = newNodeAddress[arc.target] + addressError - writer.posWrite;
+ if (delta < 0) {
+ anyNegDelta = true;
+ //System.out.println("neg: " + delta);
+ delta = 0;
+ }
+
+ if (flag(flags, BIT_TARGET_DELTA)) {
+ //System.out.println(" delta");
+ writer.writeVInt(delta);
+ if (!retry) {
+ deltaCount++;
+ }
+ } else {
+ /*
+ if (ptr != null) {
+ System.out.println(" deref");
+ } else {
+ System.out.println(" abs");
+ }
+ */
+ writer.writeVInt(absPtr);
+ if (!retry) {
+ if (absPtr >= topNodeMap.size()) {
+ absCount++;
+ } else {
+ topCount++;
+ }
+ }
+ }
+ }
+
+ if (useArcArray) {
+ final int arcBytes = writer.posWrite - arcStartPos;
+ //System.out.println(" " + arcBytes + " bytes");
+ maxBytesPerArc = Math.max(maxBytesPerArc, arcBytes);
+ // NOTE: this may in fact go "backwards", if
+ // somehow (rarely, possibly never) we use
+ // more bytesPerArc in this rewrite than the
+ // incoming FST did... but in this case we
+ // will retry (below) so it's OK to ovewrite
+ // bytes:
+ writer.setPosWrite(arcStartPos + bytesPerArc);
+ }
+
+ if (arc.isLast()) {
+ break;
+ }
+
+ readNextRealArc(arc, r);
+ }
+
+ if (useArcArray) {
+ if (maxBytesPerArc == bytesPerArc || (retry && maxBytesPerArc <= bytesPerArc)) {
+ // converged
+ break;
+ }
+ } else {
+ break;
+ }
+
+ //System.out.println(" retry this node maxBytesPerArc=" + maxBytesPerArc + " vs " + bytesPerArc);
+
+ // Retry:
+ bytesPerArc = maxBytesPerArc;
+ writer.posWrite = address;
+ nodeArcCount = 0;
+ retry = true;
+ anyNegDelta = false;
+ }
+ negDelta |= anyNegDelta;
+
+ fst.arcCount += nodeArcCount;
+ }
+
+ if (!changed) {
+ // We don't renumber the nodes (just reverse their
+ // order) so nodes should only point forward to
+ // other nodes because we only produce acyclic FSTs
+ // w/ nodes only pointing "forwards":
+ assert !negDelta;
+ // Converged!
+ break;
+ }
+ //System.out.println(" " + changedCount + " of " + fst.nodeCount + " changed; retry");
+ }
+
+ for(Map.Entry<Integer,Integer> ent : topNodeMap.entrySet()) {
+ nodeRefToAddressIn[ent.getValue()] = newNodeAddress[ent.getKey()];
+ }
+
+ fst.startNode = newNodeAddress[startNode];
+ //System.out.println("new startNode=" + startNode);
+
+ if (emptyOutput != null) {
+ fst.setEmptyOutput(emptyOutput);
+ }
+
+ 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.posWrite];
+ //System.out.println("resize " + fst.bytes.length + " down to " + writer.posWrite);
+ System.arraycopy(fst.bytes, 0, finalBytes, 0, writer.posWrite);
+ fst.bytes = finalBytes;
+ fst.cacheRootArcs();
+
+ //final int size = fst.sizeInBytes();
+ //System.out.println("nextCount=" + nextCount + " topCount=" + topCount + " deltaCount=" + deltaCount + " absCount=" + absCount);
+
+ return fst;
+ }
+
+ private static class NodeAndInCount implements Comparable<NodeAndInCount> {
+ final int node;
+ final int count;
+
+ public NodeAndInCount(int node, int count) {
+ this.node = node;
+ this.count = count;
+ }
+
+ @Override
+ public int compareTo(NodeAndInCount other) {
+ if (count > other.count) {
+ return 1;
+ } else if (count < other.count) {
+ return -1;
+ } else {
+ // Tie-break: smaller node compares as greater than
+ return other.node - node;
+ }
+ }
+ }
+
+ private static class NodeQueue extends PriorityQueue<NodeAndInCount> {
+ public NodeQueue(int topN) {
+ super(topN, false);
+ }
+
+ @Override
+ public boolean lessThan(NodeAndInCount a, NodeAndInCount b) {
+ final int cmp = a.compareTo(b);
+ assert cmp != 0;
+ return cmp < 0;
+ }
}
}
Modified: lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/FSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/FSTEnum.java?rev=1237505&r1=1237504&r2=1237505&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/FSTEnum.java (original)
+++ lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/FSTEnum.java Sun Jan 29 23:19:05 2012
@@ -151,7 +151,8 @@ abstract class FSTEnum<T> {
boolean found = false;
while (low <= high) {
mid = (low + high) >>> 1;
- in.pos = arc.posArcsStart - arc.bytesPerArc*mid - 1;
+ in.pos = arc.posArcsStart;
+ in.skip(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);
@@ -275,7 +276,7 @@ abstract class FSTEnum<T> {
// Now scan forward, matching the new suffix of the target
while(true) {
- //System.out.println(" cycle upto=" + upto + " arc.label=" + arc.label + " (" + (char) arc.label + ") targetLabel=" + targetLabel + " isLast?=" + arc.isLast());
+ //System.out.println(" cycle upto=" + upto + " arc.label=" + arc.label + " (" + (char) arc.label + ") targetLabel=" + targetLabel + " isLast?=" + arc.isLast() + " bba=" + arc.bytesPerArc);
if (arc.bytesPerArc != 0 && arc.label != FST.END_LABEL) {
// Arcs are fixed array -- use binary search to find
@@ -289,15 +290,16 @@ abstract class FSTEnum<T> {
boolean found = false;
while (low <= high) {
mid = (low + high) >>> 1;
- in.pos = arc.posArcsStart - arc.bytesPerArc*mid - 1;
+ in.pos = arc.posArcsStart;
+ in.skip(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);
- if (cmp < 0)
+ if (cmp < 0) {
low = mid + 1;
- else if (cmp > 0)
+ } else if (cmp > 0) {
high = mid - 1;
- else {
+ } else {
found = true;
break;
}
@@ -430,9 +432,11 @@ abstract class FSTEnum<T> {
FST.Arc<T> arc = getArc(upto-1);
int targetLabel = getTargetLabel();
+ final FST.BytesReader fstReader = fst.getBytesReader(0);
+
while(true) {
//System.out.println(" cycle target=" + (targetLabel == -1 ? "-1" : (char) targetLabel));
- final FST.Arc<T> nextArc = fst.findTargetArc(targetLabel, arc, getArc(upto));
+ final FST.Arc<T> nextArc = fst.findTargetArc(targetLabel, arc, getArc(upto), fstReader);
if (nextArc == null) {
// short circuit
//upto--;
Modified: lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/NodeHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/NodeHash.java?rev=1237505&r1=1237504&r2=1237505&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/NodeHash.java (original)
+++ lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/NodeHash.java Sun Jan 29 23:19:05 2012
@@ -35,7 +35,7 @@ final class NodeHash<T> {
}
private boolean nodesEqual(Builder.UnCompiledNode<T> node, int address, FST.BytesReader in) throws IOException {
- fst.readFirstRealArc(address, scratchArc, in);
+ fst.readFirstRealTargetArc(address, scratchArc, in);
if (scratchArc.bytesPerArc != 0 && node.numArcs != scratchArc.numArcs) {
return false;
}
@@ -43,7 +43,7 @@ final class NodeHash<T> {
final Builder.Arc<T> arc = node.arcs[arcUpto];
if (arc.label != scratchArc.label ||
!arc.output.equals(scratchArc.output) ||
- ((Builder.CompiledNode) arc.target).address != scratchArc.target ||
+ ((Builder.CompiledNode) arc.target).node != scratchArc.target ||
!arc.nextFinalOutput.equals(scratchArc.nextFinalOutput) ||
arc.isFinal != scratchArc.isFinal()) {
return false;
@@ -71,9 +71,9 @@ final class NodeHash<T> {
// TODO: maybe if number of arcs is high we can safely subsample?
for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
final Builder.Arc<T> arc = node.arcs[arcIdx];
- //System.out.println(" label=" + arc.label + " target=" + ((Builder.CompiledNode) arc.target).address + " h=" + h + " output=" + fst.outputs.outputToString(arc.output) + " isFinal?=" + arc.isFinal);
+ //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).address;
+ h = PRIME * h + ((Builder.CompiledNode) arc.target).node;
h = PRIME * h + arc.output.hashCode();
h = PRIME * h + arc.nextFinalOutput.hashCode();
if (arc.isFinal) {
@@ -88,9 +88,9 @@ final class NodeHash<T> {
private int hash(int node) throws IOException {
final int PRIME = 31;
final FST.BytesReader in = fst.getBytesReader(0);
- //System.out.println("hash frozen");
+ //System.out.println("hash frozen node=" + node);
int h = 0;
- fst.readFirstRealArc(node, scratchArc, in);
+ 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());
h = PRIME * h + scratchArc.label;
@@ -109,26 +109,26 @@ final class NodeHash<T> {
return h & Integer.MAX_VALUE;
}
- public int add(Builder.UnCompiledNode<T> node) throws IOException {
+ 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);
- final int h = hash(node);
+ final int h = hash(nodeIn);
int pos = h & mask;
int c = 0;
while(true) {
final int v = table[pos];
if (v == 0) {
// freeze & add
- final int address = fst.addNode(node);
- //System.out.println(" now freeze addr=" + address);
- assert hash(address) == h : "frozenHash=" + hash(address) + " vs h=" + h;
+ final int node = fst.addNode(nodeIn);
+ //System.out.println(" now freeze node=" + node);
+ assert hash(node) == h : "frozenHash=" + hash(node) + " vs h=" + h;
count++;
- table[pos] = address;
+ table[pos] = node;
if (table.length < 2*count) {
rehash();
}
- return address;
- } else if (nodesEqual(node, v, in)) {
+ return node;
+ } else if (nodesEqual(nodeIn, v, in)) {
// same node is already here
return v;
}
Modified: lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/Outputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/Outputs.java?rev=1237505&r1=1237504&r2=1237505&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/Outputs.java (original)
+++ lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/Outputs.java Sun Jan 29 23:19:05 2012
@@ -26,6 +26,10 @@ import org.apache.lucene.store.DataOutpu
* Represents the outputs for an FST, providing the basic
* algebra needed for the FST.
*
+ * <p>Note that any operation that returns NO_OUTPUT must
+ * return the same singleton object from {@link
+ * #getNoOutput}.</p>
+ *
* @lucene.experimental
*/
@@ -56,6 +60,8 @@ public abstract class Outputs<T> {
public abstract String outputToString(T output);
+ // TODO: maybe make valid(T output) public...? for asserts
+
public T merge(T first, T second) {
throw new UnsupportedOperationException();
}
Modified: lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/PairOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/PairOutputs.java?rev=1237505&r1=1237504&r2=1237505&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/PairOutputs.java (original)
+++ lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/PairOutputs.java Sun Jan 29 23:19:05 2012
@@ -38,7 +38,8 @@ public class PairOutputs<A,B> extends Ou
public final A output1;
public final B output2;
- public Pair(A output1, B output2) {
+ // use newPair
+ private Pair(A output1, B output2) {
this.output1 = output1;
this.output2 = output2;
}
@@ -66,35 +67,79 @@ public class PairOutputs<A,B> extends Ou
this.outputs2 = outputs2;
NO_OUTPUT = new Pair<A,B>(outputs1.getNoOutput(), outputs2.getNoOutput());
}
-
- public Pair<A,B> get(A output1, B output2) {
- if (output1 == outputs1.getNoOutput() && output2 == outputs2.getNoOutput()) {
+
+ /** Create a new Pair */
+ public Pair<A,B> newPair(A a, B b) {
+ if (a.equals(outputs1.getNoOutput())) {
+ a = outputs1.getNoOutput();
+ }
+ if (b.equals(outputs2.getNoOutput())) {
+ b = outputs2.getNoOutput();
+ }
+
+ if (a == outputs1.getNoOutput() && b == outputs2.getNoOutput()) {
return NO_OUTPUT;
} else {
- return new Pair<A,B>(output1, output2);
+ final Pair<A,B> p = new Pair<A,B>(a, b);
+ assert valid(p);
+ return p;
}
}
-
+
+ // for assert
+ private boolean valid(Pair<A,B> pair) {
+ final boolean noOutput1 = pair.output1.equals(outputs1.getNoOutput());
+ final boolean noOutput2 = pair.output2.equals(outputs2.getNoOutput());
+
+ if (noOutput1 && pair.output1 != outputs1.getNoOutput()) {
+ System.out.println("invalid0");
+ return false;
+ }
+
+ if (noOutput2 && pair.output2 != outputs2.getNoOutput()) {
+ System.out.println("invalid1");
+ return false;
+ }
+
+ if (noOutput1 && noOutput2) {
+ if (pair != NO_OUTPUT) {
+ System.out.println("invalid2");
+ return false;
+ } else {
+ return true;
+ }
+ } else {
+ return true;
+ }
+ }
+
@Override
public Pair<A,B> common(Pair<A,B> pair1, Pair<A,B> pair2) {
- return get(outputs1.common(pair1.output1, pair2.output1),
- outputs2.common(pair1.output2, pair2.output2));
+ assert valid(pair1);
+ assert valid(pair2);
+ return newPair(outputs1.common(pair1.output1, pair2.output1),
+ outputs2.common(pair1.output2, pair2.output2));
}
@Override
public Pair<A,B> subtract(Pair<A,B> output, Pair<A,B> inc) {
- return get(outputs1.subtract(output.output1, inc.output1),
- outputs2.subtract(output.output2, inc.output2));
+ assert valid(output);
+ assert valid(inc);
+ return newPair(outputs1.subtract(output.output1, inc.output1),
+ outputs2.subtract(output.output2, inc.output2));
}
@Override
public Pair<A,B> add(Pair<A,B> prefix, Pair<A,B> output) {
- return get(outputs1.add(prefix.output1, output.output1),
- outputs2.add(prefix.output2, output.output2));
+ assert valid(prefix);
+ assert valid(output);
+ return newPair(outputs1.add(prefix.output1, output.output1),
+ outputs2.add(prefix.output2, output.output2));
}
@Override
public void write(Pair<A,B> output, DataOutput writer) throws IOException {
+ assert valid(output);
outputs1.write(output.output1, writer);
outputs2.write(output.output2, writer);
}
@@ -103,7 +148,7 @@ public class PairOutputs<A,B> extends Ou
public Pair<A,B> read(DataInput in) throws IOException {
A output1 = outputs1.read(in);
B output2 = outputs2.read(in);
- return get(output1, output2);
+ return newPair(output1, output2);
}
@Override
@@ -113,6 +158,12 @@ public class PairOutputs<A,B> extends Ou
@Override
public String outputToString(Pair<A,B> output) {
+ assert valid(output);
return "<pair:" + outputs1.outputToString(output.output1) + "," + outputs2.outputToString(output.output2) + ">";
}
+
+ @Override
+ public String toString() {
+ return "PairOutputs<" + outputs1 + "," + outputs2 + ">";
+ }
}
Modified: lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java?rev=1237505&r1=1237504&r2=1237505&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java (original)
+++ lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java Sun Jan 29 23:19:05 2012
@@ -25,10 +25,7 @@ import org.apache.lucene.store.DataOutpu
/**
* Output is a long, for each input term. NOTE: the
* resulting FST is not guaranteed to be minimal! See
- * {@link Builder}. You must use {@link #get} to obtain the
- * output for a given long value -- do not use autoboxing
- * nor create your own Long instance (the value 0
- * must map to the {@link #getNoOutput} singleton).
+ * {@link Builder}.
*
* @lucene.experimental
*/
@@ -50,14 +47,6 @@ public final class PositiveIntOutputs ex
return doShare ? singletonShare : singletonNoShare;
}
- public Long get(long v) {
- if (v == 0) {
- return NO_OUTPUT;
- } else {
- return Long.valueOf(v);
- }
- }
-
@Override
public Long common(Long output1, Long output2) {
assert valid(output1);
Modified: lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/Util.java?rev=1237505&r1=1237504&r2=1237505&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/Util.java (original)
+++ lucene/dev/branches/lucene2858/lucene/src/java/org/apache/lucene/util/fst/Util.java Sun Jan 29 23:19:05 2012
@@ -37,23 +37,21 @@ 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);
+
// Accumulate output as we go
- final T NO_OUTPUT = fst.outputs.getNoOutput();
- T output = NO_OUTPUT;
+ T output = fst.outputs.getNoOutput();
for(int i=0;i<input.length;i++) {
- if (fst.findTargetArc(input.ints[input.offset + i], arc, arc) == null) {
+ if (fst.findTargetArc(input.ints[input.offset + i], arc, arc, fstReader) == null) {
return null;
- } else if (arc.output != NO_OUTPUT) {
- output = fst.outputs.add(output, arc.output);
}
+ output = fst.outputs.add(output, arc.output);
}
- if (fst.findTargetArc(FST.END_LABEL, arc, arc) == null) {
- return null;
- } else if (arc.output != NO_OUTPUT) {
- return fst.outputs.add(output, arc.output);
+ if (arc.isFinal()) {
+ return fst.outputs.add(output, arc.nextFinalOutput);
} else {
- return output;
+ return null;
}
}
@@ -64,26 +62,24 @@ 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);
+
// TODO: would be nice not to alloc this on every lookup
final FST.Arc<T> arc = fst.getFirstArc(new FST.Arc<T>());
// Accumulate output as we go
- final T NO_OUTPUT = fst.outputs.getNoOutput();
- T output = NO_OUTPUT;
+ T output = fst.outputs.getNoOutput();
for(int i=0;i<input.length;i++) {
- if (fst.findTargetArc(input.bytes[i+input.offset] & 0xFF, arc, arc) == null) {
+ if (fst.findTargetArc(input.bytes[i+input.offset] & 0xFF, arc, arc, fstReader) == null) {
return null;
- } else if (arc.output != NO_OUTPUT) {
- output = fst.outputs.add(output, arc.output);
}
+ output = fst.outputs.add(output, arc.output);
}
- if (fst.findTargetArc(FST.END_LABEL, arc, arc) == null) {
- return null;
- } else if (arc.output != NO_OUTPUT) {
- return fst.outputs.add(output, arc.output);
+ if (arc.isFinal()) {
+ return fst.outputs.add(output, arc.nextFinalOutput);
} else {
- return output;
+ return null;
}
}
@@ -142,7 +138,7 @@ public final class Util {
result.grow(1+upto);
}
- fst.readFirstRealArc(arc.target, arc, in);
+ fst.readFirstRealTargetArc(arc.target, arc, in);
FST.Arc<Long> prevArc = null;
@@ -238,6 +234,7 @@ public final class Util {
// A queue of transitions to consider when processing the next level.
final List<FST.Arc<T>> nextLevelQueue = new ArrayList<FST.Arc<T>>();
nextLevelQueue.add(startArc);
+ //System.out.println("toDot: startArc: " + startArc);
// A list of states on the same level (for ranking).
final List<Integer> sameLevelStates = new ArrayList<Integer>();
@@ -289,8 +286,11 @@ public final class Util {
int level = 0;
+ final FST.BytesReader r = fst.getBytesReader(0);
+
while (!nextLevelQueue.isEmpty()) {
// we could double buffer here, but it doesn't matter probably.
+ //System.out.println("next level=" + level);
thisLevelQueue.addAll(nextLevelQueue);
nextLevelQueue.clear();
@@ -298,19 +298,19 @@ public final class Util {
out.write("\n // Transitions and states at level: " + level + "\n");
while (!thisLevelQueue.isEmpty()) {
final FST.Arc<T> arc = thisLevelQueue.remove(thisLevelQueue.size() - 1);
+ //System.out.println(" pop: " + arc);
if (fst.targetHasArcs(arc)) {
- // scan all arcs
+ // scan all target arcs
+ //System.out.println(" readFirstTarget...");
final int node = arc.target;
- fst.readFirstTargetArc(arc, arc);
- if (arc.label == FST.END_LABEL) {
- // Skip it -- prior recursion took this into account already
- assert !arc.isLast();
- fst.readNextArc(arc);
- }
+ fst.readFirstRealTargetArc(arc.target, arc, r);
+
+ //System.out.println(" firstTarget: " + arc);
while (true) {
+ //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)) {
@@ -329,7 +329,7 @@ public final class Util {
if (fst.isExpandedTarget(arc)) {
stateColor = expandedNodeColor;
} else {
- stateColor = null;
+ stateColor = null;
}
final String finalOutput;
@@ -339,7 +339,9 @@ public final class Util {
finalOutput = "";
}
- emitDotState(out, Integer.toString(arc.target), arc.isFinal() ? finalStateShape : stateShape, stateColor, finalOutput);
+ emitDotState(out, Integer.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);
nextLevelQueue.add(new FST.Arc<T>().copyFrom(arc));
sameLevelStates.add(arc.target);
@@ -362,14 +364,22 @@ public final class Util {
outs = outs + "/[" + fst.outputs.outputToString(arc.nextFinalOutput) + "]";
}
+ final String arcColor;
+ if (arc.flag(FST.BIT_TARGET_NEXT)) {
+ arcColor = "red";
+ } else {
+ arcColor = "black";
+ }
+
assert arc.label != FST.END_LABEL;
- out.write(" " + node + " -> " + arc.target + " [label=\"" + printableLabel(arc.label) + outs + "\"]\n");
+ out.write(" " + node + " -> " + arc.target + " [label=\"" + printableLabel(arc.label) + outs + "\"" + (arc.isFinal() ? " style=\"bold\"" : "" ) + " color=\"" + arcColor + "\"]\n");
// Break the loop if we're on the last arc of this state.
if (arc.isLast()) {
+ //System.out.println(" break");
break;
}
- fst.readNextArc(arc);
+ fst.readNextRealArc(arc, r);
}
}
}
Modified: lucene/dev/branches/lucene2858/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1237505&r1=1237504&r2=1237505&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/branches/lucene2858/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java Sun Jan 29 23:19:05 2012
@@ -703,12 +703,6 @@ public class MockDirectoryWrapper extend
}
@Override
- public synchronized long fileModified(String name) throws IOException {
- maybeYield();
- return delegate.fileModified(name);
- }
-
- @Override
public synchronized long fileLength(String name) throws IOException {
maybeYield();
return delegate.fileLength(name);
Modified: lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java?rev=1237505&r1=1237504&r2=1237505&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java (original)
+++ lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java Sun Jan 29 23:19:05 2012
@@ -228,14 +228,6 @@ public class TestCrashCausesCorruptIndex
* {@inheritDoc}
*/
@Override
- public long fileModified(String name) throws IOException {
- return realDirectory.fileModified(name);
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
public String[] listAll() throws IOException {
return realDirectory.listAll();
}
Modified: lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/index/TestDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/index/TestDeletionPolicy.java?rev=1237505&r1=1237504&r2=1237505&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/index/TestDeletionPolicy.java (original)
+++ lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/index/TestDeletionPolicy.java Sun Jan 29 23:19:05 2012
@@ -18,10 +18,12 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
+import java.util.Map;
import java.util.Set;
-import java.util.Collection;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
@@ -32,7 +34,6 @@ import org.apache.lucene.search.IndexSea
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.similarities.DefaultSimilarity;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
@@ -47,20 +48,12 @@ public class TestDeletionPolicy extends
final IndexCommit firstCommit = commits.get(0);
long last = SegmentInfos.generationFromSegmentsFileName(firstCommit.getSegmentsFileName());
assertEquals(last, firstCommit.getGeneration());
- long lastVersion = firstCommit.getVersion();
- long lastTimestamp = firstCommit.getTimestamp();
for(int i=1;i<commits.size();i++) {
final IndexCommit commit = commits.get(i);
long now = SegmentInfos.generationFromSegmentsFileName(commit.getSegmentsFileName());
- long nowVersion = commit.getVersion();
- long nowTimestamp = commit.getTimestamp();
assertTrue("SegmentInfos commits are out-of-order", now > last);
- assertTrue("SegmentInfos versions are out-of-order", nowVersion > lastVersion);
- assertTrue("SegmentInfos timestamps are out-of-order: now=" + nowTimestamp + " vs last=" + lastTimestamp, nowTimestamp >= lastTimestamp);
assertEquals(now, commit.getGeneration());
last = now;
- lastVersion = nowVersion;
- lastTimestamp = nowTimestamp;
}
}
@@ -158,6 +151,10 @@ public class TestDeletionPolicy extends
}
}
+ static long getCommitTime(IndexCommit commit) throws IOException {
+ return Long.parseLong(commit.getUserData().get("commitTime"));
+ }
+
/*
* Delete a commit only when it has been obsoleted by N
* seconds.
@@ -184,10 +181,10 @@ public class TestDeletionPolicy extends
IndexCommit lastCommit = commits.get(commits.size()-1);
// Any commit older than expireTime should be deleted:
- double expireTime = dir.fileModified(lastCommit.getSegmentsFileName())/1000.0 - expirationTimeSeconds;
+ double expireTime = getCommitTime(lastCommit)/1000.0 - expirationTimeSeconds;
for (final IndexCommit commit : commits) {
- double modTime = dir.fileModified(commit.getSegmentsFileName())/1000.0;
+ double modTime = getCommitTime(commit)/1000.0;
if (commit != lastCommit && modTime < expireTime) {
commit.delete();
numDelete += 1;
@@ -213,6 +210,9 @@ public class TestDeletionPolicy extends
((LogMergePolicy) mp).setUseCompoundFile(true);
}
IndexWriter writer = new IndexWriter(dir, conf);
+ Map<String,String> commitData = new HashMap<String,String>();
+ commitData.put("commitTime", String.valueOf(System.currentTimeMillis()));
+ writer.commit(commitData);
writer.close();
final int ITER = 9;
@@ -233,6 +233,9 @@ public class TestDeletionPolicy extends
for(int j=0;j<17;j++) {
addDoc(writer);
}
+ commitData = new HashMap<String,String>();
+ commitData.put("commitTime", String.valueOf(System.currentTimeMillis()));
+ writer.commit(commitData);
writer.close();
if (i < ITER-1) {
@@ -269,7 +272,9 @@ public class TestDeletionPolicy extends
// if we are on a filesystem that seems to have only
// 1 second resolution, allow +1 second in commit
// age tolerance:
- long modTime = dir.fileModified(fileName);
+ SegmentInfos sis = new SegmentInfos();
+ sis.read(dir, fileName);
+ long modTime = Long.parseLong(sis.getUserData().get("commitTime"));
oneSecondResolution &= (modTime % 1000) == 0;
final long leeway = (long) ((SECONDS + (oneSecondResolution ? 1.0:0.0))*1000);
Modified: lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java?rev=1237505&r1=1237504&r2=1237505&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java (original)
+++ lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java Sun Jan 29 23:19:05 2012
@@ -126,10 +126,6 @@ public class TestFieldsReader extends Lu
return fsDir.fileExists(name);
}
@Override
- public long fileModified(String name) throws IOException {
- return fsDir.fileModified(name);
- }
- @Override
public void deleteFile(String name) throws IOException {
fsDir.deleteFile(name);
}
Modified: lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/index/TestIndexCommit.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/index/TestIndexCommit.java?rev=1237505&r1=1237504&r2=1237505&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/index/TestIndexCommit.java (original)
+++ lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/index/TestIndexCommit.java Sun Jan 29 23:19:05 2012
@@ -34,12 +34,10 @@ public class TestIndexCommit extends Luc
IndexCommit ic1 = new IndexCommit() {
@Override public String getSegmentsFileName() { return "a"; }
- @Override public long getVersion() { return 12; }
@Override public Directory getDirectory() { return dir; }
@Override public Collection<String> getFileNames() throws IOException { return null; }
@Override public void delete() {}
@Override public long getGeneration() { return 0; }
- @Override public long getTimestamp() throws IOException { return 1;}
@Override public Map<String, String> getUserData() throws IOException { return null; }
@Override public boolean isDeleted() { return false; }
@Override public int getSegmentCount() { return 2; }
@@ -47,12 +45,10 @@ public class TestIndexCommit extends Luc
IndexCommit ic2 = new IndexCommit() {
@Override public String getSegmentsFileName() { return "b"; }
- @Override public long getVersion() { return 12; }
@Override public Directory getDirectory() { return dir; }
@Override public Collection<String> getFileNames() throws IOException { return null; }
@Override public void delete() {}
@Override public long getGeneration() { return 0; }
- @Override public long getTimestamp() throws IOException { return 1;}
@Override public Map<String, String> getUserData() throws IOException { return null; }
@Override public boolean isDeleted() { return false; }
@Override public int getSegmentCount() { return 2; }
Modified: lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/index/TestIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/index/TestIndexReader.java?rev=1237505&r1=1237504&r2=1237505&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/index/TestIndexReader.java (original)
+++ lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/index/TestIndexReader.java Sun Jan 29 23:19:05 2012
@@ -381,60 +381,6 @@ public class TestIndexReader extends Luc
_TestUtil.rmDir(dirFile);
}
- public void testLastModified() throws Exception {
- for(int i=0;i<2;i++) {
- final Directory dir = newDirectory();
- assertFalse(DirectoryReader.indexExists(dir));
- IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.CREATE));
- addDocumentWithFields(writer);
- assertTrue(IndexWriter.isLocked(dir)); // writer open, so dir is locked
- writer.close();
- assertTrue(DirectoryReader.indexExists(dir));
- DirectoryReader reader = DirectoryReader.open(dir);
- assertFalse(IndexWriter.isLocked(dir)); // reader only, no lock
- long version = DirectoryReader.lastModified(dir);
- if (i == 1) {
- long version2 = DirectoryReader.lastModified(dir);
- assertEquals(version, version2);
- }
- reader.close();
- // modify index and check version has been
- // incremented:
- Thread.sleep(1000);
-
- writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.CREATE));
- addDocumentWithFields(writer);
- writer.close();
- reader = DirectoryReader.open(dir);
- assertTrue("old lastModified is " + version + "; new lastModified is " + DirectoryReader.lastModified(dir), version <= DirectoryReader.lastModified(dir));
- reader.close();
- dir.close();
- }
- }
-
- public void testVersion() throws IOException {
- Directory dir = newDirectory();
- assertFalse(DirectoryReader.indexExists(dir));
- IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)));
- addDocumentWithFields(writer);
- assertTrue(IndexWriter.isLocked(dir)); // writer open, so dir is locked
- writer.close();
- assertTrue(DirectoryReader.indexExists(dir));
- DirectoryReader reader = DirectoryReader.open(dir);
- assertFalse(IndexWriter.isLocked(dir)); // reader only, no lock
- long version = DirectoryReader.getCurrentVersion(dir);
- reader.close();
- // modify index and check version has been
- // incremented:
- writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.CREATE));
- addDocumentWithFields(writer);
- writer.close();
- reader = DirectoryReader.open(dir);
- assertTrue("old version is " + version + "; new version is " + DirectoryReader.getCurrentVersion(dir), version < DirectoryReader.getCurrentVersion(dir));
- reader.close();
- dir.close();
- }
-
public void testOpenReaderAfterDelete() throws IOException {
File dirFile = _TestUtil.getTempDir("deletetest");
Directory dir = newFSDirectory(dirFile);
Modified: lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java?rev=1237505&r1=1237504&r2=1237505&view=diff
==============================================================================
--- lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java (original)
+++ lucene/dev/branches/lucene2858/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java Sun Jan 29 23:19:05 2012
@@ -348,12 +348,6 @@ public class TestBufferedIndexInput exte
dir.deleteFile(name);
}
@Override
- public long fileModified(String name)
- throws IOException
- {
- return dir.fileModified(name);
- }
- @Override
public boolean fileExists(String name)
throws IOException
{