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/18 15:05:39 UTC
svn commit: r1435141 [2/2] - in /lucene/dev/branches/branch_4x: ./ lucene/
lucene/analysis/
lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/
lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/
lucene/analysis/kurom...
Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java?rev=1435141&r1=1435140&r2=1435141&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java Fri Jan 18 14:05:37 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);
}
Copied: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/ReverseBytesReader.java (from r1432459, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ReverseBytesReader.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/ReverseBytesReader.java?p2=lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/ReverseBytesReader.java&p1=lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ReverseBytesReader.java&r1=1432459&r2=1435141&rev=1435141&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ReverseBytesReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/ReverseBytesReader.java Fri Jan 18 14:05:37 2013
@@ -44,13 +44,13 @@ final class ReverseBytesReader extends F
}
@Override
- public int getPosition() {
+ public long getPosition() {
return pos;
}
@Override
- public void setPosition(int pos) {
- this.pos = pos;
+ public void setPosition(long pos) {
+ this.pos = (int) pos;
}
@Override
Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java?rev=1435141&r1=1435140&r2=1435141&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java Fri Jan 18 14:05:37 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();
// 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();
// 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();
// 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;
@@ -285,7 +285,7 @@ public final class Util {
public TopNSearcher(FST<T> fst, int topN, int maxQueueDepth, Comparator<T> comparator) {
this.fst = fst;
- this.bytesReader = fst.getBytesReader(0);
+ this.bytesReader = fst.getBytesReader();
this.topN = topN;
this.maxQueueDepth = maxQueueDepth;
this.comparator = comparator;
@@ -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();
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();
// 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/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1435141&r1=1435140&r2=1435141&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Fri Jan 18 14:05:37 2013
@@ -58,7 +58,6 @@ import org.apache.lucene.search.ScoreDoc
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.BaseDirectoryWrapper;
import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@@ -108,7 +107,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.
@@ -124,10 +123,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",
- "41.cfs",
- "41.nocfs",
};
final String[] unsupportedNames = {"19.cfs",
@@ -147,7 +176,7 @@ public class TestBackwardsCompatibility
};
final static String[] oldSingleSegmentNames = {"40.optimized.cfs",
- "40.optimized.nocfs",
+ "40.optimized.nocfs",
};
static Map<String,Directory> oldIndexDirs;
@@ -910,4 +939,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();
+ }
}
Copied: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java (from r1432459, lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java?p2=lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java&p1=lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java&r1=1432459&r2=1435141&rev=1435141&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestBytesStore.java Fri Jan 18 14:05:37 2013
@@ -42,7 +42,7 @@ public class TestBytesStore extends Luce
int pos = 0;
while(pos < numBytes) {
- int op = random().nextInt(7);
+ int op = random().nextInt(8);
if (VERBOSE) {
System.out.println(" cycle pos=" + pos);
}
@@ -97,21 +97,21 @@ public class TestBytesStore extends Luce
case 3:
{
// reverse bytes
- if (pos > 0) {
- int len = _TestUtil.nextInt(random(), 1, Math.min(100, pos));
+ if (pos > 1) {
+ int len = _TestUtil.nextInt(random(), 2, Math.min(100, pos));
int start;
if (len == pos) {
start = 0;
} else {
start = random().nextInt(pos - len);
}
- int end = start + len;
+ int end = start + len - 1;
if (VERBOSE) {
- System.out.println(" reverse start=" + start + " end=" + end + " len=" + len);
+ System.out.println(" reverse start=" + start + " end=" + end + " len=" + len + " pos=" + pos);
}
bytes.reverse(start, end);
- while(start < end) {
+ while(start <= end) {
byte b = expected[end];
expected[end] = expected[start];
expected[start] = b;
@@ -159,17 +159,49 @@ public class TestBytesStore extends Luce
{
// skip
int len = random().nextInt(Math.min(100, numBytes - pos));
- pos += len;
- bytes.skip(len);
+
if (VERBOSE) {
System.out.println(" skip len=" + len);
}
+
+ pos += len;
+ bytes.skipBytes(len);
+
+ // NOTE: must fill in zeros in case truncate was
+ // used, else we get false fails:
+ if (len > 0) {
+ byte[] zeros = new byte[len];
+ bytes.writeBytes(pos-len, zeros, 0, len);
+ }
}
break;
+
+ case 7:
+ {
+ // absWriteByte
+ if (pos > 0) {
+ int dest = random().nextInt(pos);
+ byte b = (byte) random().nextInt(256);
+ expected[dest] = b;
+ bytes.writeByte(dest, b);
+ }
+ break;
+ }
}
assertEquals(pos, bytes.getPosition());
+ if (pos > 0 && random().nextInt(50) == 17) {
+ // truncate
+ int len = _TestUtil.nextInt(random(), 1, Math.min(pos, 100));
+ bytes.truncate(pos - len);
+ pos -= len;
+ Arrays.fill(expected, pos, pos+len, (byte) 0);
+ if (VERBOSE) {
+ System.out.println(" truncate len=" + len + " newPos=" + pos);
+ }
+ }
+
if ((pos > 0 && random().nextInt(200) == 17)) {
verify(bytes, expected, pos);
}
@@ -289,10 +321,10 @@ public class TestBytesStore extends Luce
if (reversed) {
expectedPos = pos-numBytes;
- left = r.getPosition();
+ left = (int) r.getPosition();
} else {
expectedPos = pos+numBytes;
- left = totalLength - r.getPosition();
+ left = (int) (totalLength - r.getPosition());
}
assertEquals(expectedPos, r.getPosition());
Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java?rev=1435141&r1=1435140&r2=1435141&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java Fri Jan 18 14:05:37 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();
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);
@@ -1157,12 +1168,12 @@ public class TestFSTs extends LuceneTest
assertEquals(nothing, startArc.nextFinalOutput);
FST.Arc<Long> arc = fst.readFirstTargetArc(startArc, new FST.Arc<Long>(),
- fst.getBytesReader(0));
+ fst.getBytesReader());
assertEquals('a', arc.label);
assertEquals(17, arc.nextFinalOutput.longValue());
assertTrue(arc.isFinal());
- arc = fst.readNextArc(arc, fst.getBytesReader(0));
+ arc = fst.readNextArc(arc, fst.getBytesReader());
assertEquals('b', arc.label);
assertFalse(arc.isFinal());
assertEquals(42, arc.output.longValue());
@@ -1292,7 +1303,7 @@ public class TestFSTs extends LuceneTest
//Util.toDot(fst, w, false, false);
//w.close();
- BytesReader reader = fst.getBytesReader(0);
+ BytesReader reader = fst.getBytesReader();
//System.out.println("testing: " + allPrefixes.size() + " prefixes");
for (String prefix : allPrefixes) {
@@ -1413,7 +1424,7 @@ public class TestFSTs extends LuceneTest
//Util.toDot(fst, w, false, false);
//w.close();
- BytesReader reader = fst.getBytesReader(0);
+ BytesReader reader = fst.getBytesReader();
//System.out.println("testing: " + allPrefixes.size() + " prefixes");
for (String prefix : allPrefixes) {
Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java?rev=1435141&r1=1435140&r2=1435141&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java Fri Jan 18 14:05:37 2013
@@ -587,7 +587,7 @@ public class AnalyzingSuggester extends
//System.out.println(" prefixPaths: " + prefixPaths.size());
- BytesReader bytesReader = fst.getBytesReader(0);
+ BytesReader bytesReader = fst.getBytesReader();
FST.Arc<Pair<Long,BytesRef>> scratchArc = new FST.Arc<Pair<Long,BytesRef>>();
Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java?rev=1435141&r1=1435140&r2=1435141&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java Fri Jan 18 14:05:37 2013
@@ -77,7 +77,7 @@ public class FSTUtil {
new IntsRef()));
final FST.Arc<T> scratchArc = new FST.Arc<T>();
- final FST.BytesReader fstReader = fst.getBytesReader(0);
+ final FST.BytesReader fstReader = fst.getBytesReader();
while (queue.size() != 0) {
final Path<T> path = queue.remove(queue.size() - 1);
Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletion.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletion.java?rev=1435141&r1=1435140&r2=1435141&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletion.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletion.java Fri Jan 18 14:05:37 2013
@@ -139,7 +139,7 @@ public class FSTCompletion {
try {
List<Arc<Object>> rootArcs = new ArrayList<Arc<Object>>();
Arc<Object> arc = automaton.getFirstArc(new Arc<Object>());
- FST.BytesReader fstReader = automaton.getBytesReader(0);
+ FST.BytesReader fstReader = automaton.getBytesReader();
automaton.readFirstTargetArc(arc, arc, fstReader);
while (true) {
rootArcs.add(new Arc<Object>().copyFrom(arc));
@@ -173,7 +173,7 @@ public class FSTCompletion {
// Get the UTF-8 bytes representation of the input key.
try {
final FST.Arc<Object> scratch = new FST.Arc<Object>();
- FST.BytesReader fstReader = automaton.getBytesReader(0);
+ FST.BytesReader fstReader = automaton.getBytesReader();
for (; rootArcIndex < rootArcs.length; rootArcIndex++) {
final FST.Arc<Object> rootArc = rootArcs[rootArcIndex];
final FST.Arc<Object> arc = scratch.copyFrom(rootArc);
@@ -338,7 +338,7 @@ public class FSTCompletion {
final int max = utf8.offset + utf8.length;
// Cannot save as instance var since multiple threads
// can use FSTCompletion at once...
- final FST.BytesReader fstReader = automaton.getBytesReader(0);
+ final FST.BytesReader fstReader = automaton.getBytesReader();
for (int i = utf8.offset; i < max; i++) {
if (automaton.findTargetArc(utf8.bytes[i] & 0xff, arc, arc, fstReader) == null) {
// No matching prefixes, return an empty result.
@@ -362,7 +362,7 @@ public class FSTCompletion {
}
assert output.offset == 0;
output.bytes[output.length++] = (byte) arc.label;
- FST.BytesReader fstReader = automaton.getBytesReader(0);
+ FST.BytesReader fstReader = automaton.getBytesReader();
automaton.readFirstTargetArc(arc, arc, fstReader);
while (true) {
if (arc.label == FST.END_LABEL) {
Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java?rev=1435141&r1=1435140&r2=1435141&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java Fri Jan 18 14:05:37 2013
@@ -24,6 +24,7 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefIterator;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.fst.*;
+import org.apache.lucene.util.packed.PackedInts;
/**
* Finite state automata based implementation of "autocomplete" functionality.
@@ -237,7 +238,8 @@ public class FSTCompletionBuilder {
final Object empty = outputs.getNoOutput();
final Builder<Object> builder = new Builder<Object>(
FST.INPUT_TYPE.BYTE1, 0, 0, true, true,
- shareMaxTailLength, outputs, null, false, true);
+ shareMaxTailLength, outputs, null, false,
+ PackedInts.DEFAULT, true, 15);
BytesRef scratch = new BytesRef();
BytesRef entry;
Modified: lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java?rev=1435141&r1=1435140&r2=1435141&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java (original)
+++ lucene/dev/branches/branch_4x/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java Fri Jan 18 14:05:37 2013
@@ -200,7 +200,7 @@ public class WFSTCompletionLookup extend
private Long lookupPrefix(BytesRef scratch, Arc<Long> arc) throws /*Bogus*/IOException {
assert 0 == fst.outputs.getNoOutput().longValue();
long output = 0;
- BytesReader bytesReader = fst.getBytesReader(0);
+ BytesReader bytesReader = fst.getBytesReader();
fst.getFirstArc(arc);
Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java?rev=1435141&r1=1435140&r2=1435141&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java Fri Jan 18 14:05:37 2013
@@ -40,6 +40,7 @@ import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.UnicodeUtil;
import org.apache.lucene.util._TestUtil;
+import org.apache.lucene.util.packed.PackedInts;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
@@ -203,7 +204,7 @@ public class FSTTester<T> {
final FST.Arc<T> arc = fst.getFirstArc(new FST.Arc<T>());
final T NO_OUTPUT = fst.outputs.getNoOutput();
T output = NO_OUTPUT;
- final FST.BytesReader fstReader = fst.getBytesReader(0);
+ final FST.BytesReader fstReader = fst.getBytesReader();
for(int i=0;i<=term.length;i++) {
final int label;
@@ -240,7 +241,7 @@ public class FSTTester<T> {
in.offset = 0;
final T NO_OUTPUT = fst.outputs.getNoOutput();
T output = NO_OUTPUT;
- final FST.BytesReader fstReader = fst.getBytesReader(0);
+ final FST.BytesReader fstReader = fst.getBytesReader();
while(true) {
// read all arcs:
@@ -288,7 +289,16 @@ public class FSTTester<T> {
outputs,
null,
willRewrite,
- true);
+ PackedInts.DEFAULT,
+ true,
+ 15);
+ if (LuceneTestCase.VERBOSE) {
+ if (willRewrite) {
+ System.out.println("TEST: packed FST");
+ } else {
+ System.out.println("TEST: non-packed FST");
+ }
+ }
for(InputOutput<T> pair : pairs) {
if (pair.output instanceof List) {